From 0e4d49570bf862bb598caf61f2b968f9da04cf8c Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 5 Sep 2017 20:13:00 +0300 Subject: [PATCH 001/108] LibraryDictionarySource fixes --- dbms/src/Dictionaries/LibraryDictionarySource.h | 2 +- .../tests/external_dictionaries/generate_and_test.py | 12 ++++++------ debian/source/format | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.h b/dbms/src/Dictionaries/LibraryDictionarySource.h index 108bae5cd65..fdde72ab7cc 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.h +++ b/dbms/src/Dictionaries/LibraryDictionarySource.h @@ -15,7 +15,7 @@ class Logger; namespace DB { -struct CStringsHolder; +class CStringsHolder; /// Allows loading dictionaries from dynamic libraries (.so) /// Experimental version diff --git a/dbms/tests/external_dictionaries/generate_and_test.py b/dbms/tests/external_dictionaries/generate_and_test.py index a51bf9a1f17..097b152b56e 100755 --- a/dbms/tests/external_dictionaries/generate_and_test.py +++ b/dbms/tests/external_dictionaries/generate_and_test.py @@ -389,16 +389,16 @@ def generate_dictionaries(args): '''.format(https_host=args.https_host, https_port=args.https_port, https_path=args.https_path) source_library = ''' - - {filename} - + + {filename} + '''.format(filename=os.path.abspath('../../../build/dbms/tests/external_dictionaries/dictionary_library/libdictionary_library.so')) # Todo? #source_library_c = ''' - # - # {filename} - # + # + # {filename} + # #'''.format(filename=os.path.abspath('../../../build/dbms/tests/external_dictionaries/dict_lib/libdict_library_c.so')) diff --git a/debian/source/format b/debian/source/format index 89ae9db8f88..163aaf8d82b 100644 --- a/debian/source/format +++ b/debian/source/format @@ -1 +1 @@ -3.0 (native) +3.0 (quilt) From 44b44a13d2f94efc9f81c581ef5af17e128fb195 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 5 Sep 2017 21:53:28 +0300 Subject: [PATCH 002/108] Fix http_head test (#1203) * Fix http_head test * Longer timeout --- dbms/tests/queries/0_stateless/00501_http_head.reference | 0 dbms/tests/queries/0_stateless/00501_http_head.sh | 4 ++++ 2 files changed, 4 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00501_http_head.reference create mode 100755 dbms/tests/queries/0_stateless/00501_http_head.sh diff --git a/dbms/tests/queries/0_stateless/00501_http_head.reference b/dbms/tests/queries/0_stateless/00501_http_head.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00501_http_head.sh b/dbms/tests/queries/0_stateless/00501_http_head.sh new file mode 100755 index 00000000000..4b99c841c45 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00501_http_head.sh @@ -0,0 +1,4 @@ +#!/usr/bin/env bash + +curl -s --max-time 0.5 -X "HEAD" 'http://localhost:8123/?query=SELECT%201' +curl -s --max-time 0.5 -X "HEAD" 'http://localhost:8123/?query=select+*+from+system.numbers+limit+1000000' From 3876340aebe4a1a418388f3e8d67c61474ff18c0 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 30 Aug 2017 17:47:35 +0300 Subject: [PATCH 003/108] More correct Context deinitialization. Part 1. [#CLICKHOUSE-3117] --- dbms/src/Server/Server.cpp | 38 ++++++++++++++++++++------------------ 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index e7820e296de..5130d612193 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -126,6 +126,15 @@ int Server::main(const std::vector & args) StatusFile status{path + "status"}; + SCOPE_EXIT({ + /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. + * At this moment, no one could own shared part of Context. + */ + global_context.reset(); + + LOG_DEBUG(log, "Destroyed global context."); + }); + /// Try to increase limit on number of open files. { rlimit rlim; @@ -265,6 +274,17 @@ int Server::main(const std::vector & args) global_context->setCurrentDatabase(default_database); + SCOPE_EXIT({ + /** Ask to cancel background jobs all table engines, + * and also query_log. + * It is important to do early, not in destructor of Context, because + * table engines could use Context on destroy. + */ + LOG_INFO(log, "Shutting down storages."); + global_context->shutdown(); + LOG_DEBUG(log, "Shutted down storages."); + }); + bool has_resharding_worker = false; if (has_zookeeper && config().has("resharding")) { @@ -281,24 +301,6 @@ int Server::main(const std::vector & args) global_context->setDDLWorker(std::make_shared(ddl_zookeeper_path, *global_context, &config(), "distributed_ddl")); } - SCOPE_EXIT({ - /** Ask to cancel background jobs all table engines, - * and also query_log. - * It is important to do early, not in destructor of Context, because - * table engines could use Context on destroy. - */ - LOG_INFO(log, "Shutting down storages."); - global_context->shutdown(); - LOG_DEBUG(log, "Shutted down storages."); - - /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. - * At this moment, no one could own shared part of Context. - */ - global_context.reset(); - - LOG_DEBUG(log, "Destroyed global context."); - }); - { Poco::Timespan keep_alive_timeout(config().getInt("keep_alive_timeout", 10), 0); From d55d5eca46a8ffc1d3207955e559f87ecfdb0ee6 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Mon, 4 Sep 2017 20:49:39 +0300 Subject: [PATCH 004/108] Remove extra Context reference and add required ones. Part 2. [#CLICKHOUSE-3117] --- .../AddingDefaultBlockInputStream.h | 2 +- .../AddingDefaultBlockOutputStream.h | 2 +- .../PushingToViewsBlockOutputStream.h | 2 +- dbms/src/Interpreters/Context.cpp | 19 +++++++------------ dbms/src/Interpreters/Context.h | 7 +++---- dbms/src/Interpreters/DDLWorker.cpp | 6 +++--- dbms/src/Interpreters/DDLWorker.h | 2 +- dbms/src/Interpreters/InterpreterAlterQuery.h | 2 +- .../src/Interpreters/InterpreterInsertQuery.h | 2 +- dbms/src/Interpreters/SystemLog.h | 1 + dbms/src/Storages/IStorage.h | 2 +- .../Storages/MergeTree/ReshardingWorker.cpp | 2 +- .../src/Storages/MergeTree/ReshardingWorker.h | 10 ++++++---- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageDistributed.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 2 +- .../src/Storages/StorageReplicatedMergeTree.h | 2 +- 17 files changed, 32 insertions(+), 35 deletions(-) diff --git a/dbms/src/DataStreams/AddingDefaultBlockInputStream.h b/dbms/src/DataStreams/AddingDefaultBlockInputStream.h index d34d8b1adf4..b5be6742371 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockInputStream.h +++ b/dbms/src/DataStreams/AddingDefaultBlockInputStream.h @@ -59,7 +59,7 @@ protected: private: NamesAndTypesListPtr required_columns; const ColumnDefaults column_defaults; - Context context; + cosnt Context & context; }; } diff --git a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h index e8f1cb30b80..b9955792f41 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h +++ b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h @@ -40,7 +40,7 @@ private: BlockOutputStreamPtr output; NamesAndTypesListPtr required_columns; const ColumnDefaults column_defaults; - Context context; + const Context & context; bool only_explicit_column_defaults; }; diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index dba48d34cdf..21198825a46 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -72,7 +72,7 @@ public: private: StoragePtr storage; BlockOutputStreamPtr output; - Context context; + const Context & context; ASTPtr query_ptr; std::vector> views; }; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index ce12cc7ce7b..cf379b83dd9 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -262,7 +262,7 @@ Context::~Context() } -InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; } +InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; } std::unique_lock Context::getLock() const { @@ -271,8 +271,8 @@ std::unique_lock Context::getLock() const return std::unique_lock(shared->mutex); } -ProcessList & Context::getProcessList() { return shared->process_list; } -const ProcessList & Context::getProcessList() const { return shared->process_list; } +ProcessList & Context::getProcessList() { return shared->process_list; } +const ProcessList & Context::getProcessList() const { return shared->process_list; } MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } @@ -1100,12 +1100,7 @@ void Context::setProcessListElement(ProcessList::Element * elem) process_list_elem = elem; } -ProcessList::Element * Context::getProcessListElement() -{ - return process_list_elem; -} - -const ProcessList::Element * Context::getProcessListElement() const +ProcessList::Element * Context::getProcessListElement() const { return process_list_elem; } @@ -1190,7 +1185,7 @@ void Context::setReshardingWorker(std::shared_ptr resharding_w shared->resharding_worker = resharding_worker; } -ReshardingWorker & Context::getReshardingWorker() +ReshardingWorker & Context::getReshardingWorker() const { auto lock = getLock(); if (!shared->resharding_worker) @@ -1207,11 +1202,11 @@ void Context::setDDLWorker(std::shared_ptr ddl_worker) shared->ddl_worker = ddl_worker; } -DDLWorker & Context::getDDLWorker() +DDLWorker & Context::getDDLWorker() const { auto lock = getLock(); if (!shared->ddl_worker) - throw Exception("DDL background thread not initialized.", ErrorCodes::LOGICAL_ERROR); + throw Exception("DDL background thread is not initialized.", ErrorCodes::LOGICAL_ERROR); return *shared->ddl_worker; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 9611de279ec..33087ff71b0 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -270,8 +270,7 @@ public: */ void setProcessListElement(ProcessListElement * elem); /// Can return nullptr if the query was not inserted into the ProcessList. - ProcessListElement * getProcessListElement(); - const ProcessListElement * getProcessListElement() const; + ProcessListElement * getProcessListElement() const; /// List all queries. ProcessList & getProcessList(); @@ -307,10 +306,10 @@ public: BackgroundProcessingPool & getBackgroundPool(); void setReshardingWorker(std::shared_ptr resharding_worker); - ReshardingWorker & getReshardingWorker(); + ReshardingWorker & getReshardingWorker() const; void setDDLWorker(std::shared_ptr ddl_worker); - DDLWorker & getDDLWorker(); + DDLWorker & getDDLWorker() const; Clusters & getClusters() const; std::shared_ptr getCluster(const std::string & cluster_name) const; diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 1519caf5763..481e7045bbe 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -932,7 +932,7 @@ class DDLQueryStatusInputSream : public IProfilingBlockInputStream { public: - DDLQueryStatusInputSream(const String & zk_node_path, const DDLLogEntry & entry, Context & context) + DDLQueryStatusInputSream(const String & zk_node_path, const DDLLogEntry & entry, const Context & context) : node_path(zk_node_path), context(context), watch(CLOCK_MONOTONIC_COARSE), log(&Logger::get("DDLQueryStatusInputSream")) { sample = Block{ @@ -1072,7 +1072,7 @@ private: private: String node_path; - Context & context; + const Context & context; Stopwatch watch; Logger * log; @@ -1087,7 +1087,7 @@ private: }; -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, Context & context) +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context) { ASTPtr query_ptr; diff --git a/dbms/src/Interpreters/DDLWorker.h b/dbms/src/Interpreters/DDLWorker.h index f84f7ccba8b..f9c296d373a 100644 --- a/dbms/src/Interpreters/DDLWorker.h +++ b/dbms/src/Interpreters/DDLWorker.h @@ -18,7 +18,7 @@ struct DDLLogEntry; struct DDLTask; -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, Context & context); +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context); class DDLWorker diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.h b/dbms/src/Interpreters/InterpreterAlterQuery.h index 753037072c7..4fd718cd781 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.h +++ b/dbms/src/Interpreters/InterpreterAlterQuery.h @@ -120,7 +120,7 @@ private: ASTPtr query_ptr; - Context context; + const Context & context; static void parseAlter(const ASTAlterQuery::ParameterContainer & params, AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands); diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.h b/dbms/src/Interpreters/InterpreterInsertQuery.h index 7b9ccbd99ac..f353d0c59fd 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.h +++ b/dbms/src/Interpreters/InterpreterInsertQuery.h @@ -30,7 +30,7 @@ private: Block getSampleBlock(); ASTPtr query_ptr; - Context context; + const Context & context; }; diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index 7b9f43a26a5..1df54449136 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -202,6 +202,7 @@ void SystemLog::threadFunction() if (element.first) { /// Shutdown. + /// NOTE: MergeTree engine can write data even it is already in shutdown state. flush(); break; } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index a8d07d07e43..473792376c6 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -261,7 +261,7 @@ public: const Field & partition, const WeightedZooKeeperPaths & weighted_zookeeper_paths, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, - Context & context) + const Context & context) { throw Exception("Method reshardPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp index d0a93780162..bc1f866c8b8 100644 --- a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp +++ b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp @@ -320,7 +320,7 @@ std::string computeHashFromPartition(const std::string & data_path, const std::s /// ReshardingWorker::ReshardingWorker(const Poco::Util::AbstractConfiguration & config, - const std::string & config_name, Context & context_) + const std::string & config_name, const Context & context_) : context{context_}, get_zookeeper{[&]() { return context.getZooKeeper(); }} { Arguments arguments(config, config_name); diff --git a/dbms/src/Storages/MergeTree/ReshardingWorker.h b/dbms/src/Storages/MergeTree/ReshardingWorker.h index d39354ec6bf..ef0c14e16a0 100644 --- a/dbms/src/Storages/MergeTree/ReshardingWorker.h +++ b/dbms/src/Storages/MergeTree/ReshardingWorker.h @@ -2,10 +2,11 @@ #include #include -#include +#include #include #include +#include #include @@ -14,10 +15,10 @@ #include #include + namespace DB { -class Context; class Cluster; class StorageReplicatedMergeTree; @@ -42,7 +43,7 @@ public: public: ReshardingWorker(const Poco::Util::AbstractConfiguration & config, - const std::string & config_name, Context & context_); + const std::string & config_name, const Context & context_); ReshardingWorker(const ReshardingWorker &) = delete; ReshardingWorker & operator=(const ReshardingWorker &) = delete; @@ -385,7 +386,8 @@ private: std::string distributed_lock_path; std::string coordination_path; - Context & context; + /// Use own Context (not reference) to execute queries in isolated environment + Context context; Logger * log = &Logger::get("ReshardingWorker"); zkutil::EventPtr event = std::make_shared(); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index f389c57f8b8..7eb6e0b117e 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -273,7 +273,7 @@ void StorageDistributed::reshardPartitions( const Field & partition, const WeightedZooKeeperPaths & weighted_zookeeper_paths, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, - Context & context) + const Context & context) { auto & resharding_worker = context.getReshardingWorker(); if (!resharding_worker.isStarted()) diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index d9e17b3d644..cd986803434 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -79,7 +79,7 @@ public: const Field & partition, const WeightedZooKeeperPaths & weighted_zookeeper_paths, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, - Context & context) override; + const Context & context) override; /// From each replica, get a description of the corresponding local table. BlockInputStreams describe(const Context & context, const Settings & settings); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ef5728b9f04..f8e835d071d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3381,7 +3381,7 @@ void StorageReplicatedMergeTree::reshardPartitions( const Field & partition, const WeightedZooKeeperPaths & weighted_zookeeper_paths, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, - Context & context) + const Context & context) { auto & resharding_worker = context.getReshardingWorker(); if (!resharding_worker.isStarted()) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 06b45ae8f1d..4379d38983d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -146,7 +146,7 @@ public: const Field & partition, const WeightedZooKeeperPaths & weighted_zookeeper_paths, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, - Context & context) override; + const Context & context) override; /** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper. */ From 325ceb85b5de000411bab8287aaf2585a4b901c3 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Tue, 5 Sep 2017 17:16:07 +0300 Subject: [PATCH 005/108] Delete unusing files. [#CLICKHOUSE-2] --- .../AddingDefaultBlockInputStream.h | 65 ------------------- 1 file changed, 65 deletions(-) delete mode 100644 dbms/src/DataStreams/AddingDefaultBlockInputStream.h diff --git a/dbms/src/DataStreams/AddingDefaultBlockInputStream.h b/dbms/src/DataStreams/AddingDefaultBlockInputStream.h deleted file mode 100644 index b5be6742371..00000000000 --- a/dbms/src/DataStreams/AddingDefaultBlockInputStream.h +++ /dev/null @@ -1,65 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - - -namespace DB -{ - - -/** Adds missing columns to the block with default values. - * These columns are materialized (not constants). - */ -class AddingDefaultBlockInputStream : public IProfilingBlockInputStream -{ -public: - AddingDefaultBlockInputStream( - BlockInputStreamPtr input_, - NamesAndTypesListPtr required_columns_, - const ColumnDefaults & column_defaults_, - const Context & context_) - : required_columns(required_columns_), - column_defaults(column_defaults_), context(context_) - { - children.push_back(input_); - } - - String getName() const override { return "AddingDefault"; } - - String getID() const override - { - std::stringstream res; - res << "AddingDefault(" << children.back()->getID(); - - for (NamesAndTypesList::const_iterator it = required_columns->begin(); it != required_columns->end(); ++it) - res << ", " << it->name << ", " << it->type->getName(); - - res << ")"; - return res.str(); - } - -protected: - Block readImpl() override - { - Block res = children.back()->read(); - if (!res) - return res; - /** @todo if somehow block does not contain values for implicitly-defaulted columns that are prerequisites - * for explicitly-defaulted ones, exception will be thrown during evaluating such columns - * (implicitly-defaulted columns are evaluated on the line after following one. */ - evaluateMissingDefaults(res, *required_columns, column_defaults, context); - res.addDefaults(*required_columns); - return res; - } - -private: - NamesAndTypesListPtr required_columns; - const ColumnDefaults column_defaults; - cosnt Context & context; -}; - -} From d82b87aaefc48aa3c93f4c588820ab4f93e231fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 04:57:27 +0300 Subject: [PATCH 006/108] Fixed redundand code in better way [#CLICKHOUSE-3281]. --- .../Storages/MergeTree/MergeTreeDataPart.cpp | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 3af6d0d8a91..eb1adf9fe33 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -109,19 +109,19 @@ bool MergeTreeDataPartChecksums::read(ReadBuffer & in) readText(format_version, in); assertChar('\n', in); - if (format_version < 1 || format_version > 4) - throw Exception("Bad checksums format version: " + DB::toString(format_version), ErrorCodes::UNKNOWN_FORMAT); - - if (format_version == 1) - return false; - if (format_version == 2) - return read_v2(in); - if (format_version == 3) - return read_v3(in); - if (format_version == 4) - return read_v4(in); - - return false; + switch (format_version) + { + case 1: + return false; + case 2: + return read_v2(in); + case 3: + return read_v3(in); + case 4: + return read_v4(in); + default: + throw Exception("Bad checksums format version: " + DB::toString(format_version), ErrorCodes::UNKNOWN_FORMAT); + } } bool MergeTreeDataPartChecksums::read_v2(ReadBuffer & in) From 31b598e1e355346eb3b7c837b8def69fbcd0ddb3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:01:22 +0300 Subject: [PATCH 007/108] Fixed bad code [#CLICKHOUSE-3281]. --- dbms/src/Common/CombinedCardinalityEstimator.h | 11 ++++------- dbms/src/Common/HyperLogLogWithSmallSetOptimization.h | 4 ++-- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/dbms/src/Common/CombinedCardinalityEstimator.h b/dbms/src/Common/CombinedCardinalityEstimator.h index 94d21064a42..7390dde607f 100644 --- a/dbms/src/Common/CombinedCardinalityEstimator.h +++ b/dbms/src/Common/CombinedCardinalityEstimator.h @@ -228,6 +228,7 @@ private: if (getContainerType() != details::ContainerType::SMALL) throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); + CurrentMemoryTracker::alloc(sizeof(Medium)); auto tmp_medium = std::make_unique(); for (const auto & x : small) @@ -235,8 +236,6 @@ private: medium = tmp_medium.release(); setContainerType(details::ContainerType::MEDIUM); - - CurrentMemoryTracker::alloc(sizeof(medium)); } void toLarge() @@ -246,6 +245,7 @@ private: if ((container_type != details::ContainerType::SMALL) && (container_type != details::ContainerType::MEDIUM)) throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); + CurrentMemoryTracker::alloc(sizeof(Large)); auto tmp_large = std::make_unique(); if (container_type == details::ContainerType::SMALL) @@ -263,9 +263,6 @@ private: large = tmp_large.release(); setContainerType(details::ContainerType::LARGE); - - CurrentMemoryTracker::alloc(sizeof(large)); - } void NO_INLINE destroy() @@ -279,14 +276,14 @@ private: delete medium; medium = nullptr; - CurrentMemoryTracker::free(sizeof(medium)); + CurrentMemoryTracker::free(sizeof(Medium)); } else if (container_type == details::ContainerType::LARGE) { delete large; large = nullptr; - CurrentMemoryTracker::free(sizeof(large)); + CurrentMemoryTracker::free(sizeof(Large)); } } diff --git a/dbms/src/Common/HyperLogLogWithSmallSetOptimization.h b/dbms/src/Common/HyperLogLogWithSmallSetOptimization.h index 5296a606121..164a434900a 100644 --- a/dbms/src/Common/HyperLogLogWithSmallSetOptimization.h +++ b/dbms/src/Common/HyperLogLogWithSmallSetOptimization.h @@ -37,7 +37,7 @@ private: void toLarge() { - CurrentMemoryTracker::alloc(sizeof(large)); + CurrentMemoryTracker::alloc(sizeof(Large)); /// At the time of copying data from `tiny`, setting the value of `large` is still not possible (otherwise it will overwrite some data). Large * tmp_large = new Large; @@ -55,7 +55,7 @@ public: { delete large; - CurrentMemoryTracker::free(sizeof(large)); + CurrentMemoryTracker::free(sizeof(Large)); } } From 3b6dd67178a10574efa6ba92d83588de04bfbabc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:11:01 +0300 Subject: [PATCH 008/108] Fixed nullptr dereference [#CLICKHOUSE-3281]. --- dbms/src/Functions/FunctionsArray.cpp | 3 +-- dbms/src/Functions/FunctionsConditional.h | 6 +++--- dbms/src/Functions/FunctionsMath.h | 6 ++---- 3 files changed, 6 insertions(+), 9 deletions(-) diff --git a/dbms/src/Functions/FunctionsArray.cpp b/dbms/src/Functions/FunctionsArray.cpp index 0e2270053b1..9e756922400 100644 --- a/dbms/src/Functions/FunctionsArray.cpp +++ b/dbms/src/Functions/FunctionsArray.cpp @@ -2413,8 +2413,7 @@ void FunctionRange::executeImpl(Block & block, const ColumnNumbers & arguments, { throw Exception{ "Illegal column " + col->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN - }; + ErrorCodes::ILLEGAL_COLUMN}; } } diff --git a/dbms/src/Functions/FunctionsConditional.h b/dbms/src/Functions/FunctionsConditional.h index 38e0d134b0d..ad9dd4ac1c5 100644 --- a/dbms/src/Functions/FunctionsConditional.h +++ b/dbms/src/Functions/FunctionsConditional.h @@ -762,7 +762,7 @@ private: : makeNullableColumnIfNot(arg_else.column); } else - throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName() + throw Exception("Illegal column " + arg_cond.column->getName() + " of first argument of function " + getName() + ". Must be ColumnUInt8 or ColumnConstUInt8.", ErrorCodes::ILLEGAL_COLUMN); return true; @@ -802,7 +802,7 @@ private: : block.getByPosition(result).type->createColumn()->cloneResized(block.rows()); } else - throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName() + throw Exception("Illegal column " + arg_cond.column->getName() + " of first argument of function " + getName() + ". Must be ColumnUInt8 or ColumnConstUInt8.", ErrorCodes::ILLEGAL_COLUMN); return true; @@ -1058,7 +1058,7 @@ public: ErrorCodes::ILLEGAL_COLUMN); } else - throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName() + throw Exception("Illegal column " + arg_cond.column->getName() + " of first argument of function " + getName() + ". Must be ColumnUInt8 or ColumnConstUInt8.", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/dbms/src/Functions/FunctionsMath.h b/dbms/src/Functions/FunctionsMath.h index fea7266e379..60ee1fadaca 100644 --- a/dbms/src/Functions/FunctionsMath.h +++ b/dbms/src/Functions/FunctionsMath.h @@ -92,8 +92,7 @@ private: { throw Exception{ "Illegal type " + arg->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } }; @@ -158,8 +157,7 @@ private: { throw Exception{ "Illegal column " + arg->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN - }; + ErrorCodes::ILLEGAL_COLUMN}; } } }; From 452c40e85e67d9fcf7c5e3cf06d4d76b28689f87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:13:54 +0300 Subject: [PATCH 009/108] Fixed nullptr dereference [#CLICKHOUSE-3281]. --- dbms/src/Analyzers/TypeAndConstantInference.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Analyzers/TypeAndConstantInference.cpp b/dbms/src/Analyzers/TypeAndConstantInference.cpp index 47c14ed9bfe..548a0e50c58 100644 --- a/dbms/src/Analyzers/TypeAndConstantInference.cpp +++ b/dbms/src/Analyzers/TypeAndConstantInference.cpp @@ -356,7 +356,6 @@ void processHigherOrderFunction(const String & column_name, if (lambda && lambda->name == "lambda") { const DataTypeExpression * lambda_type = typeid_cast(types[i].get()); - const DataTypes & lambda_argument_types = lambda_type->getArgumentTypes(); if (!lambda_type) throw Exception("Logical error: IFunction::getLambdaArgumentTypes returned data type for lambda expression," @@ -369,6 +368,8 @@ void processHigherOrderFunction(const String & column_name, AnalyzeLambdas::LambdaParameters parameters = AnalyzeLambdas::extractLambdaParameters(lambda->arguments->children[0]); + const DataTypes & lambda_argument_types = lambda_type->getArgumentTypes(); + if (parameters.size() != lambda_argument_types.size()) throw Exception("DataTypeExpression for lambda function has different number of argument types than number of lambda parameters", ErrorCodes::LOGICAL_ERROR); From e1e0d517aa0529d37e207432ecfa77b70386a43b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:19:52 +0300 Subject: [PATCH 010/108] Fixed bad style in PerformanceTest [#CLICKHOUSE-3281]. --- dbms/src/Server/PerformanceTest.cpp | 171 +++++++++++----------------- dbms/src/Server/main.cpp | 4 +- 2 files changed, 71 insertions(+), 104 deletions(-) diff --git a/dbms/src/Server/PerformanceTest.cpp b/dbms/src/Server/PerformanceTest.cpp index b421a437e94..81499dc254a 100644 --- a/dbms/src/Server/PerformanceTest.cpp +++ b/dbms/src/Server/PerformanceTest.cpp @@ -5,8 +5,8 @@ #include #include -#include #include +#include #include #include @@ -15,8 +15,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -24,8 +24,8 @@ #include #include #include +#include #include -#include #include #include @@ -53,7 +53,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static String pad(size_t padding) { +static String pad(size_t padding) +{ return String(padding * 4, ' '); } @@ -81,7 +82,7 @@ public: template typename std::enable_if::value>::type set(const String key, T value) { - set(key, std::to_string(value), /*wrap= */false); + set(key, std::to_string(value), /*wrap= */ false); } void set(const String key, const std::vector & run_infos) @@ -101,7 +102,10 @@ public: content[key] = value; } - String asString() const { return asString(padding); } + String asString() const + { + return asString(padding); + } String asString(size_t padding) const { String repr = "{"; @@ -222,12 +226,12 @@ public: return !conditions_all_of.initialized_count && !conditions_any_of.initialized_count; } -#define DEFINE_REPORT_FUNC(FUNC_NAME, CONDITION) \ - void FUNC_NAME(UInt64 value) \ - { \ - conditions_all_of.report(value, conditions_all_of.CONDITION); \ - conditions_any_of.report(value, conditions_any_of.CONDITION); \ - } \ +#define DEFINE_REPORT_FUNC(FUNC_NAME, CONDITION) \ + void FUNC_NAME(UInt64 value) \ + { \ + conditions_all_of.report(value, conditions_all_of.CONDITION); \ + conditions_any_of.report(value, conditions_any_of.CONDITION); \ + } DEFINE_REPORT_FUNC(reportTotalTime, total_time_ms); DEFINE_REPORT_FUNC(reportRowsRead, rows_read); @@ -241,9 +245,7 @@ public: bool areFulfilled() const { - return - (conditions_all_of.initialized_count - && conditions_all_of.fulfilled_count >= conditions_all_of.initialized_count) + return (conditions_all_of.initialized_count && conditions_all_of.fulfilled_count >= conditions_all_of.initialized_count) || (conditions_any_of.initialized_count && conditions_any_of.fulfilled_count); } @@ -318,8 +320,8 @@ struct Stats result += "\n"; } result += FOUR_SPACES + "0.95: " + std::to_string(sampler.quantileInterpolated(95 / 100.0)) + "\n"; - result += FOUR_SPACES + "0.99: " + std::to_string(sampler.quantileInterpolated(99 / 100.0)) + "\n"; - result += FOUR_SPACES + "0.999: " + std::to_string(sampler.quantileInterpolated(99.9 / 100.)) + "\n"; + result += FOUR_SPACES + "0.99: " + std::to_string(sampler.quantileInterpolated(99 / 100.0)) + "\n"; + result += FOUR_SPACES + "0.999: " + std::to_string(sampler.quantileInterpolated(99.9 / 100.)) + "\n"; result += FOUR_SPACES + "0.9999: " + std::to_string(sampler.quantileInterpolated(99.99 / 100.)); return result; @@ -476,7 +478,7 @@ struct Stats } }; -double Stats::avg_rows_speed_precision = 0.001; +double Stats::avg_rows_speed_precision = 0.001; double Stats::avg_bytes_speed_precision = 0.001; class PerformanceTest @@ -484,8 +486,7 @@ class PerformanceTest public: using Strings = std::vector; - PerformanceTest( - const String & host_, + PerformanceTest(const String & host_, const UInt16 port_, const String & default_database_, const String & user_, @@ -498,8 +499,7 @@ public: Strings && tests_names_, Strings && skip_names_, Strings && tests_names_regexp_, - Strings && skip_names_regexp_ - ) + Strings && skip_names_regexp_) : connection(host_, port_, default_database_, user_, password_), gotSIGINT(false), lite_output(lite_output_), @@ -593,11 +593,9 @@ private: /// Removes configurations that has a given value. If leave is true, the logic is reversed. void removeConfigurationsIf( - std::vector & configs, - FilterType filter_type, const Strings & values, bool leave = false) + std::vector & configs, FilterType filter_type, const Strings & values, bool leave = false) { - auto checker = [&filter_type, &values, &leave](XMLConfigurationPtr & config) - { + auto checker = [&filter_type, &values, &leave](XMLConfigurationPtr & config) { if (values.size() == 0) return false; @@ -612,7 +610,8 @@ private: for (size_t i = 0; i != tags_keys.size(); ++i) tags[i] = config->getString("tags.tag[" + std::to_string(i) + "]"); - for (const String & config_tag : tags) { + for (const String & config_tag : tags) + { if (std::find(values.begin(), values.end(), config_tag) != values.end()) remove_or_not = true; } @@ -631,8 +630,7 @@ private: return std::regex_search(config_name, pattern); }; - remove_or_not = config->has("name") ? (std::find_if(values.begin(), values.end(), regex_checker) != values.end()) - : false; + remove_or_not = config->has("name") ? (std::find_if(values.begin(), values.end(), regex_checker) != values.end()) : false; } if (leave) @@ -672,7 +670,8 @@ private: for (const String & precondition : preconditions) { if (precondition == "flush_disk_cache") - if (system("(>&2 echo 'Flushing disk cache...') && (sudo sh -c 'echo 3 > /proc/sys/vm/drop_caches') && (>&2 echo 'Flushed.')")) + if (system( + "(>&2 echo 'Flushing disk cache...') && (sudo sh -c 'echo 3 > /proc/sys/vm/drop_caches') && (>&2 echo 'Flushed.')")) { std::cerr << "Failed to flush disk cache" << std::endl; return false; @@ -784,8 +783,7 @@ private: } void extractSettings( - const XMLConfigurationPtr & config, const String & key, - const Strings & settings_list, std::map & settings_to_apply) + const XMLConfigurationPtr & config, const String & key, const Strings & settings_list, std::map & settings_to_apply) { for (const String & setup : settings_list) { @@ -834,15 +832,15 @@ private: /// and, if found any settings in test's xml configuration /// with the same name, sets its value to settings std::map::iterator it; - #define EXTRACT_SETTING(TYPE, NAME, DEFAULT) \ - it = settings_to_apply.find(#NAME); \ - if (it != settings_to_apply.end()) \ - settings.set(#NAME, settings_to_apply[#NAME]); +#define EXTRACT_SETTING(TYPE, NAME, DEFAULT) \ + it = settings_to_apply.find(#NAME); \ + if (it != settings_to_apply.end()) \ + settings.set(#NAME, settings_to_apply[#NAME]); APPLY_FOR_SETTINGS(EXTRACT_SETTING) APPLY_FOR_LIMITS(EXTRACT_SETTING) - #undef EXTRACT_SETTING +#undef EXTRACT_SETTING if (std::find(config_settings.begin(), config_settings.end(), "average_rows_speed_precision") != config_settings.end()) { @@ -1074,10 +1072,8 @@ private: RemoteBlockInputStream stream(connection, query, global_context, &settings); - stream.setProgressCallback([&](const Progress & value) - { - this->checkFulfilledConditionsAndUpdate(value, stream, statistics, stop_conditions); - }); + stream.setProgressCallback( + [&](const Progress & value) { this->checkFulfilledConditionsAndUpdate(value, stream, statistics, stop_conditions); }); stream.readPrefix(); while (Block block = stream.read()) @@ -1091,10 +1087,7 @@ private: } void checkFulfilledConditionsAndUpdate( - const Progress & progress, - RemoteBlockInputStream & stream, - Stats & statistics, - TestStopConditions & stop_conditions) + const Progress & progress, RemoteBlockInputStream & stream, Stats & statistics, TestStopConditions & stop_conditions) { statistics.add(progress.rows, progress.bytes); @@ -1102,10 +1095,8 @@ private: stop_conditions.reportBytesReadUncompressed(statistics.total_bytes_read); stop_conditions.reportTotalTime(statistics.watch.elapsed() / (1000 * 1000)); stop_conditions.reportMinTimeNotChangingFor(statistics.min_time_watch.elapsed() / (1000 * 1000)); - stop_conditions.reportMaxSpeedNotChangingFor( - statistics.max_rows_speed_watch.elapsed() / (1000 * 1000)); - stop_conditions.reportAverageSpeedNotChangingFor( - statistics.avg_rows_speed_watch.elapsed() / (1000 * 1000)); + stop_conditions.reportMaxSpeedNotChangingFor(statistics.max_rows_speed_watch.elapsed() / (1000 * 1000)); + stop_conditions.reportAverageSpeedNotChangingFor(statistics.avg_rows_speed_watch.elapsed() / (1000 * 1000)); if (stop_conditions.areFulfilled()) { @@ -1128,8 +1119,7 @@ private: for (size_t i = 0; i != xml_substitutions.size(); ++i) { - const ConfigurationPtr xml_substitution( - substitutions_view->createView("substitution[" + std::to_string(i) + "]")); + const ConfigurationPtr xml_substitution(substitutions_view->createView("substitution[" + std::to_string(i) + "]")); /// Property values for substitution will be stored in a vector /// accessible by property name @@ -1273,7 +1263,6 @@ public: } - if (exec_type == ExecutionType::Loop) { /// in seconds @@ -1291,9 +1280,9 @@ public: quantiles.set(quantile_key, statistics.sampler.quantileInterpolated(percent / 100.0)); } - quantiles.set("0.95", statistics.sampler.quantileInterpolated(95 / 100.0)); - quantiles.set("0.99", statistics.sampler.quantileInterpolated(99 / 100.0)); - quantiles.set("0.999", statistics.sampler.quantileInterpolated(99.9 / 100.0)); + quantiles.set("0.95", statistics.sampler.quantileInterpolated(95 / 100.0)); + quantiles.set("0.99", statistics.sampler.quantileInterpolated(99 / 100.0)); + quantiles.set("0.999", statistics.sampler.quantileInterpolated(99.9 / 100.0)); quantiles.set("0.9999", statistics.sampler.quantileInterpolated(99.99 / 100.0)); runJSON.set("quantiles", quantiles.asString()); @@ -1303,16 +1292,13 @@ public: runJSON.set("total_time", statistics.total_time); if (std::find(metrics.begin(), metrics.end(), "queries_per_second") != metrics.end()) - runJSON.set("queries_per_second", double(statistics.queries) / - statistics.total_time); + runJSON.set("queries_per_second", double(statistics.queries) / statistics.total_time); if (std::find(metrics.begin(), metrics.end(), "rows_per_second") != metrics.end()) - runJSON.set("rows_per_second", double(statistics.total_rows_read) / - statistics.total_time); + runJSON.set("rows_per_second", double(statistics.total_rows_read) / statistics.total_time); if (std::find(metrics.begin(), metrics.end(), "bytes_per_second") != metrics.end()) - runJSON.set("bytes_per_second", double(statistics.total_bytes_read) / - statistics.total_time); + runJSON.set("bytes_per_second", double(statistics.total_bytes_read) / statistics.total_time); } else { @@ -1387,7 +1373,7 @@ static void getFilesFromDir(const FS::path & dir, std::vector & input_fi } } -int mainEntryClickhousePerformanceTest(int argc, char ** argv) +int mainEntryClickHousePerformanceTest(int argc, char ** argv) { using namespace DB; @@ -1397,22 +1383,17 @@ int mainEntryClickhousePerformanceTest(int argc, char ** argv) using Strings = std::vector; boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help", "produce help message") - ("lite", "use lite version of output") - ("profiles-file", value()->default_value(""), "Specify a file with global profiles") - ("host,h", value()->default_value("localhost"), "") - ("port", value()->default_value(9000), "") - ("database", value()->default_value("default"), "") - ("user", value()->default_value("default"), "") - ("password", value()->default_value(""), "") - ("tags", value()->multitoken(), "Run only tests with tag") - ("skip-tags", value()->multitoken(), "Do not run tests with tag") - ("names", value()->multitoken(), "Run tests with specific name") - ("skip-names", value()->multitoken(), "Do not run tests with name") - ("names-regexp", value()->multitoken(), "Run tests with names matching regexp") - ("skip-names-regexp", value()->multitoken(), "Do not run tests with names matching regexp") - ("recursive,r", "Recurse in directories to find all xml's"); + desc.add_options()("help", "produce help message")("lite", "use lite version of output")( + "profiles-file", value()->default_value(""), "Specify a file with global profiles")( + "host,h", value()->default_value("localhost"), "")("port", value()->default_value(9000), "")( + "database", value()->default_value("default"), "")("user", value()->default_value("default"), "")( + "password", value()->default_value(""), "")("tags", value()->multitoken(), "Run only tests with tag")( + "skip-tags", value()->multitoken(), "Do not run tests with tag")("names", + value()->multitoken(), + "Run tests with specific name")("skip-names", value()->multitoken(), "Do not run tests with name")( + "names-regexp", value()->multitoken(), "Run tests with names matching regexp")("skip-names-regexp", + value()->multitoken(), + "Do not run tests with names matching regexp")("recursive,r", "Recurse in directories to find all xml's"); /// These options will not be displayed in --help boost::program_options::options_description hidden("Hidden options"); @@ -1468,7 +1449,7 @@ int mainEntryClickhousePerformanceTest(int argc, char ** argv) if (FS::is_directory(file)) { - input_files.erase( std::remove(input_files.begin(), input_files.end(), filename) , input_files.end() ); + input_files.erase(std::remove(input_files.begin(), input_files.end(), filename), input_files.end()); getFilesFromDir(file, input_files, recursive); } else @@ -1479,27 +1460,14 @@ int mainEntryClickhousePerformanceTest(int argc, char ** argv) } } - Strings tests_tags = options.count("tags") - ? options["tags"].as() - : Strings({}); - Strings skip_tags = options.count("skip-tags") - ? options["skip-tags"].as() - : Strings({}); - Strings tests_names = options.count("names") - ? options["names"].as() - : Strings({}); - Strings skip_names = options.count("skip-names") - ? options["skip-names"].as() - : Strings({}); - Strings tests_names_regexp = options.count("names-regexp") - ? options["names-regexp"].as() - : Strings({}); - Strings skip_names_regexp = options.count("skip-names-regexp") - ? options["skip-names-regexp"].as() - : Strings({}); + Strings tests_tags = options.count("tags") ? options["tags"].as() : Strings({}); + Strings skip_tags = options.count("skip-tags") ? options["skip-tags"].as() : Strings({}); + Strings tests_names = options.count("names") ? options["names"].as() : Strings({}); + Strings skip_names = options.count("skip-names") ? options["skip-names"].as() : Strings({}); + Strings tests_names_regexp = options.count("names-regexp") ? options["names-regexp"].as() : Strings({}); + Strings skip_names_regexp = options.count("skip-names-regexp") ? options["skip-names-regexp"].as() : Strings({}); - PerformanceTest performanceTest( - options["host"].as(), + PerformanceTest performanceTest(options["host"].as(), options["port"].as(), options["database"].as(), options["user"].as(), @@ -1512,12 +1480,11 @@ int mainEntryClickhousePerformanceTest(int argc, char ** argv) std::move(tests_names), std::move(skip_names), std::move(tests_names_regexp), - std::move(skip_names_regexp) - ); + std::move(skip_names_regexp)); } catch (...) { - std::cout << getCurrentExceptionMessage(/*with stacktrace = */true) << std::endl; + std::cout << getCurrentExceptionMessage(/*with stacktrace = */ true) << std::endl; return getCurrentExceptionCode(); } diff --git a/dbms/src/Server/main.cpp b/dbms/src/Server/main.cpp index 62c411bfbdf..f1d0a7b4deb 100644 --- a/dbms/src/Server/main.cpp +++ b/dbms/src/Server/main.cpp @@ -11,7 +11,7 @@ int mainEntryClickHouseServer(int argc, char ** argv); int mainEntryClickHouseClient(int argc, char ** argv); int mainEntryClickHouseLocal(int argc, char ** argv); int mainEntryClickHouseBenchmark(int argc, char ** argv); -int mainEntryClickhousePerformanceTest(int argc, char ** argv); +int mainEntryClickHousePerformanceTest(int argc, char ** argv); int mainEntryClickHouseExtractFromConfig(int argc, char ** argv); static bool isClickhouseApp(const std::string & app_suffix, std::vector & argv) @@ -55,7 +55,7 @@ int main(int argc_, char ** argv_) else if (isClickhouseApp("server", argv)) /// --server arg should be cut main_func = mainEntryClickHouseServer; else if (isClickhouseApp("performance-test", argv)) - main_func = mainEntryClickhousePerformanceTest; + main_func = mainEntryClickHousePerformanceTest; else if (isClickhouseApp("extract-from-config", argv)) main_func = mainEntryClickHouseExtractFromConfig; From 9290d26c58d1fdc53b3471ac6f4b09165b5352d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:25:01 +0300 Subject: [PATCH 011/108] Removed some garbage from PerformanceTest [#CLICKHOUSE-3281]. --- dbms/src/Server/PerformanceTest.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/dbms/src/Server/PerformanceTest.cpp b/dbms/src/Server/PerformanceTest.cpp index 81499dc254a..5c12956b984 100644 --- a/dbms/src/Server/PerformanceTest.cpp +++ b/dbms/src/Server/PerformanceTest.cpp @@ -74,7 +74,7 @@ public: bool reserved = (value[0] == '[' || value[0] == '{' || value == "null"); if (!reserved && wrap) - value = '\"' + value + '\"'; + value = '"' + value + '"'; content[key] = value; } @@ -115,7 +115,7 @@ public: if (it != content.begin()) repr += ','; /// construct "key": "value" string with padding - repr += "\n" + pad(padding) + '\"' + it->first + '\"' + ": " + it->second; + repr += "\n" + pad(padding) + '"' + it->first + '"' + ": " + it->second; } repr += "\n" + pad(padding - 1) + '}'; @@ -380,7 +380,8 @@ struct Stats double & avg_speed_value) { avg_speed_value = ((avg_speed_value * number_of_info_batches) + new_speed_info); - avg_speed_value /= (++number_of_info_batches); + ++number_of_info_batches; + avg_speed_value /= number_of_info_batches; if (avg_speed_first == 0) { @@ -670,12 +671,14 @@ private: for (const String & precondition : preconditions) { if (precondition == "flush_disk_cache") + { if (system( "(>&2 echo 'Flushing disk cache...') && (sudo sh -c 'echo 3 > /proc/sys/vm/drop_caches') && (>&2 echo 'Flushed.')")) { std::cerr << "Failed to flush disk cache" << std::endl; return false; } + } if (precondition == "ram_size") { @@ -753,7 +756,7 @@ private: { if (!checkPreconditions(test_config)) { - std::cerr << "Preconditions are not fulfilled for test \"" + test_config->getString("name", "") + "\" "; + std::cerr << "Preconditions are not fulfilled for test '" + test_config->getString("name", "") + "' "; continue; } @@ -1222,7 +1225,7 @@ public: String array_string = "["; for (size_t i = 0; i != values.size(); ++i) { - array_string += '\"' + values[i] + '\"'; + array_string += '"' + values[i] + '"'; if (i != values.size() - 1) { array_string += ", "; @@ -1360,7 +1363,7 @@ public: static void getFilesFromDir(const FS::path & dir, std::vector & input_files, const bool recursive = false) { if (dir.extension().string() == ".xml") - std::cerr << "Warning: \"" + dir.string() + "\" is a directory, but has .xml extension" << std::endl; + std::cerr << "Warning: '" + dir.string() + "' is a directory, but has .xml extension" << std::endl; FS::directory_iterator end; for (FS::directory_iterator it(dir); it != end; ++it) @@ -1445,7 +1448,7 @@ int mainEntryClickHousePerformanceTest(int argc, char ** argv) FS::path file(filename); if (!FS::exists(file)) - throw DB::Exception("File \"" + filename + "\" does not exist", 1); + throw DB::Exception("File '" + filename + "' does not exist", 1); if (FS::is_directory(file)) { @@ -1455,7 +1458,7 @@ int mainEntryClickHousePerformanceTest(int argc, char ** argv) else { if (file.extension().string() != ".xml") - throw DB::Exception("File \"" + filename + "\" does not have .xml extension", 1); + throw DB::Exception("File '" + filename + "' does not have .xml extension", 1); } } } From 426da39acbbf5b0251e7d4d788df35b7dfe9016a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:33:24 +0300 Subject: [PATCH 012/108] Fixed bad code in PerformanceTest [#CLICKHOUSE-3281]. --- dbms/src/Server/PerformanceTest.cpp | 31 ++++++++++++++++------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/dbms/src/Server/PerformanceTest.cpp b/dbms/src/Server/PerformanceTest.cpp index 5c12956b984..36caebe9e41 100644 --- a/dbms/src/Server/PerformanceTest.cpp +++ b/dbms/src/Server/PerformanceTest.cpp @@ -39,7 +39,7 @@ * The tool walks through given or default folder in order to find files with * tests' descriptions and launches it. */ -namespace FS = boost::filesystem; +namespace fs = boost::filesystem; using String = std::string; const String FOUR_SPACES = " "; @@ -879,7 +879,7 @@ private: if (filename.empty()) throw DB::Exception("Empty file name", 1); - bool tsv = FS::path(filename).extension().string() == ".tsv"; + bool tsv = fs::path(filename).extension().string() == ".tsv"; ReadBufferFromFile query_file(filename); @@ -1360,18 +1360,18 @@ public: }; } -static void getFilesFromDir(const FS::path & dir, std::vector & input_files, const bool recursive = false) +static void getFilesFromDir(const fs::path & dir, std::vector & input_files, const bool recursive = false) { if (dir.extension().string() == ".xml") std::cerr << "Warning: '" + dir.string() + "' is a directory, but has .xml extension" << std::endl; - FS::directory_iterator end; - for (FS::directory_iterator it(dir); it != end; ++it) + fs::directory_iterator end; + for (fs::directory_iterator it(dir); it != end; ++it) { - const FS::path file = (*it); - if (recursive && FS::is_directory(file)) + const fs::path file = (*it); + if (recursive && fs::is_directory(file)) getFilesFromDir(file, input_files, recursive); - else if (!FS::is_directory(file) && file.extension().string() == ".xml") + else if (!fs::is_directory(file) && file.extension().string() == ".xml") input_files.push_back(file.string()); } } @@ -1427,7 +1427,7 @@ int mainEntryClickHousePerformanceTest(int argc, char ** argv) if (!options.count("input-files")) { std::cerr << "Trying to find test scenario files in the current folder..."; - FS::path curr_dir("."); + fs::path curr_dir("."); getFilesFromDir(curr_dir, input_files, recursive); @@ -1442,25 +1442,28 @@ int mainEntryClickHousePerformanceTest(int argc, char ** argv) else { input_files = options["input-files"].as(); + Strings collected_files; for (const String filename : input_files) { - FS::path file(filename); + fs::path file(filename); - if (!FS::exists(file)) + if (!fs::exists(file)) throw DB::Exception("File '" + filename + "' does not exist", 1); - if (FS::is_directory(file)) + if (fs::is_directory(file)) { - input_files.erase(std::remove(input_files.begin(), input_files.end(), filename), input_files.end()); - getFilesFromDir(file, input_files, recursive); + getFilesFromDir(file, collected_files, recursive); } else { if (file.extension().string() != ".xml") throw DB::Exception("File '" + filename + "' does not have .xml extension", 1); + collected_files.push_back(filename); } } + + input_files = std::move(collected_files); } Strings tests_tags = options.count("tags") ? options["tags"].as() : Strings({}); From 7237445356f7f89dc697ffa7afb5e47109cabdba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:36:00 +0300 Subject: [PATCH 013/108] Fixed another bad code in PerformanceTest [#CLICKHOUSE-3281]. --- dbms/src/Server/PerformanceTest.cpp | 221 ++++++++++++++-------------- 1 file changed, 109 insertions(+), 112 deletions(-) diff --git a/dbms/src/Server/PerformanceTest.cpp b/dbms/src/Server/PerformanceTest.cpp index 36caebe9e41..f33f1df9c04 100644 --- a/dbms/src/Server/PerformanceTest.cpp +++ b/dbms/src/Server/PerformanceTest.cpp @@ -152,7 +152,7 @@ struct StopConditionsSet else if (key == "average_speed_not_changing_for_ms") average_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); else - throw DB::Exception("Met unkown stop condition: " + key, 1); + throw DB::Exception("Met unkown stop condition: " + key); ++initialized_count; } @@ -860,12 +860,12 @@ private: if (!test_config->has("query") && !test_config->has("query_file")) { - throw DB::Exception("Missing query fields in test's config: " + test_name, 1); + throw DB::Exception("Missing query fields in test's config: " + test_name); } if (test_config->has("query") && test_config->has("query_file")) { - throw DB::Exception("Found both query and query_file fields. Choose only one", 1); + throw DB::Exception("Found both query and query_file fields. Choose only one"); } if (test_config->has("query")) @@ -877,7 +877,7 @@ private: { const String filename = test_config->getString("query_file"); if (filename.empty()) - throw DB::Exception("Empty file name", 1); + throw DB::Exception("Empty file name"); bool tsv = fs::path(filename).extension().string() == ".tsv"; @@ -901,13 +901,13 @@ private: if (queries.empty()) { - throw DB::Exception("Did not find any query to execute: " + test_name, 1); + throw DB::Exception("Did not find any query to execute: " + test_name); } if (test_config->has("substitutions")) { if (queries.size() > 1) - throw DB::Exception("Only one query is allowed when using substitutions", 1); + throw DB::Exception("Only one query is allowed when using substitutions"); /// Make "subconfig" of inner xml block ConfigurationPtr substitutions_view(test_config->createView("substitutions")); @@ -918,7 +918,7 @@ private: if (!test_config->has("type")) { - throw DB::Exception("Missing type property in config: " + test_name, 1); + throw DB::Exception("Missing type property in config: " + test_name); } String config_exec_type = test_config->getString("type"); @@ -927,9 +927,9 @@ private: else if (config_exec_type == "once") exec_type = ExecutionType::Once; else - throw DB::Exception("Unknown type " + config_exec_type + " in :" + test_name, 1); + throw DB::Exception("Unknown type " + config_exec_type + " in :" + test_name); - times_to_run = test_config->getUInt("times_to_run", 1); + times_to_run = test_config->getUInt("times_to_run"); stop_conditions_by_run.clear(); TestStopConditions stop_conditions_template; @@ -940,7 +940,7 @@ private: } if (stop_conditions_template.empty()) - throw DB::Exception("No termination conditions were found in config", 1); + throw DB::Exception("No termination conditions were found in config"); for (size_t i = 0; i < times_to_run * queries.size(); ++i) stop_conditions_by_run.push_back(stop_conditions_template); @@ -967,7 +967,7 @@ private: else { if (lite_output) - throw DB::Exception("Specify main_metric for lite output", 1); + throw DB::Exception("Specify main_metric for lite output"); } if (metrics.size() > 0) @@ -1015,7 +1015,7 @@ private: { if (std::find(non_loop_metrics.begin(), non_loop_metrics.end(), metric) != non_loop_metrics.end()) { - throw DB::Exception("Wrong type of metric for loop execution type (" + metric + ")", 1); + throw DB::Exception("Wrong type of metric for loop execution type (" + metric + ")"); } } } @@ -1025,7 +1025,7 @@ private: { if (std::find(loop_metrics.begin(), loop_metrics.end(), metric) != loop_metrics.end()) { - throw DB::Exception("Wrong type of metric for non-loop execution type (" + metric + ")", 1); + throw DB::Exception("Wrong type of metric for non-loop execution type (" + metric + ")"); } } } @@ -1377,122 +1377,119 @@ static void getFilesFromDir(const fs::path & dir, std::vector & input_fi } int mainEntryClickHousePerformanceTest(int argc, char ** argv) +try { - using namespace DB; + using boost::program_options::value; + using Strings = std::vector; - try + boost::program_options::options_description desc("Allowed options"); + desc.add_options()("help", "produce help message")("lite", "use lite version of output")( + "profiles-file", value()->default_value(""), "Specify a file with global profiles")( + "host,h", value()->default_value("localhost"), "")("port", value()->default_value(9000), "")( + "database", value()->default_value("default"), "")("user", value()->default_value("default"), "")( + "password", value()->default_value(""), "")("tags", value()->multitoken(), "Run only tests with tag")( + "skip-tags", value()->multitoken(), "Do not run tests with tag")("names", + value()->multitoken(), + "Run tests with specific name")("skip-names", value()->multitoken(), "Do not run tests with name")( + "names-regexp", value()->multitoken(), "Run tests with names matching regexp")("skip-names-regexp", + value()->multitoken(), + "Do not run tests with names matching regexp")("recursive,r", "Recurse in directories to find all xml's"); + + /// These options will not be displayed in --help + boost::program_options::options_description hidden("Hidden options"); + hidden.add_options()("input-files", value>(), ""); + + /// But they will be legit, though. And they must be given without name + boost::program_options::positional_options_description positional; + positional.add("input-files", -1); + + boost::program_options::options_description cmdline_options; + cmdline_options.add(desc).add(hidden); + + boost::program_options::variables_map options; + boost::program_options::store( + boost::program_options::command_line_parser(argc, argv).options(cmdline_options).positional(positional).run(), options); + boost::program_options::notify(options); + + if (options.count("help")) { - using boost::program_options::value; - using Strings = std::vector; + std::cout << "Usage: " << argv[0] << " [options] [test_file ...] [tests_folder]\n"; + std::cout << desc << "\n"; + return 0; + } - boost::program_options::options_description desc("Allowed options"); - desc.add_options()("help", "produce help message")("lite", "use lite version of output")( - "profiles-file", value()->default_value(""), "Specify a file with global profiles")( - "host,h", value()->default_value("localhost"), "")("port", value()->default_value(9000), "")( - "database", value()->default_value("default"), "")("user", value()->default_value("default"), "")( - "password", value()->default_value(""), "")("tags", value()->multitoken(), "Run only tests with tag")( - "skip-tags", value()->multitoken(), "Do not run tests with tag")("names", - value()->multitoken(), - "Run tests with specific name")("skip-names", value()->multitoken(), "Do not run tests with name")( - "names-regexp", value()->multitoken(), "Run tests with names matching regexp")("skip-names-regexp", - value()->multitoken(), - "Do not run tests with names matching regexp")("recursive,r", "Recurse in directories to find all xml's"); + Strings input_files; + bool recursive = options.count("recursive"); - /// These options will not be displayed in --help - boost::program_options::options_description hidden("Hidden options"); - hidden.add_options()("input-files", value>(), ""); + if (!options.count("input-files")) + { + std::cerr << "Trying to find test scenario files in the current folder..."; + fs::path curr_dir("."); - /// But they will be legit, though. And they must be given without name - boost::program_options::positional_options_description positional; - positional.add("input-files", -1); + getFilesFromDir(curr_dir, input_files, recursive); - boost::program_options::options_description cmdline_options; - cmdline_options.add(desc).add(hidden); - - boost::program_options::variables_map options; - boost::program_options::store( - boost::program_options::command_line_parser(argc, argv).options(cmdline_options).positional(positional).run(), options); - boost::program_options::notify(options); - - if (options.count("help")) + if (input_files.empty()) { - std::cout << "Usage: " << argv[0] << " [options] [test_file ...] [tests_folder]\n"; - std::cout << desc << "\n"; - return 0; - } - - Strings input_files; - bool recursive = options.count("recursive"); - - if (!options.count("input-files")) - { - std::cerr << "Trying to find test scenario files in the current folder..."; - fs::path curr_dir("."); - - getFilesFromDir(curr_dir, input_files, recursive); - - if (input_files.empty()) - { - std::cerr << std::endl; - throw DB::Exception("Did not find any xml files", 1); - } - else - std::cerr << " found " << input_files.size() << " files." << std::endl; + std::cerr << std::endl; + throw DB::Exception("Did not find any xml files"); } else + std::cerr << " found " << input_files.size() << " files." << std::endl; + } + else + { + input_files = options["input-files"].as(); + Strings collected_files; + + for (const String filename : input_files) { - input_files = options["input-files"].as(); - Strings collected_files; + fs::path file(filename); - for (const String filename : input_files) + if (!fs::exists(file)) + throw DB::Exception("File '" + filename + "' does not exist"); + + if (fs::is_directory(file)) { - fs::path file(filename); - - if (!fs::exists(file)) - throw DB::Exception("File '" + filename + "' does not exist", 1); - - if (fs::is_directory(file)) - { - getFilesFromDir(file, collected_files, recursive); - } - else - { - if (file.extension().string() != ".xml") - throw DB::Exception("File '" + filename + "' does not have .xml extension", 1); - collected_files.push_back(filename); - } + getFilesFromDir(file, collected_files, recursive); + } + else + { + if (file.extension().string() != ".xml") + throw DB::Exception("File '" + filename + "' does not have .xml extension"); + collected_files.push_back(filename); } - - input_files = std::move(collected_files); } - Strings tests_tags = options.count("tags") ? options["tags"].as() : Strings({}); - Strings skip_tags = options.count("skip-tags") ? options["skip-tags"].as() : Strings({}); - Strings tests_names = options.count("names") ? options["names"].as() : Strings({}); - Strings skip_names = options.count("skip-names") ? options["skip-names"].as() : Strings({}); - Strings tests_names_regexp = options.count("names-regexp") ? options["names-regexp"].as() : Strings({}); - Strings skip_names_regexp = options.count("skip-names-regexp") ? options["skip-names-regexp"].as() : Strings({}); + input_files = std::move(collected_files); + } - PerformanceTest performanceTest(options["host"].as(), - options["port"].as(), - options["database"].as(), - options["user"].as(), - options["password"].as(), - options.count("lite") > 0, - options["profiles-file"].as(), - std::move(input_files), - std::move(tests_tags), - std::move(skip_tags), - std::move(tests_names), - std::move(skip_names), - std::move(tests_names_regexp), - std::move(skip_names_regexp)); - } - catch (...) - { - std::cout << getCurrentExceptionMessage(/*with stacktrace = */ true) << std::endl; - return getCurrentExceptionCode(); - } + Strings tests_tags = options.count("tags") ? options["tags"].as() : Strings({}); + Strings skip_tags = options.count("skip-tags") ? options["skip-tags"].as() : Strings({}); + Strings tests_names = options.count("names") ? options["names"].as() : Strings({}); + Strings skip_names = options.count("skip-names") ? options["skip-names"].as() : Strings({}); + Strings tests_names_regexp = options.count("names-regexp") ? options["names-regexp"].as() : Strings({}); + Strings skip_names_regexp = options.count("skip-names-regexp") ? options["skip-names-regexp"].as() : Strings({}); + + DB::PerformanceTest performanceTest(options["host"].as(), + options["port"].as(), + options["database"].as(), + options["user"].as(), + options["password"].as(), + options.count("lite") > 0, + options["profiles-file"].as(), + std::move(input_files), + std::move(tests_tags), + std::move(skip_tags), + std::move(tests_names), + std::move(skip_names), + std::move(tests_names_regexp), + std::move(skip_names_regexp)); return 0; } +catch (...) +{ + std::cout << DB::getCurrentExceptionMessage(/*with stacktrace = */ true) << std::endl; + int code = DB::getCurrentExceptionCode(); + return code ? code : 1; +} From 5b952f0c55e71bf189143f27e59518f7b954ada7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:37:36 +0300 Subject: [PATCH 014/108] Fixed error in StringRange [#CLICKHOUSE-3281]. --- dbms/src/Parsers/StringRange.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Parsers/StringRange.h b/dbms/src/Parsers/StringRange.h index fef8b547250..0c901bf5309 100644 --- a/dbms/src/Parsers/StringRange.h +++ b/dbms/src/Parsers/StringRange.h @@ -25,6 +25,7 @@ struct StringRange { first = token_begin->begin; second = token_begin->begin; + return; } TokenIterator token_last = token_end; From ee469f7a98f86f28a14c7d7aec2fdb1af1276de3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:39:55 +0300 Subject: [PATCH 015/108] Fixed redundant code [#CLICKHOUSE-3281]. --- dbms/src/Functions/FunctionsComparison.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 7f7f0bdc72c..ab7cd27e84a 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1062,9 +1062,6 @@ public: || (left_is_date_time && right_is_date_time) || (left_is_date_time && right_is_string) || (left_is_string && right_is_date_time) - || (left_is_date_time && right_is_date_time) - || (left_is_date_time && right_is_string) - || (left_is_string && right_is_date_time) || (left_is_uuid && right_is_uuid) || (left_is_uuid && right_is_string) || (left_is_string && right_is_uuid) From 24ac5a68e19f8f849b4d0f1cb7a2217a04b976f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:42:17 +0300 Subject: [PATCH 016/108] More uniform style [#CLICKHOUSE-3281]. --- dbms/src/Common/formatIPv6.cpp | 2 +- dbms/src/Functions/FunctionsCoding.h | 6 +++--- dbms/src/Interpreters/DNSCache.cpp | 2 +- dbms/src/Server/Server.cpp | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Common/formatIPv6.cpp b/dbms/src/Common/formatIPv6.cpp index 8d1e9542cf7..80fa369cb6e 100644 --- a/dbms/src/Common/formatIPv6.cpp +++ b/dbms/src/Common/formatIPv6.cpp @@ -54,7 +54,7 @@ static void formatIPv4(const unsigned char * src, char *& dst, UInt8 zeroed_tail void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count) { struct { int base, len; } best{-1}, cur{-1}; - std::array words{}; + std::array words{}; /** Preprocess: * Copy the input (bytewise) array into a wordwise array. diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 048b9b88d2d..b73a81e1e43 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -325,7 +325,7 @@ public: auto endp = tp + ipv6_bytes_length; auto curtok = src; auto saw_xdigit = false; - uint16_t val{}; + UInt16 val{}; unsigned char * colonp = nullptr; while (const auto ch = *src++) @@ -355,7 +355,7 @@ public: continue; } - if (tp + sizeof(uint16_t) > endp) + if (tp + sizeof(UInt16) > endp) return clear_dst(); *tp++ = static_cast((val >> 8) & 0xffu); @@ -380,7 +380,7 @@ public: if (saw_xdigit) { - if (tp + sizeof(uint16_t) > endp) + if (tp + sizeof(UInt16) > endp) return clear_dst(); *tp++ = static_cast((val >> 8) & 0xffu); diff --git a/dbms/src/Interpreters/DNSCache.cpp b/dbms/src/Interpreters/DNSCache.cpp index d12007e908e..e4ea9e119da 100644 --- a/dbms/src/Interpreters/DNSCache.cpp +++ b/dbms/src/Interpreters/DNSCache.cpp @@ -59,7 +59,7 @@ static void splitHostAndPort(const std::string & host_and_port, std::string & ou { struct servent * se = getservbyname(port_str.c_str(), nullptr); if (se) - out_port = ntohs(static_cast(se->s_port)); + out_port = ntohs(static_cast(se->s_port)); else throw Exception("Service not found", ErrorCodes::BAD_ARGUMENTS); } diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 5130d612193..c41a408f726 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -321,7 +321,7 @@ int Server::main(const std::vector & args) try_listen = true; } - auto make_socket_address = [&](const std::string & host, std::uint16_t port) { + auto make_socket_address = [&](const std::string & host, UInt16 port) { Poco::Net::SocketAddress socket_address; try { From e03ca48d84c7fd875b40f503be1dafe115760252 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:42:44 +0300 Subject: [PATCH 017/108] More uniform style [#CLICKHOUSE-3281]. --- dbms/src/Server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index c41a408f726..8167f85e2f9 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -321,7 +321,8 @@ int Server::main(const std::vector & args) try_listen = true; } - auto make_socket_address = [&](const std::string & host, UInt16 port) { + auto make_socket_address = [&](const std::string & host, UInt16 port) + { Poco::Net::SocketAddress socket_address; try { From 60263d943e3979cd06fa1617975cbb44295b3cd8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 05:48:21 +0300 Subject: [PATCH 018/108] IPv6StringToNum: fixed missing overflow check [#CLICKHOUSE-3281]. --- dbms/src/Functions/FunctionsCoding.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index b73a81e1e43..3158b8225cf 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -325,9 +325,10 @@ public: auto endp = tp + ipv6_bytes_length; auto curtok = src; auto saw_xdigit = false; - UInt16 val{}; + UInt32 val{}; unsigned char * colonp = nullptr; + /// Assuming zero-terminated string. while (const auto ch = *src++) { const auto num = unhex(ch); From ff8940ddbe7ca1e82de71cefff6c32a0fab4eac4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 06:00:33 +0300 Subject: [PATCH 019/108] Fixed bad code [#CLICKHOUSE-3281]. --- dbms/src/Common/formatIPv6.cpp | 28 +++++++++++++++++++--------- dbms/src/Functions/FunctionsCoding.h | 20 +++++++++++++------- 2 files changed, 32 insertions(+), 16 deletions(-) diff --git a/dbms/src/Common/formatIPv6.cpp b/dbms/src/Common/formatIPv6.cpp index 80fa369cb6e..3401ea8378a 100644 --- a/dbms/src/Common/formatIPv6.cpp +++ b/dbms/src/Common/formatIPv6.cpp @@ -8,30 +8,40 @@ namespace DB { /// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base) -static constexpr uint32_t int_log(const uint32_t value, const uint32_t base, const bool carry = false) +static constexpr UInt32 intLog(const UInt32 value, const UInt32 base, const bool carry) { - return value >= base ? 1 + int_log(value / base, base, value % base || carry) : value % base > 1 || carry; + return value >= base ? 1 + intLog(value / base, base, value % base || carry) : value % base > 1 || carry; } -/// print integer in desired base, faster than sprintf -template -static void print_integer(char *& out, T value) +/// Print integer in desired base, faster than sprintf. +/// NOTE This is not the best way. See https://github.com/miloyip/itoa-benchmark +/// But it doesn't matter here. +template +static void printInteger(char *& out, T value) { if (value == 0) *out++ = '0'; else { + constexpr size_t buffer_size = sizeof(T) * intLog(256, base, false); + char buf[buffer_size]; auto ptr = buf; while (value > 0) { - *ptr++ = hexDigitLowercase(value % base); + *ptr = hexDigitLowercase(value % base); + ++ptr; value /= base; } + /// Copy to out reversed. while (ptr != buf) - *out++ = *--ptr; + { + --ptr; + *out = *ptr; + ++out; + } } } @@ -43,7 +53,7 @@ static void formatIPv4(const unsigned char * src, char *& dst, UInt8 zeroed_tail for (const auto i : ext::range(0, IPV4_BINARY_LENGTH)) { UInt8 byte = (i < limit) ? src[i] : 0; - print_integer<10, UInt8>(dst, byte); + printInteger<10, UInt8>(dst, byte); if (i != IPV4_BINARY_LENGTH - 1) *dst++ = '.'; @@ -112,7 +122,7 @@ void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_ break; } - print_integer<16>(dst, words[i]); + printInteger<16>(dst, words[i]); } /// Was it a trailing run of 0x00's? diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 3158b8225cf..eac1cdfbc82 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -472,24 +472,29 @@ public: char * begin = out; /// Write everything backwards. - for (size_t offset = 0; offset <= 24; offset += 8) + for (size_t octet = 0; octet < 4; ++octet) { - if (offset > 0) - *(out++) = '.'; + if (octet > 0) + { + *out = '.'; + ++out; + } /// Get the next byte. - UInt32 value = (ip >> offset) & static_cast(255); + UInt32 value = (ip >> (octet * 8)) & static_cast(0xFF); /// Faster than sprintf. if (value == 0) { - *(out++) = '0'; + *out = '0'; + ++out; } else { while (value > 0) { - *(out++) = '0' + value % 10; + *out = '0' + value % 10; + ++out; value /= 10; } } @@ -498,7 +503,8 @@ public: /// And reverse. std::reverse(begin, out); - *(out++) = '\0'; + *out = '\0'; + ++out; } bool useDefaultImplementationForConstants() const override { return true; } From b370551e676ddfc8739e4f9190293ffeb14edb1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 06:17:06 +0300 Subject: [PATCH 020/108] Fixed redundant code in IPv4NumToStringClassC function and removed copy-paste [#CLICKHOUSE-3281]. --- dbms/src/Functions/FunctionsCoding.cpp | 8 +- dbms/src/Functions/FunctionsCoding.h | 160 +++++++------------------ 2 files changed, 46 insertions(+), 122 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.cpp b/dbms/src/Functions/FunctionsCoding.cpp index 7de54862438..adc660df78f 100644 --- a/dbms/src/Functions/FunctionsCoding.cpp +++ b/dbms/src/Functions/FunctionsCoding.cpp @@ -4,15 +4,19 @@ namespace DB { +struct NameFunctionIPv4NumToString { static constexpr auto name = "IPv4NumToString"; }; +struct NameFunctionIPv4NumToStringClassC { static constexpr auto name = "IPv4NumToStringClassC"; }; + + void registerFunctionsCoding(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); factory.registerFunction(); - factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction>(); diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index eac1cdfbc82..4429c639285 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -444,35 +444,30 @@ public: }; +/** If mask_tail_octets > 0, the last specified number of octets will be filled with "xxx". + */ +template class FunctionIPv4NumToString : public IFunction { -public: - static constexpr auto name = "IPv4NumToString"; - static FunctionPtr create(const Context & context) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 1; } - bool isInjective(const Block &) override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!checkDataType(&*arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - +private: static void formatIP(UInt32 ip, char *& out) { char * begin = out; - /// Write everything backwards. - for (size_t octet = 0; octet < 4; ++octet) + for (size_t octet = 0; octet < mask_tail_octets; ++octet) + { + if (octet > 0) + { + *out = '.'; + ++out; + } + + memcpy(out, "xxx", 3); /// Strange choice, but meets the specification. + out += 3; + } + + /// Write everything backwards. NOTE The loop is unrolled. + for (size_t octet = mask_tail_octets; octet < 4; ++octet) { if (octet > 0) { @@ -483,7 +478,7 @@ public: /// Get the next byte. UInt32 value = (ip >> (octet * 8)) & static_cast(0xFF); - /// Faster than sprintf. + /// Faster than sprintf. NOTE Actually not good enough. LUT will be better. if (value == 0) { *out = '0'; @@ -507,6 +502,27 @@ public: ++out; } +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context & context) { return std::make_shared>(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool isInjective(const Block &) override { return mask_tail_octets == 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!checkDataType(&*arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + bool useDefaultImplementationForConstants() const override { return true; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override @@ -621,102 +637,6 @@ public: }; -class FunctionIPv4NumToStringClassC : public IFunction -{ -public: - static constexpr auto name = "IPv4NumToStringClassC"; - static FunctionPtr create(const Context & context) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 1; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!checkDataType(&*arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - static void formatIP(UInt32 ip, char *& out) - { - char * begin = out; - - for (auto i = 0; i < 3; ++i) - *(out++) = 'x'; - - /// Write everything backwards. - for (size_t offset = 8; offset <= 24; offset += 8) - { - if (offset > 0) - *(out++) = '.'; - - /// Get the next byte. - UInt32 value = (ip >> offset) & static_cast(255); - - /// Faster than sprintf. - if (value == 0) - { - *(out++) = '0'; - } - else - { - while (value > 0) - { - *(out++) = '0' + value % 10; - value /= 10; - } - } - } - - /// And reverse. - std::reverse(begin, out); - - *(out++) = '\0'; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override - { - const ColumnPtr & column = block.getByPosition(arguments[0]).column; - - if (const ColumnUInt32 * col = typeid_cast(column.get())) - { - const ColumnUInt32::Container_t & vec_in = col->getData(); - - std::shared_ptr col_res = std::make_shared(); - block.getByPosition(result).column = col_res; - - ColumnString::Chars_t & vec_res = col_res->getChars(); - ColumnString::Offsets_t & offsets_res = col_res->getOffsets(); - - vec_res.resize(vec_in.size() * (IPV4_MAX_TEXT_LENGTH + 1)); /// the longest value is: 255.255.255.255\0 - offsets_res.resize(vec_in.size()); - char * begin = reinterpret_cast(&vec_res[0]); - char * pos = begin; - - for (size_t i = 0; i < vec_in.size(); ++i) - { - formatIP(vec_in[i], pos); - offsets_res[i] = pos - begin; - } - - vec_res.resize(pos - begin); - } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - class FunctionIPv4ToIPv6 : public IFunction { public: From 7e460444741309b510321466f72815cc3069b1f6 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 6 Sep 2017 18:19:03 +0300 Subject: [PATCH 021/108] Fix test 00501_http_head --- .../queries/0_stateless/00501_http_head.reference | 12 ++++++++++++ dbms/tests/queries/0_stateless/00501_http_head.sh | 4 ++-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00501_http_head.reference b/dbms/tests/queries/0_stateless/00501_http_head.reference index e69de29bb2d..8bbf4ff0b36 100644 --- a/dbms/tests/queries/0_stateless/00501_http_head.reference +++ b/dbms/tests/queries/0_stateless/00501_http_head.reference @@ -0,0 +1,12 @@ +< HTTP/1.1 200 OK +< Connection: Keep-Alive +< Content-Type: text/tab-separated-values; charset=UTF-8 +< Transfer-Encoding: chunked +< Keep-Alive: timeout=3 +< +< HTTP/1.1 200 OK +< Connection: Keep-Alive +< Content-Type: text/tab-separated-values; charset=UTF-8 +< Transfer-Encoding: chunked +< Keep-Alive: timeout=3 +< diff --git a/dbms/tests/queries/0_stateless/00501_http_head.sh b/dbms/tests/queries/0_stateless/00501_http_head.sh index 4b99c841c45..79699041736 100755 --- a/dbms/tests/queries/0_stateless/00501_http_head.sh +++ b/dbms/tests/queries/0_stateless/00501_http_head.sh @@ -1,4 +1,4 @@ #!/usr/bin/env bash -curl -s --max-time 0.5 -X "HEAD" 'http://localhost:8123/?query=SELECT%201' -curl -s --max-time 0.5 -X "HEAD" 'http://localhost:8123/?query=select+*+from+system.numbers+limit+1000000' +( curl -v --max-time 0.3 -X "HEAD" 'http://localhost:18123/?query=SELECT%201'; + curl -v --max-time 0.3 -X "HEAD" 'http://localhost:18123/?query=select+*+from+system.numbers+limit+1000000' ) 2>&1 | grep -e "^<" | grep -v "Date:" From 16bd99493b9b3be49487ab3595892b82e79d2357 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 6 Sep 2017 20:16:17 +0300 Subject: [PATCH 022/108] Reverse tests order --- dbms/tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 4b188920d64..21d30bf1876 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -93,7 +93,7 @@ def main(args): print("Won't run stateful tests because test data wasn't loaded. See README.txt.") continue - for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir))): + for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir)), reverse=True): if SERVER_DIED: break From 888214e33df6e933efacb6bd0c179b177092c07c Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 6 Sep 2017 21:39:04 +0300 Subject: [PATCH 023/108] Revert "Reverse tests order" - need smarter reverse This reverts commit 16bd99493b9b3be49487ab3595892b82e79d2357. --- dbms/tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 21d30bf1876..4b188920d64 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -93,7 +93,7 @@ def main(args): print("Won't run stateful tests because test data wasn't loaded. See README.txt.") continue - for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir)), reverse=True): + for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir))): if SERVER_DIED: break From b538df661a007c3a795cbfa821e38fd25a6c3d3e Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 6 Sep 2017 22:07:22 +0300 Subject: [PATCH 024/108] Again fix test 00501_http_head --- .../0_stateless/00501_http_head.reference | 24 +++++++++---------- .../queries/0_stateless/00501_http_head.sh | 4 ++-- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00501_http_head.reference b/dbms/tests/queries/0_stateless/00501_http_head.reference index 8bbf4ff0b36..9727c63b2d8 100644 --- a/dbms/tests/queries/0_stateless/00501_http_head.reference +++ b/dbms/tests/queries/0_stateless/00501_http_head.reference @@ -1,12 +1,12 @@ -< HTTP/1.1 200 OK -< Connection: Keep-Alive -< Content-Type: text/tab-separated-values; charset=UTF-8 -< Transfer-Encoding: chunked -< Keep-Alive: timeout=3 -< -< HTTP/1.1 200 OK -< Connection: Keep-Alive -< Content-Type: text/tab-separated-values; charset=UTF-8 -< Transfer-Encoding: chunked -< Keep-Alive: timeout=3 -< +HTTP/1.1 200 OK +Connection: Keep-Alive +Content-Type: text/tab-separated-values; charset=UTF-8 +Transfer-Encoding: chunked +Keep-Alive: timeout=3 + +HTTP/1.1 200 OK +Connection: Keep-Alive +Content-Type: text/tab-separated-values; charset=UTF-8 +Transfer-Encoding: chunked +Keep-Alive: timeout=3 + diff --git a/dbms/tests/queries/0_stateless/00501_http_head.sh b/dbms/tests/queries/0_stateless/00501_http_head.sh index 79699041736..332b06dc944 100755 --- a/dbms/tests/queries/0_stateless/00501_http_head.sh +++ b/dbms/tests/queries/0_stateless/00501_http_head.sh @@ -1,4 +1,4 @@ #!/usr/bin/env bash -( curl -v --max-time 0.3 -X "HEAD" 'http://localhost:18123/?query=SELECT%201'; - curl -v --max-time 0.3 -X "HEAD" 'http://localhost:18123/?query=select+*+from+system.numbers+limit+1000000' ) 2>&1 | grep -e "^<" | grep -v "Date:" +( curl -s --head "${CLICKHOUSE_URL:=http://localhost:8123/}?query=SELECT%201"; + curl -s --head "${CLICKHOUSE_URL:=http://localhost:8123/}?query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:" From 7c2327f3c8c8880696bf92b45429793f4cf04b11 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 6 Sep 2017 21:40:49 +0300 Subject: [PATCH 025/108] LibraryDictionarySource: fix compile on macos and 32bit --- dbms/src/Dictionaries/LibraryDictionarySource.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index 52559f98e69..e61f1c85384 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -120,7 +120,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadAll() auto columns_holder = std::make_unique(dict_struct.attributes.size()); ClickHouseLibrary::CStrings columns{ - reinterpret_cast(columns_holder.get()), dict_struct.attributes.size()}; + static_cast(columns_holder.get()), dict_struct.attributes.size()}; size_t i = 0; for (auto & a : dict_struct.attributes) { @@ -147,7 +147,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector & const ClickHouseLibrary::VectorUInt64 ids_data{ids.data(), ids.size()}; auto columns_holder = std::make_unique(dict_struct.attributes.size()); ClickHouseLibrary::CStrings columns_pass{ - reinterpret_cast(columns_holder.get()), dict_struct.attributes.size()}; + static_cast(columns_holder.get()), dict_struct.attributes.size()}; size_t i = 0; for (auto & a : dict_struct.attributes) { @@ -183,14 +183,14 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column */ auto columns_holder = std::make_unique(key_columns.size()); ClickHouseLibrary::CStrings columns_pass{ - reinterpret_cast(columns_holder.get()), key_columns.size()}; + static_cast(columns_holder.get()), key_columns.size()}; size_t key_columns_n = 0; for (auto & column : key_columns) { columns_pass.data[key_columns_n] = column->getName().c_str(); ++key_columns_n; } - const ClickHouseLibrary::VectorUInt64 requested_rows_c{requested_rows.data(), requested_rows.size()}; + const ClickHouseLibrary::VectorUInt64 requested_rows_c{static_cast(requested_rows.data()), requested_rows.size()}; void * data_ptr = nullptr; /// Get function pointer before dataAllocate call because library->get may throw. From 7142e43be6b4d42d791eb2c0c9c65c4b9b960c88 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 6 Sep 2017 22:21:52 +0300 Subject: [PATCH 026/108] Fix compile 32bit --- dbms/src/Dictionaries/LibraryDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index e61f1c85384..d571a71fae8 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -190,7 +190,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column columns_pass.data[key_columns_n] = column->getName().c_str(); ++key_columns_n; } - const ClickHouseLibrary::VectorUInt64 requested_rows_c{static_cast(requested_rows.data()), requested_rows.size()}; + const ClickHouseLibrary::VectorUInt64 requested_rows_c{reinterpret_cast(requested_rows.data()), requested_rows.size()}; void * data_ptr = nullptr; /// Get function pointer before dataAllocate call because library->get may throw. From 3022d0b8a0177164fb2a410c7b70d996be1c7e85 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 6 Sep 2017 22:28:19 +0300 Subject: [PATCH 027/108] YQL-3278: add some links to ClickHouse Meetup in Berlin on October 5, 2017 (#1199) * update presentations * CLICKHOUSE-2936: redirect from clickhouse.yandex.ru and clickhouse.yandex.com * update submodule * lost files * CLICKHOUSE-2981: prefer sphinx docs over original reference * CLICKHOUSE-2981: docs styles more similar to main website + add flags to switch language links * update presentations * Less confusing directory structure (docs -> doc/reference/) * Minify sphinx docs too * Website release script: fail fast + pass docker hash on deploy * Do not underline links in docs * shorter * cleanup docker images * tune nginx config * CLICKHOUSE-3043: get rid of habrastorage links * Lost translation * CLICKHOUSE-2936: temporary client-side redirect * behaves weird in test * put redirect back * CLICKHOUSE-3047: copy docs txts to public too * move to proper file * remove old pages to avoid confusion * Remove reference redirect warning for now * Refresh README.md * Yellow buttons in docs * Use svg flags instead of unicode ones in docs * fix test website instance * Put flags to separate files * wrong flag * Copy Yandex.Metrica introduction from main page to docs * Yet another home page structure change, couple new blocks (CLICKHOUSE-3045) * Update Contacts section * CLICKHOUSE-2849: more detailed legal information * CLICKHOUSE-2978 preparation - split by files * More changes in Contacts block * Tune texts on index page * update presentations * One more benchmark * Add usage sections to index page, adapted from slides * Get the roadmap started, based on slides from last ClickHouse Meetup * CLICKHOUSE-2977: some rendering tuning * Get rid of excessive section in the end of getting started * Make headers linkable * CLICKHOUSE-2981: links to editing reference - https://github.com/yandex/ClickHouse/issues/849 * CLICKHOUSE-2981: fix mobile styles in docs * Ban crawling of duplicating docs * Open some external links in new tab * Ban old docs too * Lots of trivial fixes in english docs * Lots of trivial fixes in russian docs * Remove getting started copies in markdown * Add Yandex.Webmaster * Fix some sphinx warnings * More warnings fixed in english docs * More sphinx warnings fixed * Add code-block:: text * More code-block:: text * These headers look not that well * Better switch between documentation languages * merge use_case.rst into ya_metrika_task.rst * Edit the agg_functions.rst texts * Add lost empty lines * Lost blank lines * Add new logo sizes * update presentations * Next step in migrating to new documentation * Fix all warnings in en reference * Fix all warnings in ru reference * Re-arrange existing reference * Move operation tips to main reference * Fix typos noticed by milovidov@ * Get rid of zookeeper.md * Looks like duplicate of tutorial.html * Fix some mess with html tags in tutorial * No idea why nobody noticed this before, but it was completely not clear whet to get the data * Match code block styling between main and tutorial pages (in favor of the latter) * Get rid of some copypaste in tutorial * Normalize header styles * Move example_datasets to sphinx * Move presentations submodule to website * Move and update README.md * No point in duplicating articles from habrahabr here * Move development-related docs as is for now * doc/reference/ -> docs/ (to match the URL on website) * Adapt links to match the previous commit * Adapt development docs to rst (still lacks translation and strikethrough support) * clean on release * blacklist presentations in gulp * strikethrough support in sphinx * just copy development folder for now * fix weird introduction in style article * Style guide translation (WIP) * Finish style guide translation to English * gulp clean separately * Update year in LICENSE * Initial CONTRIBUTING.md * Fix remaining links to old docs in tutorial * Some tutorial fixes * Typo * Another typo * Update list of authors from yandex-team accoding to git log * Fix diff with master * couple fixes in en what_is_clickhouse.rst * Try different link to blog in Russian * Swap words * Slightly larger line height * CLICKHOUSE-3089: disable hyphenation in docs * update presentations * Fix copying of txt files * update submodule * CLICKHOUSE-3108: fix overflow issues in mobile version * Less weird tutorial header in mobile version * CLICKHOUSE-3073: skip sourcemaps by default * CLICKHOUSE-3067: rename item in docs navigation * fix list markup * CLICKHOUSE-3067: some documentation style tuning * CLICKHOUSE-3067: less laggy single page documentation * update presentations * YQL-3278: add some links to ClickHouse Meetup in Berlin on October 5, 2017 --- docs/en/conf.py | 1 + website/index.css | 17 +++++++++++++++-- website/index.html | 5 +++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/docs/en/conf.py b/docs/en/conf.py index c58139ce779..b35129346ce 100644 --- a/docs/en/conf.py +++ b/docs/en/conf.py @@ -125,6 +125,7 @@ html_theme_options = { 'link': '#08f', 'link_hover': 'red', 'extra_nav_links': collections.OrderedDict([ + ('ClickHouse Meetup Berlin', 'https://events.yandex.com/events/meetings/05-10-2017/'), ('Switch to Russian ', '#ru'), ('Single page documentation', '/docs/en/single/'), ('Website home', '/'), diff --git a/website/index.css b/website/index.css index bd419701f44..3f72e172b83 100644 --- a/website/index.css +++ b/website/index.css @@ -209,9 +209,22 @@ a:hover, a:active { margin-top: 0; } -#performance { +#performance, #announcement { background: #fc0; } +#announcement { + margin: 0 0 60px 0; + padding: 20px 0; +} +#announcement-link { + color: #000; + text-decoration: underline; + font: 400 200%/133% 'Yandex Sans Display Web',Arial,sans-serif; + margin: 1em 0; +} +#announcement-link:hover { + color: #000; +} #call_to_action, #benchmark_learn_more { text-decoration: none; @@ -402,7 +415,7 @@ img { } #hero { - padding: 40px 0 10px 0; + padding: 40px 0 0 0; } .desktop-only { diff --git a/website/index.html b/website/index.html index b0c8e052ed4..ca5364f0802 100644 --- a/website/index.html +++ b/website/index.html @@ -90,6 +90,11 @@
+

ClickHouse. Just makes you think faster.

From e3e506d4495488daaf0108a5c7511307bba85281 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 23:01:44 +0300 Subject: [PATCH 028/108] Fixed with_mergeable_state option for clickhouse-benchmark [#CLICKHOUSE-3282]. --- dbms/src/Server/Benchmark.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Server/Benchmark.cpp b/dbms/src/Server/Benchmark.cpp index 986a2efac09..f3bd262ad6f 100644 --- a/dbms/src/Server/Benchmark.cpp +++ b/dbms/src/Server/Benchmark.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include @@ -71,6 +72,10 @@ public: { std::cerr << std::fixed << std::setprecision(3); + /// This is needed to receive blocks with columns of AggregateFunction data type + /// (example: when using stage = 'with_mergeable_state') + registerAggregateFunctions(); + if (stage == "complete") query_processing_stage = QueryProcessingStage::Complete; else if (stage == "fetch_columns") From 76957f77c46a9a522f9e25fbaf9681c1daf39afe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 23:02:11 +0300 Subject: [PATCH 029/108] Removed strange empty space [#CLICKHOUSE-2]. --- libs/libdaemon/src/BaseDaemon.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 434545b89cc..56bb0826479 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -328,11 +328,6 @@ private: static const int max_frames = 50; void * frames[max_frames]; - - - - - #if USE_UNWIND int frames_size = backtraceLibUnwind(frames, max_frames, context); From ea19c4494f1279fabc6575e9c77c37b4ad0a10bc Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 6 Sep 2017 23:13:21 +0300 Subject: [PATCH 030/108] Better test reverse (#1207) * Again fix test 00501_http_head * Better test reverse * Fix test reverse sort * Update clickhouse-test --- dbms/tests/clickhouse-test | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 4b188920d64..58304f1768e 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -93,7 +93,12 @@ def main(args): print("Won't run stateful tests because test data wasn't loaded. See README.txt.") continue - for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir))): + # Reverse sort order: we want run newest test first. + # And not reverse subtests + def key_func(item): + prefix, suffix = item.split('_', 1) + return -int(prefix), suffix + for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir)), key=key_func): if SERVER_DIED: break From 9c37dfd75f40e74a56c87beeec2d705355a6168f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 23:24:40 +0300 Subject: [PATCH 031/108] Fixed common source of errors while deploying newer version of ClickHouse [#CLICKHOUSE-3282]. --- dbms/src/Interpreters/Compiler.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index 7dc37a92a2f..e798c8add55 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -181,6 +181,17 @@ SharedLibraryPtr Compiler::getOrCount( } +/// This will guarantee that code will compile only when version of headers match version of running server. +static void addCodeToAssertHeadersMatch(WriteBuffer & out) +{ + out << + "#include \n" + "#if VERSION_REVISION != " << ClickHouseRevision::get() << "\n" + "#error \"ClickHouse headers revision doesn't match runtime revision of the server.\"\n" + "#endif\n\n"; +} + + void Compiler::compile( HashedKey hashed_key, std::string file_name, @@ -197,6 +208,8 @@ void Compiler::compile( { WriteBufferFromFile out(cpp_file_path); + + addCodeToAssertHeadersMatch(out); out << get_code(); } From 0d11c735fb13f5526d3f5d902380a3e26e4ee475 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Sep 2017 23:47:08 +0300 Subject: [PATCH 032/108] Addition to prev. revision [#CLICKHOUSE-3282]. --- copy_headers.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/copy_headers.sh b/copy_headers.sh index 1254c397bfe..7b0202889f4 100755 --- a/copy_headers.sh +++ b/copy_headers.sh @@ -28,6 +28,7 @@ fi for src_file in $($CLANG -M -xc++ -std=gnu++1z -Wall -Werror -msse4 -mcx16 -mpopcnt -O3 -g -fPIC \ $(cat "$SOURCE_PATH/build/include_directories.txt") \ + "$SOURCE_PATH/build/dbms/src/Common/config_version.h" \ "$SOURCE_PATH/dbms/src/Interpreters/SpecializedAggregator.h" | tr -d '\\' | sed -r -e 's/^\w+\.o://'); From b5f94b67381ceea3f920fcc35ea5f38090c47b7a Mon Sep 17 00:00:00 2001 From: robot-metrika-test Date: Wed, 6 Sep 2017 23:49:16 +0300 Subject: [PATCH 033/108] Auto version update to [54287] --- dbms/cmake/version.cmake | 4 ++-- debian/changelog | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index cd9a7744327..41e08de697d 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -1,6 +1,6 @@ # This strings autochanged from release_lib.sh: -set(VERSION_DESCRIBE v1.1.54286-testing) -set(VERSION_REVISION 54286) +set(VERSION_DESCRIBE v1.1.54287-testing) +set(VERSION_REVISION 54287) # end of autochange set (VERSION_MAJOR 1) diff --git a/debian/changelog b/debian/changelog index 5df6d6edaac..2d309002fe3 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (1.1.54286) unstable; urgency=low +clickhouse (1.1.54287) unstable; urgency=low * Modified source code - -- Tue, 05 Sep 2017 00:37:29 +0300 + -- Wed, 06 Sep 2017 23:49:16 +0300 From be4deb79793a5d4e18dd8e8dabae1bfa509fb500 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 7 Sep 2017 00:04:23 +0300 Subject: [PATCH 034/108] Fix test order (#1209) * Again fix test 00501_http_head * Better test reverse * Fix test reverse sort * Update clickhouse-test --- ...reference => 00028_shard_big_non_agg_aj_distributed.reference} | 0 ...distributed.sql => 00028_shard_big_non_agg_aj_distributed.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{00029_shard_big_non_agg_aj_distributed.reference => 00028_shard_big_non_agg_aj_distributed.reference} (100%) rename dbms/tests/queries/0_stateless/{00029_shard_big_non_agg_aj_distributed.sql => 00028_shard_big_non_agg_aj_distributed.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/00029_shard_big_non_agg_aj_distributed.reference b/dbms/tests/queries/0_stateless/00028_shard_big_non_agg_aj_distributed.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00029_shard_big_non_agg_aj_distributed.reference rename to dbms/tests/queries/0_stateless/00028_shard_big_non_agg_aj_distributed.reference diff --git a/dbms/tests/queries/0_stateless/00029_shard_big_non_agg_aj_distributed.sql b/dbms/tests/queries/0_stateless/00028_shard_big_non_agg_aj_distributed.sql similarity index 100% rename from dbms/tests/queries/0_stateless/00029_shard_big_non_agg_aj_distributed.sql rename to dbms/tests/queries/0_stateless/00028_shard_big_non_agg_aj_distributed.sql From 90f1d8689bfa4003bc4604e8c7d9760a74e0336b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Sep 2017 00:42:20 +0300 Subject: [PATCH 035/108] Addition to prev. revision [#CLICKHOUSE-3282]. --- copy_headers.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/copy_headers.sh b/copy_headers.sh index 7b0202889f4..25787566e32 100755 --- a/copy_headers.sh +++ b/copy_headers.sh @@ -34,7 +34,7 @@ for src_file in $($CLANG -M -xc++ -std=gnu++1z -Wall -Werror -msse4 -mcx16 -mpop sed -r -e 's/^\w+\.o://'); do dst_file=$src_file; - mkdir -p "$DST/$(echo $dst_file | sed -r -e 's/\/[^/]*$/\//')"; + mkdir -p "$DST/$(echo $dst_file | sed -r -e 's/build\///; s/\/[^/]*$/\//;')"; cp "$src_file" "$DST/$dst_file"; done From d422777d9cf9a51908d10db11d05f961334b7294 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 7 Sep 2017 03:12:39 +0300 Subject: [PATCH 036/108] ShellCommand: automatic wait call in destructor (#1210) * ShellCommand: automatic wait call in destructor * ShellCommand: tryWait in destructor * Update ShellCommand.cpp * Update ShellCommand.cpp --- dbms/src/Common/ShellCommand.cpp | 11 +++++++++-- dbms/src/Common/ShellCommand.h | 3 +++ dbms/src/Common/tests/shell_command_test.cpp | 11 +++++++++++ 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/ShellCommand.cpp b/dbms/src/Common/ShellCommand.cpp index a4c16ee5c48..00010799aee 100644 --- a/dbms/src/Common/ShellCommand.cpp +++ b/dbms/src/Common/ShellCommand.cpp @@ -66,8 +66,8 @@ namespace enum class ReturnCodes : int { CANNOT_DUP_STDIN = 42, /// The value is not important, but it is chosen so that it's rare to conflict with the program return code. - CANNOT_DUP_STDOUT = 43, - CANNOT_DUP_STDERR = 44, + CANNOT_DUP_STDOUT = 43, + CANNOT_DUP_STDERR = 44, CANNOT_EXEC = 45, }; } @@ -76,6 +76,11 @@ namespace namespace DB { +ShellCommand::~ShellCommand() +{ + if (!wait_called) + tryWait(); +} std::unique_ptr ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only) { @@ -176,6 +181,8 @@ std::unique_ptr ShellCommand::executeDirect(const std::string & pa int ShellCommand::tryWait() { + wait_called = true; + int status = 0; if (-1 == waitpid(pid, &status, 0)) throwFromErrno("Cannot waitpid", ErrorCodes::CANNOT_WAITPID); diff --git a/dbms/src/Common/ShellCommand.h b/dbms/src/Common/ShellCommand.h index a558216fcbf..8df7de1de35 100644 --- a/dbms/src/Common/ShellCommand.h +++ b/dbms/src/Common/ShellCommand.h @@ -27,6 +27,7 @@ class ShellCommand { private: pid_t pid; + bool wait_called = false; ShellCommand(pid_t pid, int in_fd, int out_fd, int err_fd) : pid(pid), in(in_fd), out(out_fd), err(err_fd) {}; @@ -38,6 +39,8 @@ public: ReadBufferFromFile out; ReadBufferFromFile err; + ~ShellCommand(); + /// Run the command using /bin/sh -c static std::unique_ptr execute(const std::string & command, bool pipe_stdin_only = false); diff --git a/dbms/src/Common/tests/shell_command_test.cpp b/dbms/src/Common/tests/shell_command_test.cpp index 564567373c8..81fadfd4a51 100644 --- a/dbms/src/Common/tests/shell_command_test.cpp +++ b/dbms/src/Common/tests/shell_command_test.cpp @@ -5,6 +5,8 @@ #include #include +#include +#include using namespace DB; @@ -43,6 +45,15 @@ try command->wait(); } + + // hunting: + for (int i = 0; i < 1000; ++i) { + auto command = ShellCommand::execute("echo " + std::to_string(i)); + //command->wait(); // now automatic + } + + // std::cerr << "inspect me: ps auxwwf" << "\n"; + // std::this_thread::sleep_for(std::chrono::seconds(100)); } catch (...) { From 89933328f1899443ceb43f97f1d985e0c1630e30 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Sep 2017 05:33:59 +0300 Subject: [PATCH 037/108] Fixed build [#CLICKHOUSE-3284]. --- copy_headers.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/copy_headers.sh b/copy_headers.sh index 25787566e32..69a1d366180 100755 --- a/copy_headers.sh +++ b/copy_headers.sh @@ -34,7 +34,8 @@ for src_file in $($CLANG -M -xc++ -std=gnu++1z -Wall -Werror -msse4 -mcx16 -mpop sed -r -e 's/^\w+\.o://'); do dst_file=$src_file; - mkdir -p "$DST/$(echo $dst_file | sed -r -e 's/build\///; s/\/[^/]*$/\//;')"; + dst_file=$(echo $dst_file | sed -r -e 's/build\///') # for simplicity reasons, will put generated headers near the rest. + mkdir -p "$DST/$(echo $dst_file | sed -r -e 's/\/[^/]*$/\//')"; cp "$src_file" "$DST/$dst_file"; done From 17edcdec96b01d988e749f44c4be0194f8546aee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Sep 2017 06:33:53 +0300 Subject: [PATCH 038/108] Updated zstd to version 1.3.1, properly [#CLICKHOUSE-3277]. --- .../libzstd/include/zstd/common/bitstream.h | 81 ++- .../include/zstd/common/error_private.c | 18 +- .../include/zstd/common/error_private.h | 8 +- contrib/libzstd/include/zstd/common/fse.h | 16 +- .../include/zstd/common/fse_decompress.c | 25 +- contrib/libzstd/include/zstd/common/huf.h | 11 +- contrib/libzstd/include/zstd/common/mem.h | 12 +- contrib/libzstd/include/zstd/common/pool.c | 88 ++- contrib/libzstd/include/zstd/common/pool.h | 20 +- .../libzstd/include/zstd/common/threading.h | 11 +- contrib/libzstd/include/zstd/common/xxhash.c | 50 +- .../libzstd/include/zstd/common/zstd_common.c | 8 +- .../libzstd/include/zstd/common/zstd_errors.h | 62 +- .../include/zstd/common/zstd_internal.h | 113 ++-- .../include/zstd/compress/fse_compress.c | 26 +- .../include/zstd/compress/huf_compress.c | 7 +- .../include/zstd/compress/zstd_compress.c | 622 +++++++++--------- .../libzstd/include/zstd/compress/zstd_opt.h | 234 +++---- .../include/zstd/compress/zstdmt_compress.c | 377 ++++++----- .../include/zstd/compress/zstdmt_compress.h | 19 +- .../include/zstd/decompress/huf_decompress.c | 32 +- .../include/zstd/decompress/zstd_decompress.c | 210 +++--- .../libzstd/include/zstd/deprecated/zbuff.h | 8 +- .../include/zstd/deprecated/zbuff_common.c | 9 +- .../include/zstd/deprecated/zbuff_compress.c | 8 +- .../zstd/deprecated/zbuff_decompress.c | 8 +- .../libzstd/include/zstd/dictBuilder/cover.c | 16 +- .../libzstd/include/zstd/dictBuilder/zdict.c | 10 +- .../libzstd/include/zstd/dictBuilder/zdict.h | 8 +- .../libzstd/include/zstd/legacy/zstd_legacy.h | 12 +- .../libzstd/include/zstd/legacy/zstd_v01.c | 8 +- .../libzstd/include/zstd/legacy/zstd_v01.h | 8 +- .../libzstd/include/zstd/legacy/zstd_v02.c | 8 +- .../libzstd/include/zstd/legacy/zstd_v02.h | 8 +- .../libzstd/include/zstd/legacy/zstd_v03.c | 8 +- .../libzstd/include/zstd/legacy/zstd_v03.h | 8 +- .../libzstd/include/zstd/legacy/zstd_v04.c | 10 +- .../libzstd/include/zstd/legacy/zstd_v04.h | 8 +- .../libzstd/include/zstd/legacy/zstd_v05.c | 10 +- .../libzstd/include/zstd/legacy/zstd_v05.h | 8 +- .../libzstd/include/zstd/legacy/zstd_v06.c | 10 +- .../libzstd/include/zstd/legacy/zstd_v06.h | 8 +- .../libzstd/include/zstd/legacy/zstd_v07.c | 8 +- .../libzstd/include/zstd/legacy/zstd_v07.h | 8 +- contrib/libzstd/include/zstd/zstd.h | 30 +- 45 files changed, 1164 insertions(+), 1113 deletions(-) diff --git a/contrib/libzstd/include/zstd/common/bitstream.h b/contrib/libzstd/include/zstd/common/bitstream.h index 07b85026c95..06121f21c5b 100644 --- a/contrib/libzstd/include/zstd/common/bitstream.h +++ b/contrib/libzstd/include/zstd/common/bitstream.h @@ -80,9 +80,9 @@ extern "C" { * bitStream encoding API (write forward) ********************************************/ /* bitStream can mix input from multiple sources. -* A critical property of these streams is that they encode and decode in **reverse** direction. -* So the first bit sequence you add will be the last to be read, like a LIFO stack. -*/ + * A critical property of these streams is that they encode and decode in **reverse** direction. + * So the first bit sequence you add will be the last to be read, like a LIFO stack. + */ typedef struct { size_t bitContainer; @@ -203,7 +203,7 @@ static const unsigned BIT_mask[] = { 0, 1, 3, 7, 0xF, 0x1F, 0x3F, 0x7F, /*! BIT_initCStream() : * `dstCapacity` must be > sizeof(size_t) * @return : 0 if success, - otherwise an error code (can be tested using ERR_isError() ) */ + * otherwise an error code (can be tested using ERR_isError()) */ MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, void* startPtr, size_t dstCapacity) { @@ -217,8 +217,8 @@ MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, } /*! BIT_addBits() : - can add up to 26 bits into `bitC`. - Does not check for register overflow ! */ + * can add up to 26 bits into `bitC`. + * Note : does not check for register overflow ! */ MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC, size_t value, unsigned nbBits) { @@ -268,7 +268,7 @@ MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC) /*! BIT_closeCStream() : * @return : size of CStream, in bytes, - or 0 if it could not fit into dstBuffer */ + * or 0 if it could not fit into dstBuffer */ MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC) { BIT_addBitsFast(bitC, 1, 1); /* endMark */ @@ -279,14 +279,14 @@ MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC) /*-******************************************************** -* bitStream decoding +* bitStream decoding **********************************************************/ /*! BIT_initDStream() : -* Initialize a BIT_DStream_t. -* `bitD` : a pointer to an already allocated BIT_DStream_t structure. -* `srcSize` must be the *exact* size of the bitStream, in bytes. -* @return : size of stream (== srcSize) or an errorCode if a problem is detected -*/ + * Initialize a BIT_DStream_t. + * `bitD` : a pointer to an already allocated BIT_DStream_t structure. + * `srcSize` must be the *exact* size of the bitStream, in bytes. + * @return : size of stream (== srcSize), or an errorCode if a problem is detected + */ MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize) { if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); } @@ -305,29 +305,30 @@ MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, si bitD->bitContainer = *(const BYTE*)(bitD->start); switch(srcSize) { - case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16); - /* fall-through */ + case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16); + /* fall-through */ - case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24); - /* fall-through */ + case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24); + /* fall-through */ - case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32); - /* fall-through */ + case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32); + /* fall-through */ - case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24; - /* fall-through */ + case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24; + /* fall-through */ - case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16; - /* fall-through */ + case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16; + /* fall-through */ - case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) << 8; - /* fall-through */ + case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) << 8; + /* fall-through */ - default: break; + default: break; + } + { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1]; + bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0; + if (lastByte == 0) return ERROR(corruption_detected); /* endMark not present */ } - { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1]; - bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0; - if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ } bitD->bitsConsumed += (U32)(sizeof(bitD->bitContainer) - srcSize)*8; } @@ -363,9 +364,8 @@ MEM_STATIC size_t BIT_getLowerBits(size_t bitContainer, U32 const nbBits) * local register is not modified. * On 32-bits, maxNbBits==24. * On 64-bits, maxNbBits==56. - * @return : value extracted - */ - MEM_STATIC size_t BIT_lookBits(const BIT_DStream_t* bitD, U32 nbBits) + * @return : value extracted */ +MEM_STATIC size_t BIT_lookBits(const BIT_DStream_t* bitD, U32 nbBits) { #if defined(__BMI__) && defined(__GNUC__) /* experimental; fails if bitD->bitsConsumed + nbBits > sizeof(bitD->bitContainer)*8 */ return BIT_getMiddleBits(bitD->bitContainer, (sizeof(bitD->bitContainer)*8) - bitD->bitsConsumed - nbBits, nbBits); @@ -392,8 +392,7 @@ MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits) /*! BIT_readBits() : * Read (consume) next n bits from local register and update. * Pay attention to not read more than nbBits contained into local register. - * @return : extracted value. - */ + * @return : extracted value. */ MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, U32 nbBits) { size_t const value = BIT_lookBits(bitD, nbBits); @@ -402,7 +401,7 @@ MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, U32 nbBits) } /*! BIT_readBitsFast() : -* unsafe version; only works only if nbBits >= 1 */ + * unsafe version; only works only if nbBits >= 1 */ MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, U32 nbBits) { size_t const value = BIT_lookBitsFast(bitD, nbBits); @@ -412,10 +411,10 @@ MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, U32 nbBits) } /*! BIT_reloadDStream() : -* Refill `bitD` from buffer previously set in BIT_initDStream() . -* This function is safe, it guarantees it will not read beyond src buffer. -* @return : status of `BIT_DStream_t` internal register. - if status == BIT_DStream_unfinished, internal register is filled with >= (sizeof(bitD->bitContainer)*8 - 7) bits */ + * Refill `bitD` from buffer previously set in BIT_initDStream() . + * This function is safe, it guarantees it will not read beyond src buffer. + * @return : status of `BIT_DStream_t` internal register. + * when status == BIT_DStream_unfinished, internal register is filled with at least 25 or 57 bits */ MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD) { if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8)) /* overflow detected, like end of stream */ @@ -446,8 +445,8 @@ MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD) } /*! BIT_endOfDStream() : -* @return Tells if DStream has exactly reached its end (all bits consumed). -*/ + * @return : 1 if DStream has _exactly_ reached its end (all bits consumed). + */ MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream) { return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8)); diff --git a/contrib/libzstd/include/zstd/common/error_private.c b/contrib/libzstd/include/zstd/common/error_private.c index 2d752cd23a7..b5b14b509cf 100644 --- a/contrib/libzstd/include/zstd/common/error_private.c +++ b/contrib/libzstd/include/zstd/common/error_private.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ /* The purpose of this file is to have a single list of error strings embedded in binary */ @@ -20,19 +20,17 @@ const char* ERR_getErrorString(ERR_enum code) case PREFIX(GENERIC): return "Error (generic)"; case PREFIX(prefix_unknown): return "Unknown frame descriptor"; case PREFIX(version_unsupported): return "Version not supported"; - case PREFIX(parameter_unknown): return "Unknown parameter type"; case PREFIX(frameParameter_unsupported): return "Unsupported frame parameter"; - case PREFIX(frameParameter_unsupportedBy32bits): return "Frame parameter unsupported in 32-bits mode"; case PREFIX(frameParameter_windowTooLarge): return "Frame requires too much memory for decoding"; - case PREFIX(compressionParameter_unsupported): return "Compression parameter is not supported"; - case PREFIX(compressionParameter_outOfBound): return "Compression parameter is out of bound"; + case PREFIX(corruption_detected): return "Corrupted block detected"; + case PREFIX(checksum_wrong): return "Restored data doesn't match checksum"; + case PREFIX(parameter_unsupported): return "Unsupported parameter"; + case PREFIX(parameter_outOfBound): return "Parameter is out of bound"; case PREFIX(init_missing): return "Context should be init first"; case PREFIX(memory_allocation): return "Allocation error : not enough memory"; case PREFIX(stage_wrong): return "Operation not authorized at current processing stage"; case PREFIX(dstSize_tooSmall): return "Destination buffer is too small"; case PREFIX(srcSize_wrong): return "Src size is incorrect"; - case PREFIX(corruption_detected): return "Corrupted block detected"; - case PREFIX(checksum_wrong): return "Restored data doesn't match checksum"; case PREFIX(tableLog_tooLarge): return "tableLog requires too much memory : unsupported"; case PREFIX(maxSymbolValue_tooLarge): return "Unsupported max Symbol Value : too large"; case PREFIX(maxSymbolValue_tooSmall): return "Specified maxSymbolValue is too small"; diff --git a/contrib/libzstd/include/zstd/common/error_private.h b/contrib/libzstd/include/zstd/common/error_private.h index 1bc2e495481..9dd9a87cfac 100644 --- a/contrib/libzstd/include/zstd/common/error_private.h +++ b/contrib/libzstd/include/zstd/common/error_private.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ /* Note : this module is expected to remain private, do not expose it */ diff --git a/contrib/libzstd/include/zstd/common/fse.h b/contrib/libzstd/include/zstd/common/fse.h index 6d5d41def19..1c44f837507 100644 --- a/contrib/libzstd/include/zstd/common/fse.h +++ b/contrib/libzstd/include/zstd/common/fse.h @@ -31,13 +31,14 @@ You can contact the author at : - Source repository : https://github.com/Cyan4973/FiniteStateEntropy ****************************************************************** */ -#ifndef FSE_H -#define FSE_H #if defined (__cplusplus) extern "C" { #endif +#ifndef FSE_H +#define FSE_H + /*-***************************************** * Dependencies @@ -297,8 +298,10 @@ FSE_decompress_usingDTable() result will tell how many bytes were regenerated (< If there is an error, the function will return an error code, which can be tested using FSE_isError(). (ex: dst buffer too small) */ +#endif /* FSE_H */ -#ifdef FSE_STATIC_LINKING_ONLY +#if defined(FSE_STATIC_LINKING_ONLY) && !defined(FSE_H_FSE_STATIC_LINKING_ONLY) +#define FSE_H_FSE_STATIC_LINKING_ONLY /* *** Dependency *** */ #include "bitstream.h" @@ -381,6 +384,11 @@ size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char symbolValue); size_t FSE_decompress_wksp(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, FSE_DTable* workSpace, unsigned maxLog); /**< same as FSE_decompress(), using an externally allocated `workSpace` produced with `FSE_DTABLE_SIZE_U32(maxLog)` */ +typedef enum { + FSE_repeat_none, /**< Cannot use the previous table */ + FSE_repeat_check, /**< Can use the previous table but it must be checked */ + FSE_repeat_valid /**< Can use the previous table and it is asumed to be valid */ + } FSE_repeat; /* ***************************************** * FSE symbol compression API @@ -694,5 +702,3 @@ MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr) #if defined (__cplusplus) } #endif - -#endif /* FSE_H */ diff --git a/contrib/libzstd/include/zstd/common/fse_decompress.c b/contrib/libzstd/include/zstd/common/fse_decompress.c index 8474a4c079b..8e3f0035f69 100644 --- a/contrib/libzstd/include/zstd/common/fse_decompress.c +++ b/contrib/libzstd/include/zstd/common/fse_decompress.c @@ -33,35 +33,16 @@ ****************************************************************** */ -/* ************************************************************** -* Compiler specifics -****************************************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define FORCE_INLINE static __forceinline -# include /* For Visual 2005 */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ -#else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif - - /* ************************************************************** * Includes ****************************************************************/ #include /* malloc, free, qsort */ #include /* memcpy, memset */ #include "bitstream.h" +#include "compiler.h" #define FSE_STATIC_LINKING_ONLY #include "fse.h" +#include "error_private.h" /* ************************************************************** @@ -216,7 +197,7 @@ size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits) return 0; } -FORCE_INLINE size_t FSE_decompress_usingDTable_generic( +FORCE_INLINE_TEMPLATE size_t FSE_decompress_usingDTable_generic( void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const FSE_DTable* dt, const unsigned fast) diff --git a/contrib/libzstd/include/zstd/common/huf.h b/contrib/libzstd/include/zstd/common/huf.h index dabd359915a..2b3015a84c1 100644 --- a/contrib/libzstd/include/zstd/common/huf.h +++ b/contrib/libzstd/include/zstd/common/huf.h @@ -31,13 +31,13 @@ You can contact the author at : - Source repository : https://github.com/Cyan4973/FiniteStateEntropy ****************************************************************** */ -#ifndef HUF_H_298734234 -#define HUF_H_298734234 #if defined (__cplusplus) extern "C" { #endif +#ifndef HUF_H_298734234 +#define HUF_H_298734234 /* *** Dependencies *** */ #include /* size_t */ @@ -124,6 +124,7 @@ HUF_PUBLIC_API size_t HUF_compress4X_wksp (void* dst, size_t dstCapacity, const #define HUF_DECOMPRESS_WORKSPACE_SIZE (2 << 10) #define HUF_DECOMPRESS_WORKSPACE_SIZE_U32 (HUF_DECOMPRESS_WORKSPACE_SIZE / sizeof(U32)) +#endif /* HUF_H_298734234 */ /* ****************************************************************** * WARNING !! @@ -132,7 +133,8 @@ HUF_PUBLIC_API size_t HUF_compress4X_wksp (void* dst, size_t dstCapacity, const * because they are not guaranteed to remain stable in the future. * Only consider them in association with static linking. *******************************************************************/ -#ifdef HUF_STATIC_LINKING_ONLY +#if defined(HUF_STATIC_LINKING_ONLY) && !defined(HUF_H_HUF_STATIC_LINKING_ONLY) +#define HUF_H_HUF_STATIC_LINKING_ONLY /* *** Dependencies *** */ #include "mem.h" /* U32 */ @@ -295,9 +297,6 @@ size_t HUF_decompress1X4_usingDTable(void* dst, size_t maxDstSize, const void* c #endif /* HUF_STATIC_LINKING_ONLY */ - #if defined (__cplusplus) } #endif - -#endif /* HUF_H_298734234 */ diff --git a/contrib/libzstd/include/zstd/common/mem.h b/contrib/libzstd/include/zstd/common/mem.h index b0e5bf60b43..df85404fb86 100644 --- a/contrib/libzstd/include/zstd/common/mem.h +++ b/contrib/libzstd/include/zstd/common/mem.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef MEM_H_MODULE @@ -110,7 +110,7 @@ Only use if no other choice to achieve best performance on target platform */ MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; } MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; } MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; } -MEM_STATIC U64 MEM_readST(const void* memPtr) { return *(const size_t*) memPtr; } +MEM_STATIC size_t MEM_readST(const void* memPtr) { return *(const size_t*) memPtr; } MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; } MEM_STATIC void MEM_write32(void* memPtr, U32 value) { *(U32*)memPtr = value; } @@ -131,7 +131,7 @@ MEM_STATIC void MEM_write64(void* memPtr, U64 value) { *(U64*)memPtr = value; } MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; } MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; } MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; } -MEM_STATIC U64 MEM_readST(const void* ptr) { return ((const unalign*)ptr)->st; } +MEM_STATIC size_t MEM_readST(const void* ptr) { return ((const unalign*)ptr)->st; } MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; } MEM_STATIC void MEM_write32(void* memPtr, U32 value) { ((unalign*)memPtr)->u32 = value; } diff --git a/contrib/libzstd/include/zstd/common/pool.c b/contrib/libzstd/include/zstd/common/pool.c index 749fa4f2f7b..a227044f7f1 100644 --- a/contrib/libzstd/include/zstd/common/pool.c +++ b/contrib/libzstd/include/zstd/common/pool.c @@ -1,10 +1,10 @@ -/** - * Copyright (c) 2016-present, Facebook, Inc. +/* + * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -39,6 +39,12 @@ struct POOL_ctx_s { size_t queueHead; size_t queueTail; size_t queueSize; + + /* The number of threads working on jobs */ + size_t numThreadsBusy; + /* Indicates if the queue is empty */ + int queueEmpty; + /* The mutex protects the queue */ pthread_mutex_t queueMutex; /* Condition variable for pushers to wait on when the queue is full */ @@ -60,30 +66,41 @@ static void* POOL_thread(void* opaque) { for (;;) { /* Lock the mutex and wait for a non-empty queue or until shutdown */ pthread_mutex_lock(&ctx->queueMutex); - while (ctx->queueHead == ctx->queueTail && !ctx->shutdown) { + + while (ctx->queueEmpty && !ctx->shutdown) { pthread_cond_wait(&ctx->queuePopCond, &ctx->queueMutex); } /* empty => shutting down: so stop */ - if (ctx->queueHead == ctx->queueTail) { + if (ctx->queueEmpty) { pthread_mutex_unlock(&ctx->queueMutex); return opaque; } /* Pop a job off the queue */ { POOL_job const job = ctx->queue[ctx->queueHead]; ctx->queueHead = (ctx->queueHead + 1) % ctx->queueSize; + ctx->numThreadsBusy++; + ctx->queueEmpty = ctx->queueHead == ctx->queueTail; /* Unlock the mutex, signal a pusher, and run the job */ pthread_mutex_unlock(&ctx->queueMutex); pthread_cond_signal(&ctx->queuePushCond); + job.function(job.opaque); - } - } + + /* If the intended queue size was 0, signal after finishing job */ + if (ctx->queueSize == 1) { + pthread_mutex_lock(&ctx->queueMutex); + ctx->numThreadsBusy--; + pthread_mutex_unlock(&ctx->queueMutex); + pthread_cond_signal(&ctx->queuePushCond); + } } + } /* for (;;) */ /* Unreachable */ } POOL_ctx *POOL_create(size_t numThreads, size_t queueSize) { POOL_ctx *ctx; /* Check the parameters */ - if (!numThreads || !queueSize) { return NULL; } + if (!numThreads) { return NULL; } /* Allocate the context and zero initialize */ ctx = (POOL_ctx *)calloc(1, sizeof(POOL_ctx)); if (!ctx) { return NULL; } @@ -92,15 +109,17 @@ POOL_ctx *POOL_create(size_t numThreads, size_t queueSize) { * and full queues. */ ctx->queueSize = queueSize + 1; - ctx->queue = (POOL_job *)malloc(ctx->queueSize * sizeof(POOL_job)); + ctx->queue = (POOL_job*) malloc(ctx->queueSize * sizeof(POOL_job)); ctx->queueHead = 0; ctx->queueTail = 0; - pthread_mutex_init(&ctx->queueMutex, NULL); - pthread_cond_init(&ctx->queuePushCond, NULL); - pthread_cond_init(&ctx->queuePopCond, NULL); + ctx->numThreadsBusy = 0; + ctx->queueEmpty = 1; + (void)pthread_mutex_init(&ctx->queueMutex, NULL); + (void)pthread_cond_init(&ctx->queuePushCond, NULL); + (void)pthread_cond_init(&ctx->queuePopCond, NULL); ctx->shutdown = 0; /* Allocate space for the thread handles */ - ctx->threads = (pthread_t *)malloc(numThreads * sizeof(pthread_t)); + ctx->threads = (pthread_t*)malloc(numThreads * sizeof(pthread_t)); ctx->numThreads = 0; /* Check for errors */ if (!ctx->threads || !ctx->queue) { POOL_free(ctx); return NULL; } @@ -153,22 +172,37 @@ size_t POOL_sizeof(POOL_ctx *ctx) { + ctx->numThreads * sizeof(pthread_t); } -void POOL_add(void *ctxVoid, POOL_function function, void *opaque) { - POOL_ctx *ctx = (POOL_ctx *)ctxVoid; +/** + * Returns 1 if the queue is full and 0 otherwise. + * + * If the queueSize is 1 (the pool was created with an intended queueSize of 0), + * then a queue is empty if there is a thread free and no job is waiting. + */ +static int isQueueFull(POOL_ctx const* ctx) { + if (ctx->queueSize > 1) { + return ctx->queueHead == ((ctx->queueTail + 1) % ctx->queueSize); + } else { + return ctx->numThreadsBusy == ctx->numThreads || + !ctx->queueEmpty; + } +} + +void POOL_add(void* ctxVoid, POOL_function function, void *opaque) { + POOL_ctx* const ctx = (POOL_ctx*)ctxVoid; if (!ctx) { return; } pthread_mutex_lock(&ctx->queueMutex); { POOL_job const job = {function, opaque}; + /* Wait until there is space in the queue for the new job */ - size_t newTail = (ctx->queueTail + 1) % ctx->queueSize; - while (ctx->queueHead == newTail && !ctx->shutdown) { + while (isQueueFull(ctx) && !ctx->shutdown) { pthread_cond_wait(&ctx->queuePushCond, &ctx->queueMutex); - newTail = (ctx->queueTail + 1) % ctx->queueSize; } /* The queue is still going => there is space */ if (!ctx->shutdown) { + ctx->queueEmpty = 0; ctx->queue[ctx->queueTail] = job; - ctx->queueTail = newTail; + ctx->queueTail = (ctx->queueTail + 1) % ctx->queueSize; } } pthread_mutex_unlock(&ctx->queueMutex); @@ -183,22 +217,22 @@ struct POOL_ctx_s { int data; }; -POOL_ctx *POOL_create(size_t numThreads, size_t queueSize) { +POOL_ctx* POOL_create(size_t numThreads, size_t queueSize) { (void)numThreads; (void)queueSize; - return (POOL_ctx *)malloc(sizeof(POOL_ctx)); + return (POOL_ctx*)malloc(sizeof(POOL_ctx)); } -void POOL_free(POOL_ctx *ctx) { - if (ctx) free(ctx); +void POOL_free(POOL_ctx* ctx) { + free(ctx); } -void POOL_add(void *ctx, POOL_function function, void *opaque) { +void POOL_add(void* ctx, POOL_function function, void* opaque) { (void)ctx; function(opaque); } -size_t POOL_sizeof(POOL_ctx *ctx) { +size_t POOL_sizeof(POOL_ctx* ctx) { if (ctx==NULL) return 0; /* supports sizeof NULL */ return sizeof(*ctx); } diff --git a/contrib/libzstd/include/zstd/common/pool.h b/contrib/libzstd/include/zstd/common/pool.h index 386cd674b7c..264c5c9ca7e 100644 --- a/contrib/libzstd/include/zstd/common/pool.h +++ b/contrib/libzstd/include/zstd/common/pool.h @@ -1,11 +1,12 @@ -/** - * Copyright (c) 2016-present, Facebook, Inc. +/* + * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ + #ifndef POOL_H #define POOL_H @@ -19,11 +20,10 @@ extern "C" { typedef struct POOL_ctx_s POOL_ctx; /*! POOL_create() : - Create a thread pool with at most `numThreads` threads. - `numThreads` must be at least 1. - The maximum number of queued jobs before blocking is `queueSize`. - `queueSize` must be at least 1. - @return : The POOL_ctx pointer on success else NULL. + * Create a thread pool with at most `numThreads` threads. + * `numThreads` must be at least 1. + * The maximum number of queued jobs before blocking is `queueSize`. + * @return : POOL_ctx pointer on success, else NULL. */ POOL_ctx *POOL_create(size_t numThreads, size_t queueSize); diff --git a/contrib/libzstd/include/zstd/common/threading.h b/contrib/libzstd/include/zstd/common/threading.h index c0086139ea3..ab09977a86d 100644 --- a/contrib/libzstd/include/zstd/common/threading.h +++ b/contrib/libzstd/include/zstd/common/threading.h @@ -1,4 +1,3 @@ - /** * Copyright (c) 2016 Tino Reichardt * All rights reserved. @@ -42,14 +41,14 @@ extern "C" { /* mutex */ #define pthread_mutex_t CRITICAL_SECTION -#define pthread_mutex_init(a,b) InitializeCriticalSection((a)) +#define pthread_mutex_init(a,b) (InitializeCriticalSection((a)), 0) #define pthread_mutex_destroy(a) DeleteCriticalSection((a)) #define pthread_mutex_lock(a) EnterCriticalSection((a)) #define pthread_mutex_unlock(a) LeaveCriticalSection((a)) /* condition variable */ #define pthread_cond_t CONDITION_VARIABLE -#define pthread_cond_init(a, b) InitializeConditionVariable((a)) +#define pthread_cond_init(a, b) (InitializeConditionVariable((a)), 0) #define pthread_cond_destroy(a) /* No delete */ #define pthread_cond_wait(a, b) SleepConditionVariableCS((a), (b), INFINITE) #define pthread_cond_signal(a) WakeConditionVariable((a)) @@ -80,14 +79,14 @@ int _pthread_join(pthread_t* thread, void** value_ptr); #else /* ZSTD_MULTITHREAD not defined */ /* No multithreading support */ -#define pthread_mutex_t int /* #define rather than typedef, as sometimes pthread support is implicit, resulting in duplicated symbols */ -#define pthread_mutex_init(a,b) +#define pthread_mutex_t int /* #define rather than typedef, because sometimes pthread support is implicit, resulting in duplicated symbols */ +#define pthread_mutex_init(a,b) ((void)a, 0) #define pthread_mutex_destroy(a) #define pthread_mutex_lock(a) #define pthread_mutex_unlock(a) #define pthread_cond_t int -#define pthread_cond_init(a,b) +#define pthread_cond_init(a,b) ((void)a, 0) #define pthread_cond_destroy(a) #define pthread_cond_wait(a,b) #define pthread_cond_signal(a) diff --git a/contrib/libzstd/include/zstd/common/xxhash.c b/contrib/libzstd/include/zstd/common/xxhash.c index eb44222c5fc..9d9c0e963cb 100644 --- a/contrib/libzstd/include/zstd/common/xxhash.c +++ b/contrib/libzstd/include/zstd/common/xxhash.c @@ -113,19 +113,25 @@ static void* XXH_memcpy(void* dest, const void* src, size_t size) { return memcp /* ************************************* * Compiler Specific Options ***************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# define FORCE_INLINE static __forceinline +#if defined (__GNUC__) || defined(__cplusplus) || defined(__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ +# define INLINE_KEYWORD inline #else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ +# define INLINE_KEYWORD +#endif + +#if defined(__GNUC__) +# define FORCE_INLINE_ATTR __attribute__((always_inline)) +#elif defined(_MSC_VER) +# define FORCE_INLINE_ATTR __forceinline +#else +# define FORCE_INLINE_ATTR +#endif + +#define FORCE_INLINE_TEMPLATE static INLINE_KEYWORD FORCE_INLINE_ATTR + + +#ifdef _MSC_VER +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ #endif @@ -248,7 +254,7 @@ typedef enum { XXH_bigEndian=0, XXH_littleEndian=1 } XXH_endianess; *****************************/ typedef enum { XXH_aligned, XXH_unaligned } XXH_alignment; -FORCE_INLINE U32 XXH_readLE32_align(const void* ptr, XXH_endianess endian, XXH_alignment align) +FORCE_INLINE_TEMPLATE U32 XXH_readLE32_align(const void* ptr, XXH_endianess endian, XXH_alignment align) { if (align==XXH_unaligned) return endian==XXH_littleEndian ? XXH_read32(ptr) : XXH_swap32(XXH_read32(ptr)); @@ -256,7 +262,7 @@ FORCE_INLINE U32 XXH_readLE32_align(const void* ptr, XXH_endianess endian, XXH_a return endian==XXH_littleEndian ? *(const U32*)ptr : XXH_swap32(*(const U32*)ptr); } -FORCE_INLINE U32 XXH_readLE32(const void* ptr, XXH_endianess endian) +FORCE_INLINE_TEMPLATE U32 XXH_readLE32(const void* ptr, XXH_endianess endian) { return XXH_readLE32_align(ptr, endian, XXH_unaligned); } @@ -266,7 +272,7 @@ static U32 XXH_readBE32(const void* ptr) return XXH_CPU_LITTLE_ENDIAN ? XXH_swap32(XXH_read32(ptr)) : XXH_read32(ptr); } -FORCE_INLINE U64 XXH_readLE64_align(const void* ptr, XXH_endianess endian, XXH_alignment align) +FORCE_INLINE_TEMPLATE U64 XXH_readLE64_align(const void* ptr, XXH_endianess endian, XXH_alignment align) { if (align==XXH_unaligned) return endian==XXH_littleEndian ? XXH_read64(ptr) : XXH_swap64(XXH_read64(ptr)); @@ -274,7 +280,7 @@ FORCE_INLINE U64 XXH_readLE64_align(const void* ptr, XXH_endianess endian, XXH_a return endian==XXH_littleEndian ? *(const U64*)ptr : XXH_swap64(*(const U64*)ptr); } -FORCE_INLINE U64 XXH_readLE64(const void* ptr, XXH_endianess endian) +FORCE_INLINE_TEMPLATE U64 XXH_readLE64(const void* ptr, XXH_endianess endian) { return XXH_readLE64_align(ptr, endian, XXH_unaligned); } @@ -335,7 +341,7 @@ static U32 XXH32_round(U32 seed, U32 input) return seed; } -FORCE_INLINE U32 XXH32_endian_align(const void* input, size_t len, U32 seed, XXH_endianess endian, XXH_alignment align) +FORCE_INLINE_TEMPLATE U32 XXH32_endian_align(const void* input, size_t len, U32 seed, XXH_endianess endian, XXH_alignment align) { const BYTE* p = (const BYTE*)input; const BYTE* bEnd = p + len; @@ -435,7 +441,7 @@ static U64 XXH64_mergeRound(U64 acc, U64 val) return acc; } -FORCE_INLINE U64 XXH64_endian_align(const void* input, size_t len, U64 seed, XXH_endianess endian, XXH_alignment align) +FORCE_INLINE_TEMPLATE U64 XXH64_endian_align(const void* input, size_t len, U64 seed, XXH_endianess endian, XXH_alignment align) { const BYTE* p = (const BYTE*)input; const BYTE* const bEnd = p + len; @@ -584,7 +590,7 @@ XXH_PUBLIC_API XXH_errorcode XXH64_reset(XXH64_state_t* statePtr, unsigned long } -FORCE_INLINE XXH_errorcode XXH32_update_endian (XXH32_state_t* state, const void* input, size_t len, XXH_endianess endian) +FORCE_INLINE_TEMPLATE XXH_errorcode XXH32_update_endian (XXH32_state_t* state, const void* input, size_t len, XXH_endianess endian) { const BYTE* p = (const BYTE*)input; const BYTE* const bEnd = p + len; @@ -654,7 +660,7 @@ XXH_PUBLIC_API XXH_errorcode XXH32_update (XXH32_state_t* state_in, const void* -FORCE_INLINE U32 XXH32_digest_endian (const XXH32_state_t* state, XXH_endianess endian) +FORCE_INLINE_TEMPLATE U32 XXH32_digest_endian (const XXH32_state_t* state, XXH_endianess endian) { const BYTE * p = (const BYTE*)state->mem32; const BYTE* const bEnd = (const BYTE*)(state->mem32) + state->memsize; @@ -704,7 +710,7 @@ XXH_PUBLIC_API unsigned int XXH32_digest (const XXH32_state_t* state_in) /* **** XXH64 **** */ -FORCE_INLINE XXH_errorcode XXH64_update_endian (XXH64_state_t* state, const void* input, size_t len, XXH_endianess endian) +FORCE_INLINE_TEMPLATE XXH_errorcode XXH64_update_endian (XXH64_state_t* state, const void* input, size_t len, XXH_endianess endian) { const BYTE* p = (const BYTE*)input; const BYTE* const bEnd = p + len; @@ -771,7 +777,7 @@ XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH64_state_t* state_in, const void* -FORCE_INLINE U64 XXH64_digest_endian (const XXH64_state_t* state, XXH_endianess endian) +FORCE_INLINE_TEMPLATE U64 XXH64_digest_endian (const XXH64_state_t* state, XXH_endianess endian) { const BYTE * p = (const BYTE*)state->mem64; const BYTE* const bEnd = (const BYTE*)state->mem64 + state->memsize; diff --git a/contrib/libzstd/include/zstd/common/zstd_common.c b/contrib/libzstd/include/zstd/common/zstd_common.c index f6816723815..08384cabf59 100644 --- a/contrib/libzstd/include/zstd/common/zstd_common.c +++ b/contrib/libzstd/include/zstd/common/zstd_common.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ diff --git a/contrib/libzstd/include/zstd/common/zstd_errors.h b/contrib/libzstd/include/zstd/common/zstd_errors.h index 19f1597aa34..a69387b714a 100644 --- a/contrib/libzstd/include/zstd/common/zstd_errors.h +++ b/contrib/libzstd/include/zstd/common/zstd_errors.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTD_ERRORS_H_398273423 @@ -37,43 +37,41 @@ extern "C" { /*-**************************************** * error codes list * note : this API is still considered unstable - * it should not be used with a dynamic library + * and shall not be used with a dynamic library. * only static linking is allowed ******************************************/ typedef enum { - ZSTD_error_no_error, - ZSTD_error_GENERIC, - ZSTD_error_prefix_unknown, - ZSTD_error_version_unsupported, - ZSTD_error_parameter_unknown, - ZSTD_error_frameParameter_unsupported, - ZSTD_error_frameParameter_unsupportedBy32bits, - ZSTD_error_frameParameter_windowTooLarge, - ZSTD_error_compressionParameter_unsupported, - ZSTD_error_compressionParameter_outOfBound, - ZSTD_error_init_missing, - ZSTD_error_memory_allocation, - ZSTD_error_stage_wrong, - ZSTD_error_dstSize_tooSmall, - ZSTD_error_srcSize_wrong, - ZSTD_error_corruption_detected, - ZSTD_error_checksum_wrong, - ZSTD_error_tableLog_tooLarge, - ZSTD_error_maxSymbolValue_tooLarge, - ZSTD_error_maxSymbolValue_tooSmall, - ZSTD_error_dictionary_corrupted, - ZSTD_error_dictionary_wrong, - ZSTD_error_dictionaryCreation_failed, - ZSTD_error_frameIndex_tooLarge, - ZSTD_error_seekableIO, - ZSTD_error_maxCode + ZSTD_error_no_error = 0, + ZSTD_error_GENERIC = 1, + ZSTD_error_prefix_unknown = 10, + ZSTD_error_version_unsupported = 12, + ZSTD_error_frameParameter_unsupported = 14, + ZSTD_error_frameParameter_windowTooLarge = 16, + ZSTD_error_corruption_detected = 20, + ZSTD_error_checksum_wrong = 22, + ZSTD_error_dictionary_corrupted = 30, + ZSTD_error_dictionary_wrong = 32, + ZSTD_error_dictionaryCreation_failed = 34, + ZSTD_error_parameter_unsupported = 40, + ZSTD_error_parameter_outOfBound = 42, + ZSTD_error_tableLog_tooLarge = 44, + ZSTD_error_maxSymbolValue_tooLarge = 46, + ZSTD_error_maxSymbolValue_tooSmall = 48, + ZSTD_error_stage_wrong = 60, + ZSTD_error_init_missing = 62, + ZSTD_error_memory_allocation = 64, + ZSTD_error_dstSize_tooSmall = 70, + ZSTD_error_srcSize_wrong = 72, + ZSTD_error_frameIndex_tooLarge = 100, + ZSTD_error_seekableIO = 102, + ZSTD_error_maxCode = 120 /* never EVER use this value directly, it may change in future versions! Use ZSTD_isError() instead */ } ZSTD_ErrorCode; /*! ZSTD_getErrorCode() : convert a `size_t` function result into a `ZSTD_ErrorCode` enum type, which can be used to compare with enum list published above */ ZSTDERRORLIB_API ZSTD_ErrorCode ZSTD_getErrorCode(size_t functionResult); -ZSTDERRORLIB_API const char* ZSTD_getErrorString(ZSTD_ErrorCode code); +ZSTDERRORLIB_API const char* ZSTD_getErrorString(ZSTD_ErrorCode code); /**< Same as ZSTD_getErrorName, but using a `ZSTD_ErrorCode` enum argument */ #if defined (__cplusplus) diff --git a/contrib/libzstd/include/zstd/common/zstd_internal.h b/contrib/libzstd/include/zstd/common/zstd_internal.h index f2c4e6249fb..2610528608d 100644 --- a/contrib/libzstd/include/zstd/common/zstd_internal.h +++ b/contrib/libzstd/include/zstd/common/zstd_internal.h @@ -1,55 +1,28 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTD_CCOMMON_H_MODULE #define ZSTD_CCOMMON_H_MODULE -/*-******************************************************* -* Compiler specifics -*********************************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define FORCE_INLINE static __forceinline -# include /* For Visual 2005 */ -# pragma warning(disable : 4100) /* disable: C4100: unreferenced formal parameter */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4204) /* disable: C4204: non-constant aggregate initializer */ -# pragma warning(disable : 4324) /* disable: C4324: padded structure */ -#else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif - -#ifdef _MSC_VER -# define FORCE_NOINLINE static __declspec(noinline) -#else -# ifdef __GNUC__ -# define FORCE_NOINLINE static __attribute__((__noinline__)) -# else -# define FORCE_NOINLINE static -# endif -#endif - /*-************************************* * Dependencies ***************************************/ +#include "compiler.h" #include "mem.h" #include "error_private.h" #define ZSTD_STATIC_LINKING_ONLY #include "zstd.h" +#define FSE_STATIC_LINKING_ONLY +#include "fse.h" +#define HUF_STATIC_LINKING_ONLY +#include "huf.h" #ifndef XXH_STATIC_LINKING_ONLY # define XXH_STATIC_LINKING_ONLY /* XXH64_state_t */ #endif @@ -211,20 +184,6 @@ MEM_STATIC void ZSTD_wildcopy_e(void* dst, const void* src, void* dstEnd) /* s *********************************************/ typedef struct ZSTD_stats_s ZSTD_stats_t; -typedef struct { - U32 off; - U32 len; -} ZSTD_match_t; - -typedef struct { - U32 price; - U32 off; - U32 mlen; - U32 litlen; - U32 rep[ZSTD_REP_NUM]; -} ZSTD_optimal_t; - - typedef struct seqDef_s { U32 offset; U16 litLength; @@ -242,13 +201,31 @@ typedef struct { BYTE* ofCode; U32 longLengthID; /* 0 == no longLength; 1 == Lit.longLength; 2 == Match.longLength; */ U32 longLengthPos; - /* opt */ - ZSTD_optimal_t* priceTable; - ZSTD_match_t* matchTable; - U32* matchLengthFreq; - U32* litLengthFreq; + U32 rep[ZSTD_REP_NUM]; + U32 repToConfirm[ZSTD_REP_NUM]; +} seqStore_t; + +typedef struct { + U32 off; + U32 len; +} ZSTD_match_t; + +typedef struct { + U32 price; + U32 off; + U32 mlen; + U32 litlen; + U32 rep[ZSTD_REP_NUM]; +} ZSTD_optimal_t; + +typedef struct { U32* litFreq; + U32* litLengthFreq; + U32* matchLengthFreq; U32* offCodeFreq; + ZSTD_match_t* matchTable; + ZSTD_optimal_t* priceTable; + U32 matchLengthSum; U32 matchSum; U32 litLengthSum; @@ -264,7 +241,19 @@ typedef struct { U32 cachedPrice; U32 cachedLitLength; const BYTE* cachedLiterals; -} seqStore_t; +} optState_t; + +typedef struct { + U32 hufCTable[HUF_CTABLE_SIZE_U32(255)]; + FSE_CTable offcodeCTable[FSE_CTABLE_SIZE_U32(OffFSELog, MaxOff)]; + FSE_CTable matchlengthCTable[FSE_CTABLE_SIZE_U32(MLFSELog, MaxML)]; + FSE_CTable litlengthCTable[FSE_CTABLE_SIZE_U32(LLFSELog, MaxLL)]; + U32 workspace[HUF_WORKSPACE_SIZE_U32]; + HUF_repeat hufCTable_repeatMode; + FSE_repeat offcode_repeatMode; + FSE_repeat matchlength_repeatMode; + FSE_repeat litlength_repeatMode; +} ZSTD_entropyCTables_t; const seqStore_t* ZSTD_getSeqStore(const ZSTD_CCtx* ctx); void ZSTD_seqToCodes(const seqStore_t* seqStorePtr); @@ -331,4 +320,16 @@ size_t ZSTD_compressStream_generic(ZSTD_CStream* zcs, ZSTD_parameters ZSTD_getParamsFromCDict(const ZSTD_CDict* cdict); +typedef struct { + blockType_e blockType; + U32 lastBlock; + U32 origSize; +} blockProperties_t; + +/*! ZSTD_getcBlockSize() : +* Provides the size of compressed block from block header `src` */ +size_t ZSTD_getcBlockSize(const void* src, size_t srcSize, + blockProperties_t* bpPtr); + + #endif /* ZSTD_CCOMMON_H_MODULE */ diff --git a/contrib/libzstd/include/zstd/compress/fse_compress.c b/contrib/libzstd/include/zstd/compress/fse_compress.c index 26e8052ddcc..cc9fa73514a 100644 --- a/contrib/libzstd/include/zstd/compress/fse_compress.c +++ b/contrib/libzstd/include/zstd/compress/fse_compress.c @@ -32,27 +32,6 @@ - Public forum : https://groups.google.com/forum/#!forum/lz4c ****************************************************************** */ -/* ************************************************************** -* Compiler specifics -****************************************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define FORCE_INLINE static __forceinline -# include /* For Visual 2005 */ -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ -#else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif - - /* ************************************************************** * Includes ****************************************************************/ @@ -60,13 +39,16 @@ #include /* memcpy, memset */ #include /* printf (debug) */ #include "bitstream.h" +#include "compiler.h" #define FSE_STATIC_LINKING_ONLY #include "fse.h" +#include "error_private.h" /* ************************************************************** * Error Management ****************************************************************/ +#define FSE_isError ERR_isError #define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ @@ -781,7 +763,7 @@ size_t FSE_compress_usingCTable (void* dst, size_t dstSize, size_t FSE_compressBound(size_t size) { return FSE_COMPRESSBOUND(size); } -#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return f +#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return e #define CHECK_F(f) { CHECK_V_F(_var_err__, f); } /* FSE_compress_wksp() : diff --git a/contrib/libzstd/include/zstd/compress/huf_compress.c b/contrib/libzstd/include/zstd/compress/huf_compress.c index 7af0789a9c5..2a47c182052 100644 --- a/contrib/libzstd/include/zstd/compress/huf_compress.c +++ b/contrib/libzstd/include/zstd/compress/huf_compress.c @@ -50,13 +50,15 @@ #include "fse.h" /* header compression */ #define HUF_STATIC_LINKING_ONLY #include "huf.h" +#include "error_private.h" /* ************************************************************** * Error Management ****************************************************************/ +#define HUF_isError ERR_isError #define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ -#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return f +#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return e #define CHECK_F(f) { CHECK_V_F(_var_err__, f); } @@ -436,7 +438,7 @@ static void HUF_encodeSymbol(BIT_CStream_t* bitCPtr, U32 symbol, const HUF_CElt* size_t HUF_compressBound(size_t size) { return HUF_COMPRESSBOUND(size); } -#define HUF_FLUSHBITS(s) (fast ? BIT_flushBitsFast(s) : BIT_flushBits(s)) +#define HUF_FLUSHBITS(s) BIT_flushBits(s) #define HUF_FLUSHBITS_1(stream) \ if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*2+7) HUF_FLUSHBITS(stream) @@ -451,7 +453,6 @@ size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, si BYTE* const oend = ostart + dstSize; BYTE* op = ostart; size_t n; - const unsigned fast = (dstSize >= HUF_BLOCKBOUND(srcSize)); BIT_CStream_t bitC; /* init */ diff --git a/contrib/libzstd/include/zstd/compress/zstd_compress.c b/contrib/libzstd/include/zstd/compress/zstd_compress.c index 9300357f2d3..0322c03eb31 100644 --- a/contrib/libzstd/include/zstd/compress/zstd_compress.c +++ b/contrib/libzstd/include/zstd/compress/zstd_compress.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -36,13 +36,6 @@ static const U32 g_searchStrength = 8; /* control skip over incompressible dat #define HASH_READ_SIZE 8 typedef enum { ZSTDcs_created=0, ZSTDcs_init, ZSTDcs_ongoing, ZSTDcs_ending } ZSTD_compressionStage_e; -/* entropy tables always have same size */ -static size_t const hufCTable_size = HUF_CTABLE_SIZE(255); -static size_t const litlengthCTable_size = FSE_CTABLE_SIZE(LLFSELog, MaxLL); -static size_t const offcodeCTable_size = FSE_CTABLE_SIZE(OffFSELog, MaxOff); -static size_t const matchlengthCTable_size = FSE_CTABLE_SIZE(MLFSELog, MaxML); -static size_t const entropyScratchSpace_size = HUF_WORKSPACE_SIZE; - /*-************************************* * Helper functions @@ -89,8 +82,6 @@ struct ZSTD_CCtx_s { U32 loadedDictEnd; /* index of end of dictionary */ U32 forceWindow; /* force back-references to respect limit of 1<workSpaceSize = workspaceSize - sizeof(ZSTD_CCtx); /* entropy space (never moves) */ - /* note : this code should be shared with resetCCtx, rather than copy/pasted */ - { void* ptr = cctx->workSpace; - cctx->hufCTable = (HUF_CElt*)ptr; - ptr = (char*)cctx->hufCTable + hufCTable_size; - cctx->offcodeCTable = (FSE_CTable*) ptr; - ptr = (char*)ptr + offcodeCTable_size; - cctx->matchlengthCTable = (FSE_CTable*) ptr; - ptr = (char*)ptr + matchlengthCTable_size; - cctx->litlengthCTable = (FSE_CTable*) ptr; - ptr = (char*)ptr + litlengthCTable_size; - assert(((size_t)ptr & 3) == 0); /* ensure correct alignment */ - cctx->entropyScratchSpace = (unsigned*) ptr; - } + if (cctx->workSpaceSize < sizeof(ZSTD_entropyCTables_t)) return NULL; + assert(((size_t)cctx->workSpace & 7) == 0); /* ensure correct alignment */ + cctx->entropy = (ZSTD_entropyCTables_t*)cctx->workSpace; return cctx; } @@ -237,7 +213,7 @@ size_t ZSTD_setCCtxParameter(ZSTD_CCtx* cctx, ZSTD_CCtxParameter param, unsigned ZSTD_STATIC_ASSERT(ZSTD_dm_auto==0); ZSTD_STATIC_ASSERT(ZSTD_dm_rawContent==1); case ZSTD_p_forceRawDict : cctx->dictMode = (ZSTD_dictMode_e)(value>0); return 0; - default: return ERROR(parameter_unknown); + default: return ERROR(parameter_unsupported); } } @@ -251,9 +227,9 @@ static void ZSTD_cLevelToCParams(ZSTD_CCtx* cctx) cctx->compressionLevel = ZSTD_CLEVEL_CUSTOM; } -#define CLAMPCHECK(val,min,max) { \ - if (((val)<(min)) | ((val)>(max))) { \ - return ERROR(compressionParameter_outOfBound); \ +#define CLAMPCHECK(val,min,max) { \ + if (((val)<(min)) | ((val)>(max))) { \ + return ERROR(parameter_outOfBound); \ } } size_t ZSTD_CCtx_setParameter(ZSTD_CCtx* cctx, ZSTD_cParameter param, unsigned value) @@ -349,7 +325,7 @@ size_t ZSTD_CCtx_setParameter(ZSTD_CCtx* cctx, ZSTD_cParameter param, unsigned v /* restrict dictionary mode, to "rawContent" or "fullDict" only */ ZSTD_STATIC_ASSERT((U32)ZSTD_dm_fullDict > (U32)ZSTD_dm_rawContent); if (value > (unsigned)ZSTD_dm_fullDict) - return ERROR(compressionParameter_outOfBound); + return ERROR(parameter_outOfBound); cctx->dictMode = (ZSTD_dictMode_e)value; return 0; @@ -370,31 +346,31 @@ size_t ZSTD_CCtx_setParameter(ZSTD_CCtx* cctx, ZSTD_cParameter param, unsigned v if (value==0) return 0; DEBUGLOG(5, " setting nbThreads : %u", value); #ifndef ZSTD_MULTITHREAD - if (value > 1) return ERROR(compressionParameter_unsupported); + if (value > 1) return ERROR(parameter_unsupported); #endif if ((value>1) && (cctx->nbThreads != value)) { if (cctx->staticSize) /* MT not compatible with static alloc */ - return ERROR(compressionParameter_unsupported); + return ERROR(parameter_unsupported); ZSTDMT_freeCCtx(cctx->mtctx); cctx->nbThreads = 1; - cctx->mtctx = ZSTDMT_createCCtx(value); + cctx->mtctx = ZSTDMT_createCCtx_advanced(value, cctx->customMem); if (cctx->mtctx == NULL) return ERROR(memory_allocation); } cctx->nbThreads = value; return 0; case ZSTD_p_jobSize: - if (cctx->nbThreads <= 1) return ERROR(compressionParameter_unsupported); + if (cctx->nbThreads <= 1) return ERROR(parameter_unsupported); assert(cctx->mtctx != NULL); return ZSTDMT_setMTCtxParameter(cctx->mtctx, ZSTDMT_p_sectionSize, value); case ZSTD_p_overlapSizeLog: DEBUGLOG(5, " setting overlap with nbThreads == %u", cctx->nbThreads); - if (cctx->nbThreads <= 1) return ERROR(compressionParameter_unsupported); + if (cctx->nbThreads <= 1) return ERROR(parameter_unsupported); assert(cctx->mtctx != NULL); return ZSTDMT_setMTCtxParameter(cctx->mtctx, ZSTDMT_p_overlapSectionLog, value); - default: return ERROR(parameter_unknown); + default: return ERROR(parameter_unsupported); } } @@ -474,7 +450,8 @@ size_t ZSTD_checkCParams(ZSTD_compressionParameters cParams) CLAMPCHECK(cParams.searchLog, ZSTD_SEARCHLOG_MIN, ZSTD_SEARCHLOG_MAX); CLAMPCHECK(cParams.searchLength, ZSTD_SEARCHLENGTH_MIN, ZSTD_SEARCHLENGTH_MAX); CLAMPCHECK(cParams.targetLength, ZSTD_TARGETLENGTH_MIN, ZSTD_TARGETLENGTH_MAX); - if ((U32)(cParams.strategy) > (U32)ZSTD_btultra) return ERROR(compressionParameter_unsupported); + if ((U32)(cParams.strategy) > (U32)ZSTD_btultra) + return ERROR(parameter_unsupported); return 0; } @@ -551,9 +528,7 @@ size_t ZSTD_estimateCCtxSize_advanced(ZSTD_compressionParameters cParams) size_t const hSize = ((size_t)1) << cParams.hashLog; U32 const hashLog3 = (cParams.searchLength>3) ? 0 : MIN(ZSTD_HASHLOG3_MAX, cParams.windowLog); size_t const h3Size = ((size_t)1) << hashLog3; - size_t const entropySpace = hufCTable_size + litlengthCTable_size - + offcodeCTable_size + matchlengthCTable_size - + entropyScratchSpace_size; + size_t const entropySpace = sizeof(ZSTD_entropyCTables_t); size_t const tableSpace = (chainSize + hSize + h3Size) * sizeof(U32); size_t const optBudget = ((MaxML+1) + (MaxLL+1) + (MaxOff+1) + (1<stage = ZSTDcs_init; cctx->dictID = 0; cctx->loadedDictEnd = 0; - { int i; for (i=0; irep[i] = repStartValue[i]; } - cctx->seqStore.litLengthSum = 0; /* force reset of btopt stats */ + { int i; for (i=0; iseqStore.rep[i] = repStartValue[i]; } + cctx->optState.litLengthSum = 0; /* force reset of btopt stats */ XXH64_reset(&cctx->xxhState, 0); return 0; } @@ -641,8 +616,10 @@ static size_t ZSTD_resetCCtx_internal(ZSTD_CCtx* zc, if (crp == ZSTDcrp_continue) { if (ZSTD_equivalentParams(params.cParams, zc->appliedParams.cParams)) { DEBUGLOG(5, "ZSTD_equivalentParams()==1"); - zc->fseCTables_ready = 0; - zc->hufCTable_repeatMode = HUF_repeat_none; + zc->entropy->hufCTable_repeatMode = HUF_repeat_none; + zc->entropy->offcode_repeatMode = FSE_repeat_none; + zc->entropy->matchlength_repeatMode = FSE_repeat_none; + zc->entropy->litlength_repeatMode = FSE_repeat_none; return ZSTD_continueCCtx(zc, params, pledgedSrcSize); } } @@ -662,9 +639,7 @@ static size_t ZSTD_resetCCtx_internal(ZSTD_CCtx* zc, void* ptr; /* Check if workSpace is large enough, alloc a new one if needed */ - { size_t const entropySpace = hufCTable_size + litlengthCTable_size - + offcodeCTable_size + matchlengthCTable_size - + entropyScratchSpace_size; + { size_t const entropySpace = sizeof(ZSTD_entropyCTables_t); size_t const optPotentialSpace = ((MaxML+1) + (MaxLL+1) + (MaxOff+1) + (1<workSpace; /* entropy space */ - zc->hufCTable = (HUF_CElt*)ptr; - ptr = (char*)zc->hufCTable + hufCTable_size; /* note : HUF_CElt* is incomplete type, size is estimated via macro */ - zc->offcodeCTable = (FSE_CTable*) ptr; - ptr = (char*)ptr + offcodeCTable_size; - zc->matchlengthCTable = (FSE_CTable*) ptr; - ptr = (char*)ptr + matchlengthCTable_size; - zc->litlengthCTable = (FSE_CTable*) ptr; - ptr = (char*)ptr + litlengthCTable_size; - assert(((size_t)ptr & 3) == 0); /* ensure correct alignment */ - zc->entropyScratchSpace = (unsigned*) ptr; + assert(((size_t)zc->workSpace & 3) == 0); /* ensure correct alignment */ + assert(zc->workSpaceSize >= sizeof(ZSTD_entropyCTables_t)); + zc->entropy = (ZSTD_entropyCTables_t*)zc->workSpace; } } /* init params */ @@ -715,39 +683,35 @@ static size_t ZSTD_resetCCtx_internal(ZSTD_CCtx* zc, zc->stage = ZSTDcs_init; zc->dictID = 0; zc->loadedDictEnd = 0; - zc->fseCTables_ready = 0; - zc->hufCTable_repeatMode = HUF_repeat_none; + zc->entropy->hufCTable_repeatMode = HUF_repeat_none; + zc->entropy->offcode_repeatMode = FSE_repeat_none; + zc->entropy->matchlength_repeatMode = FSE_repeat_none; + zc->entropy->litlength_repeatMode = FSE_repeat_none; zc->nextToUpdate = 1; zc->nextSrc = NULL; zc->base = NULL; zc->dictBase = NULL; zc->dictLimit = 0; zc->lowLimit = 0; - { int i; for (i=0; irep[i] = repStartValue[i]; } + { int i; for (i=0; iseqStore.rep[i] = repStartValue[i]; } zc->hashLog3 = hashLog3; - zc->seqStore.litLengthSum = 0; + zc->optState.litLengthSum = 0; - /* ensure entropy tables are close together at the beginning */ - assert((void*)zc->hufCTable == zc->workSpace); - assert((char*)zc->offcodeCTable == (char*)zc->hufCTable + hufCTable_size); - assert((char*)zc->matchlengthCTable == (char*)zc->offcodeCTable + offcodeCTable_size); - assert((char*)zc->litlengthCTable == (char*)zc->matchlengthCTable + matchlengthCTable_size); - assert((char*)zc->entropyScratchSpace == (char*)zc->litlengthCTable + litlengthCTable_size); - ptr = (char*)zc->entropyScratchSpace + entropyScratchSpace_size; + ptr = zc->entropy + 1; /* opt parser space */ if ((params.cParams.strategy == ZSTD_btopt) || (params.cParams.strategy == ZSTD_btultra)) { DEBUGLOG(5, "reserving optimal parser space"); assert(((size_t)ptr & 3) == 0); /* ensure ptr is properly aligned */ - zc->seqStore.litFreq = (U32*)ptr; - zc->seqStore.litLengthFreq = zc->seqStore.litFreq + (1<seqStore.matchLengthFreq = zc->seqStore.litLengthFreq + (MaxLL+1); - zc->seqStore.offCodeFreq = zc->seqStore.matchLengthFreq + (MaxML+1); - ptr = zc->seqStore.offCodeFreq + (MaxOff+1); - zc->seqStore.matchTable = (ZSTD_match_t*)ptr; - ptr = zc->seqStore.matchTable + ZSTD_OPT_NUM+1; - zc->seqStore.priceTable = (ZSTD_optimal_t*)ptr; - ptr = zc->seqStore.priceTable + ZSTD_OPT_NUM+1; + zc->optState.litFreq = (U32*)ptr; + zc->optState.litLengthFreq = zc->optState.litFreq + (1<optState.matchLengthFreq = zc->optState.litLengthFreq + (MaxLL+1); + zc->optState.offCodeFreq = zc->optState.matchLengthFreq + (MaxML+1); + ptr = zc->optState.offCodeFreq + (MaxOff+1); + zc->optState.matchTable = (ZSTD_match_t*)ptr; + ptr = zc->optState.matchTable + ZSTD_OPT_NUM+1; + zc->optState.priceTable = (ZSTD_optimal_t*)ptr; + ptr = zc->optState.priceTable + ZSTD_OPT_NUM+1; } /* table Space */ @@ -783,7 +747,7 @@ static size_t ZSTD_resetCCtx_internal(ZSTD_CCtx* zc, * do not use with extDict variant ! */ void ZSTD_invalidateRepCodes(ZSTD_CCtx* cctx) { int i; - for (i=0; irep[i] = 0; + for (i=0; iseqStore.rep[i] = 0; } @@ -830,16 +794,7 @@ static size_t ZSTD_copyCCtx_internal(ZSTD_CCtx* dstCCtx, dstCCtx->dictID = srcCCtx->dictID; /* copy entropy tables */ - dstCCtx->fseCTables_ready = srcCCtx->fseCTables_ready; - if (srcCCtx->fseCTables_ready) { - memcpy(dstCCtx->litlengthCTable, srcCCtx->litlengthCTable, litlengthCTable_size); - memcpy(dstCCtx->matchlengthCTable, srcCCtx->matchlengthCTable, matchlengthCTable_size); - memcpy(dstCCtx->offcodeCTable, srcCCtx->offcodeCTable, offcodeCTable_size); - } - dstCCtx->hufCTable_repeatMode = srcCCtx->hufCTable_repeatMode; - if (srcCCtx->hufCTable_repeatMode) { - memcpy(dstCCtx->hufCTable, srcCCtx->hufCTable, hufCTable_size); - } + memcpy(dstCCtx->entropy, srcCCtx->entropy, sizeof(ZSTD_entropyCTables_t)); return 0; } @@ -956,7 +911,8 @@ static size_t ZSTD_compressRleLiteralsBlock (void* dst, size_t dstCapacity, cons static size_t ZSTD_minGain(size_t srcSize) { return (srcSize >> 6) + 2; } -static size_t ZSTD_compressLiterals (ZSTD_CCtx* zc, +static size_t ZSTD_compressLiterals (ZSTD_entropyCTables_t * entropy, + ZSTD_strategy strategy, void* dst, size_t dstCapacity, const void* src, size_t srcSize) { @@ -970,28 +926,28 @@ static size_t ZSTD_compressLiterals (ZSTD_CCtx* zc, /* small ? don't even attempt compression (speed opt) */ # define LITERAL_NOENTROPY 63 - { size_t const minLitSize = zc->hufCTable_repeatMode == HUF_repeat_valid ? 6 : LITERAL_NOENTROPY; + { size_t const minLitSize = entropy->hufCTable_repeatMode == HUF_repeat_valid ? 6 : LITERAL_NOENTROPY; if (srcSize <= minLitSize) return ZSTD_noCompressLiterals(dst, dstCapacity, src, srcSize); } if (dstCapacity < lhSize+1) return ERROR(dstSize_tooSmall); /* not enough space for compression */ - { HUF_repeat repeat = zc->hufCTable_repeatMode; - int const preferRepeat = zc->appliedParams.cParams.strategy < ZSTD_lazy ? srcSize <= 1024 : 0; + { HUF_repeat repeat = entropy->hufCTable_repeatMode; + int const preferRepeat = strategy < ZSTD_lazy ? srcSize <= 1024 : 0; if (repeat == HUF_repeat_valid && lhSize == 3) singleStream = 1; cLitSize = singleStream ? HUF_compress1X_repeat(ostart+lhSize, dstCapacity-lhSize, src, srcSize, 255, 11, - zc->entropyScratchSpace, entropyScratchSpace_size, zc->hufCTable, &repeat, preferRepeat) + entropy->workspace, sizeof(entropy->workspace), (HUF_CElt*)entropy->hufCTable, &repeat, preferRepeat) : HUF_compress4X_repeat(ostart+lhSize, dstCapacity-lhSize, src, srcSize, 255, 11, - zc->entropyScratchSpace, entropyScratchSpace_size, zc->hufCTable, &repeat, preferRepeat); + entropy->workspace, sizeof(entropy->workspace), (HUF_CElt*)entropy->hufCTable, &repeat, preferRepeat); if (repeat != HUF_repeat_none) { hType = set_repeat; } /* reused the existing table */ - else { zc->hufCTable_repeatMode = HUF_repeat_check; } /* now have a table to reuse */ + else { entropy->hufCTable_repeatMode = HUF_repeat_check; } /* now have a table to reuse */ } - if ((cLitSize==0) | (cLitSize >= srcSize - minGain)) { - zc->hufCTable_repeatMode = HUF_repeat_none; + if ((cLitSize==0) | (cLitSize >= srcSize - minGain) | ERR_isError(cLitSize)) { + entropy->hufCTable_repeatMode = HUF_repeat_none; return ZSTD_noCompressLiterals(dst, dstCapacity, src, srcSize); } if (cLitSize==1) { - zc->hufCTable_repeatMode = HUF_repeat_none; + entropy->hufCTable_repeatMode = HUF_repeat_none; return ZSTD_compressRleLiteralsBlock(dst, dstCapacity, src, srcSize); } @@ -1062,17 +1018,154 @@ void ZSTD_seqToCodes(const seqStore_t* seqStorePtr) mlCodeTable[seqStorePtr->longLengthPos] = MaxML; } -MEM_STATIC size_t ZSTD_compressSequences (ZSTD_CCtx* zc, - void* dst, size_t dstCapacity, - size_t srcSize) +MEM_STATIC symbolEncodingType_e ZSTD_selectEncodingType(FSE_repeat* repeatMode, + size_t const mostFrequent, size_t nbSeq, U32 defaultNormLog) { - const int longOffsets = zc->appliedParams.cParams.windowLog > STREAM_ACCUMULATOR_MIN; - const seqStore_t* seqStorePtr = &(zc->seqStore); +#define MIN_SEQ_FOR_DYNAMIC_FSE 64 +#define MAX_SEQ_FOR_STATIC_FSE 1000 + + if ((mostFrequent == nbSeq) && (nbSeq > 2)) { + *repeatMode = FSE_repeat_check; + return set_rle; + } + if ((*repeatMode == FSE_repeat_valid) && (nbSeq < MAX_SEQ_FOR_STATIC_FSE)) { + return set_repeat; + } + if ((nbSeq < MIN_SEQ_FOR_DYNAMIC_FSE) || (mostFrequent < (nbSeq >> (defaultNormLog-1)))) { + *repeatMode = FSE_repeat_valid; + return set_basic; + } + *repeatMode = FSE_repeat_check; + return set_compressed; +} + +MEM_STATIC size_t ZSTD_buildCTable(void* dst, size_t dstCapacity, + FSE_CTable* CTable, U32 FSELog, symbolEncodingType_e type, + U32* count, U32 max, + BYTE const* codeTable, size_t nbSeq, + S16 const* defaultNorm, U32 defaultNormLog, U32 defaultMax, + void* workspace, size_t workspaceSize) +{ + BYTE* op = (BYTE*)dst; + BYTE const* const oend = op + dstCapacity; + + switch (type) { + case set_rle: + *op = codeTable[0]; + CHECK_F(FSE_buildCTable_rle(CTable, (BYTE)max)); + return 1; + case set_repeat: + return 0; + case set_basic: + CHECK_F(FSE_buildCTable_wksp(CTable, defaultNorm, defaultMax, defaultNormLog, workspace, workspaceSize)); + return 0; + case set_compressed: { + S16 norm[MaxSeq + 1]; + size_t nbSeq_1 = nbSeq; + const U32 tableLog = FSE_optimalTableLog(FSELog, nbSeq, max); + if (count[codeTable[nbSeq-1]] > 1) { + count[codeTable[nbSeq-1]]--; + nbSeq_1--; + } + CHECK_F(FSE_normalizeCount(norm, tableLog, count, nbSeq_1, max)); + { size_t const NCountSize = FSE_writeNCount(op, oend - op, norm, max, tableLog); /* overflow protected */ + if (FSE_isError(NCountSize)) return NCountSize; + CHECK_F(FSE_buildCTable_wksp(CTable, norm, max, tableLog, workspace, workspaceSize)); + return NCountSize; + } + } + default: return assert(0), ERROR(GENERIC); + } +} + +MEM_STATIC size_t ZSTD_encodeSequences(void* dst, size_t dstCapacity, + FSE_CTable const* CTable_MatchLength, BYTE const* mlCodeTable, + FSE_CTable const* CTable_OffsetBits, BYTE const* ofCodeTable, + FSE_CTable const* CTable_LitLength, BYTE const* llCodeTable, + seqDef const* sequences, size_t nbSeq, int longOffsets) +{ + BIT_CStream_t blockStream; + FSE_CState_t stateMatchLength; + FSE_CState_t stateOffsetBits; + FSE_CState_t stateLitLength; + + CHECK_E(BIT_initCStream(&blockStream, dst, dstCapacity), dstSize_tooSmall); /* not enough space remaining */ + + /* first symbols */ + FSE_initCState2(&stateMatchLength, CTable_MatchLength, mlCodeTable[nbSeq-1]); + FSE_initCState2(&stateOffsetBits, CTable_OffsetBits, ofCodeTable[nbSeq-1]); + FSE_initCState2(&stateLitLength, CTable_LitLength, llCodeTable[nbSeq-1]); + BIT_addBits(&blockStream, sequences[nbSeq-1].litLength, LL_bits[llCodeTable[nbSeq-1]]); + if (MEM_32bits()) BIT_flushBits(&blockStream); + BIT_addBits(&blockStream, sequences[nbSeq-1].matchLength, ML_bits[mlCodeTable[nbSeq-1]]); + if (MEM_32bits()) BIT_flushBits(&blockStream); + if (longOffsets) { + U32 const ofBits = ofCodeTable[nbSeq-1]; + int const extraBits = ofBits - MIN(ofBits, STREAM_ACCUMULATOR_MIN-1); + if (extraBits) { + BIT_addBits(&blockStream, sequences[nbSeq-1].offset, extraBits); + BIT_flushBits(&blockStream); + } + BIT_addBits(&blockStream, sequences[nbSeq-1].offset >> extraBits, + ofBits - extraBits); + } else { + BIT_addBits(&blockStream, sequences[nbSeq-1].offset, ofCodeTable[nbSeq-1]); + } + BIT_flushBits(&blockStream); + + { size_t n; + for (n=nbSeq-2 ; n= 64-7-(LLFSELog+MLFSELog+OffFSELog))) + BIT_flushBits(&blockStream); /* (7)*/ + BIT_addBits(&blockStream, sequences[n].litLength, llBits); + if (MEM_32bits() && ((llBits+mlBits)>24)) BIT_flushBits(&blockStream); + BIT_addBits(&blockStream, sequences[n].matchLength, mlBits); + if (MEM_32bits()) BIT_flushBits(&blockStream); /* (7)*/ + if (longOffsets) { + int const extraBits = ofBits - MIN(ofBits, STREAM_ACCUMULATOR_MIN-1); + if (extraBits) { + BIT_addBits(&blockStream, sequences[n].offset, extraBits); + BIT_flushBits(&blockStream); /* (7)*/ + } + BIT_addBits(&blockStream, sequences[n].offset >> extraBits, + ofBits - extraBits); /* 31 */ + } else { + BIT_addBits(&blockStream, sequences[n].offset, ofBits); /* 31 */ + } + BIT_flushBits(&blockStream); /* (7)*/ + } } + + FSE_flushCState(&blockStream, &stateMatchLength); + FSE_flushCState(&blockStream, &stateOffsetBits); + FSE_flushCState(&blockStream, &stateLitLength); + + { size_t const streamSize = BIT_closeCStream(&blockStream); + if (streamSize==0) return ERROR(dstSize_tooSmall); /* not enough space */ + return streamSize; + } +} + +MEM_STATIC size_t ZSTD_compressSequences_internal(seqStore_t* seqStorePtr, + ZSTD_entropyCTables_t* entropy, + ZSTD_compressionParameters const* cParams, + void* dst, size_t dstCapacity) +{ + const int longOffsets = cParams->windowLog > STREAM_ACCUMULATOR_MIN; U32 count[MaxSeq+1]; - S16 norm[MaxSeq+1]; - FSE_CTable* CTable_LitLength = zc->litlengthCTable; - FSE_CTable* CTable_OffsetBits = zc->offcodeCTable; - FSE_CTable* CTable_MatchLength = zc->matchlengthCTable; + FSE_CTable* CTable_LitLength = entropy->litlengthCTable; + FSE_CTable* CTable_OffsetBits = entropy->offcodeCTable; + FSE_CTable* CTable_MatchLength = entropy->matchlengthCTable; U32 LLtype, Offtype, MLtype; /* compressed, raw or rle */ const seqDef* const sequences = seqStorePtr->sequencesStart; const BYTE* const ofCodeTable = seqStorePtr->ofCode; @@ -1083,13 +1176,16 @@ MEM_STATIC size_t ZSTD_compressSequences (ZSTD_CCtx* zc, BYTE* op = ostart; size_t const nbSeq = seqStorePtr->sequences - seqStorePtr->sequencesStart; BYTE* seqHead; - BYTE scratchBuffer[1<workspace) >= (1<litStart; size_t const litSize = seqStorePtr->lit - literals; - size_t const cSize = ZSTD_compressLiterals(zc, op, dstCapacity, literals, litSize); - if (ZSTD_isError(cSize)) return cSize; + size_t const cSize = ZSTD_compressLiterals( + entropy, cParams->strategy, op, dstCapacity, literals, litSize); + if (ZSTD_isError(cSize)) + return cSize; op += cSize; } @@ -1098,179 +1194,91 @@ MEM_STATIC size_t ZSTD_compressSequences (ZSTD_CCtx* zc, if (nbSeq < 0x7F) *op++ = (BYTE)nbSeq; else if (nbSeq < LONGNBSEQ) op[0] = (BYTE)((nbSeq>>8) + 0x80), op[1] = (BYTE)nbSeq, op+=2; else op[0]=0xFF, MEM_writeLE16(op+1, (U16)(nbSeq - LONGNBSEQ)), op+=3; - if (nbSeq==0) goto _check_compressibility; + if (nbSeq==0) return op - ostart; /* seqHead : flags for FSE encoding type */ seqHead = op++; -#define MIN_SEQ_FOR_DYNAMIC_FSE 64 -#define MAX_SEQ_FOR_STATIC_FSE 1000 - /* convert length/distances into codes */ ZSTD_seqToCodes(seqStorePtr); - /* CTable for Literal Lengths */ { U32 max = MaxLL; - size_t const mostFrequent = FSE_countFast_wksp(count, &max, llCodeTable, nbSeq, zc->entropyScratchSpace); - if ((mostFrequent == nbSeq) && (nbSeq > 2)) { - *op++ = llCodeTable[0]; - FSE_buildCTable_rle(CTable_LitLength, (BYTE)max); - LLtype = set_rle; - } else if ((zc->fseCTables_ready) && (nbSeq < MAX_SEQ_FOR_STATIC_FSE)) { - LLtype = set_repeat; - } else if ((nbSeq < MIN_SEQ_FOR_DYNAMIC_FSE) || (mostFrequent < (nbSeq >> (LL_defaultNormLog-1)))) { - FSE_buildCTable_wksp(CTable_LitLength, LL_defaultNorm, MaxLL, LL_defaultNormLog, scratchBuffer, sizeof(scratchBuffer)); - LLtype = set_basic; - } else { - size_t nbSeq_1 = nbSeq; - const U32 tableLog = FSE_optimalTableLog(LLFSELog, nbSeq, max); - if (count[llCodeTable[nbSeq-1]]>1) { count[llCodeTable[nbSeq-1]]--; nbSeq_1--; } - FSE_normalizeCount(norm, tableLog, count, nbSeq_1, max); - { size_t const NCountSize = FSE_writeNCount(op, oend-op, norm, max, tableLog); /* overflow protected */ - if (FSE_isError(NCountSize)) return NCountSize; - op += NCountSize; } - FSE_buildCTable_wksp(CTable_LitLength, norm, max, tableLog, scratchBuffer, sizeof(scratchBuffer)); - LLtype = set_compressed; + size_t const mostFrequent = FSE_countFast_wksp(count, &max, llCodeTable, nbSeq, entropy->workspace); + LLtype = ZSTD_selectEncodingType(&entropy->litlength_repeatMode, mostFrequent, nbSeq, LL_defaultNormLog); + { size_t const countSize = ZSTD_buildCTable(op, oend - op, CTable_LitLength, LLFSELog, (symbolEncodingType_e)LLtype, + count, max, llCodeTable, nbSeq, LL_defaultNorm, LL_defaultNormLog, MaxLL, + entropy->workspace, sizeof(entropy->workspace)); + if (ZSTD_isError(countSize)) return countSize; + op += countSize; } } - /* CTable for Offsets */ { U32 max = MaxOff; - size_t const mostFrequent = FSE_countFast_wksp(count, &max, ofCodeTable, nbSeq, zc->entropyScratchSpace); - if ((mostFrequent == nbSeq) && (nbSeq > 2)) { - *op++ = ofCodeTable[0]; - FSE_buildCTable_rle(CTable_OffsetBits, (BYTE)max); - Offtype = set_rle; - } else if ((zc->fseCTables_ready) && (nbSeq < MAX_SEQ_FOR_STATIC_FSE)) { - Offtype = set_repeat; - } else if ((nbSeq < MIN_SEQ_FOR_DYNAMIC_FSE) || (mostFrequent < (nbSeq >> (OF_defaultNormLog-1)))) { - FSE_buildCTable_wksp(CTable_OffsetBits, OF_defaultNorm, MaxOff, OF_defaultNormLog, scratchBuffer, sizeof(scratchBuffer)); - Offtype = set_basic; - } else { - size_t nbSeq_1 = nbSeq; - const U32 tableLog = FSE_optimalTableLog(OffFSELog, nbSeq, max); - if (count[ofCodeTable[nbSeq-1]]>1) { count[ofCodeTable[nbSeq-1]]--; nbSeq_1--; } - FSE_normalizeCount(norm, tableLog, count, nbSeq_1, max); - { size_t const NCountSize = FSE_writeNCount(op, oend-op, norm, max, tableLog); /* overflow protected */ - if (FSE_isError(NCountSize)) return NCountSize; - op += NCountSize; } - FSE_buildCTable_wksp(CTable_OffsetBits, norm, max, tableLog, scratchBuffer, sizeof(scratchBuffer)); - Offtype = set_compressed; + size_t const mostFrequent = FSE_countFast_wksp(count, &max, ofCodeTable, nbSeq, entropy->workspace); + Offtype = ZSTD_selectEncodingType(&entropy->offcode_repeatMode, mostFrequent, nbSeq, OF_defaultNormLog); + { size_t const countSize = ZSTD_buildCTable(op, oend - op, CTable_OffsetBits, OffFSELog, (symbolEncodingType_e)Offtype, + count, max, ofCodeTable, nbSeq, OF_defaultNorm, OF_defaultNormLog, MaxOff, + entropy->workspace, sizeof(entropy->workspace)); + if (ZSTD_isError(countSize)) return countSize; + op += countSize; } } - /* CTable for MatchLengths */ { U32 max = MaxML; - size_t const mostFrequent = FSE_countFast_wksp(count, &max, mlCodeTable, nbSeq, zc->entropyScratchSpace); - if ((mostFrequent == nbSeq) && (nbSeq > 2)) { - *op++ = *mlCodeTable; - FSE_buildCTable_rle(CTable_MatchLength, (BYTE)max); - MLtype = set_rle; - } else if ((zc->fseCTables_ready) && (nbSeq < MAX_SEQ_FOR_STATIC_FSE)) { - MLtype = set_repeat; - } else if ((nbSeq < MIN_SEQ_FOR_DYNAMIC_FSE) || (mostFrequent < (nbSeq >> (ML_defaultNormLog-1)))) { - FSE_buildCTable_wksp(CTable_MatchLength, ML_defaultNorm, MaxML, ML_defaultNormLog, scratchBuffer, sizeof(scratchBuffer)); - MLtype = set_basic; - } else { - size_t nbSeq_1 = nbSeq; - const U32 tableLog = FSE_optimalTableLog(MLFSELog, nbSeq, max); - if (count[mlCodeTable[nbSeq-1]]>1) { count[mlCodeTable[nbSeq-1]]--; nbSeq_1--; } - FSE_normalizeCount(norm, tableLog, count, nbSeq_1, max); - { size_t const NCountSize = FSE_writeNCount(op, oend-op, norm, max, tableLog); /* overflow protected */ - if (FSE_isError(NCountSize)) return NCountSize; - op += NCountSize; } - FSE_buildCTable_wksp(CTable_MatchLength, norm, max, tableLog, scratchBuffer, sizeof(scratchBuffer)); - MLtype = set_compressed; + size_t const mostFrequent = FSE_countFast_wksp(count, &max, mlCodeTable, nbSeq, entropy->workspace); + MLtype = ZSTD_selectEncodingType(&entropy->matchlength_repeatMode, mostFrequent, nbSeq, ML_defaultNormLog); + { size_t const countSize = ZSTD_buildCTable(op, oend - op, CTable_MatchLength, MLFSELog, (symbolEncodingType_e)MLtype, + count, max, mlCodeTable, nbSeq, ML_defaultNorm, ML_defaultNormLog, MaxML, + entropy->workspace, sizeof(entropy->workspace)); + if (ZSTD_isError(countSize)) return countSize; + op += countSize; } } *seqHead = (BYTE)((LLtype<<6) + (Offtype<<4) + (MLtype<<2)); - zc->fseCTables_ready = 0; - /* Encoding Sequences */ - { BIT_CStream_t blockStream; - FSE_CState_t stateMatchLength; - FSE_CState_t stateOffsetBits; - FSE_CState_t stateLitLength; - - CHECK_E(BIT_initCStream(&blockStream, op, oend-op), dstSize_tooSmall); /* not enough space remaining */ - - /* first symbols */ - FSE_initCState2(&stateMatchLength, CTable_MatchLength, mlCodeTable[nbSeq-1]); - FSE_initCState2(&stateOffsetBits, CTable_OffsetBits, ofCodeTable[nbSeq-1]); - FSE_initCState2(&stateLitLength, CTable_LitLength, llCodeTable[nbSeq-1]); - BIT_addBits(&blockStream, sequences[nbSeq-1].litLength, LL_bits[llCodeTable[nbSeq-1]]); - if (MEM_32bits()) BIT_flushBits(&blockStream); - BIT_addBits(&blockStream, sequences[nbSeq-1].matchLength, ML_bits[mlCodeTable[nbSeq-1]]); - if (MEM_32bits()) BIT_flushBits(&blockStream); - if (longOffsets) { - U32 const ofBits = ofCodeTable[nbSeq-1]; - int const extraBits = ofBits - MIN(ofBits, STREAM_ACCUMULATOR_MIN-1); - if (extraBits) { - BIT_addBits(&blockStream, sequences[nbSeq-1].offset, extraBits); - BIT_flushBits(&blockStream); - } - BIT_addBits(&blockStream, sequences[nbSeq-1].offset >> extraBits, - ofBits - extraBits); - } else { - BIT_addBits(&blockStream, sequences[nbSeq-1].offset, ofCodeTable[nbSeq-1]); - } - BIT_flushBits(&blockStream); - - { size_t n; - for (n=nbSeq-2 ; n= 64-7-(LLFSELog+MLFSELog+OffFSELog))) - BIT_flushBits(&blockStream); /* (7)*/ - BIT_addBits(&blockStream, sequences[n].litLength, llBits); - if (MEM_32bits() && ((llBits+mlBits)>24)) BIT_flushBits(&blockStream); - BIT_addBits(&blockStream, sequences[n].matchLength, mlBits); - if (MEM_32bits()) BIT_flushBits(&blockStream); /* (7)*/ - if (longOffsets) { - int const extraBits = ofBits - MIN(ofBits, STREAM_ACCUMULATOR_MIN-1); - if (extraBits) { - BIT_addBits(&blockStream, sequences[n].offset, extraBits); - BIT_flushBits(&blockStream); /* (7)*/ - } - BIT_addBits(&blockStream, sequences[n].offset >> extraBits, - ofBits - extraBits); /* 31 */ - } else { - BIT_addBits(&blockStream, sequences[n].offset, ofBits); /* 31 */ - } - BIT_flushBits(&blockStream); /* (7)*/ - } } - - FSE_flushCState(&blockStream, &stateMatchLength); - FSE_flushCState(&blockStream, &stateOffsetBits); - FSE_flushCState(&blockStream, &stateLitLength); - - { size_t const streamSize = BIT_closeCStream(&blockStream); - if (streamSize==0) return ERROR(dstSize_tooSmall); /* not enough space */ - op += streamSize; - } } - - /* check compressibility */ -_check_compressibility: - { size_t const minGain = ZSTD_minGain(srcSize); - size_t const maxCSize = srcSize - minGain; - if ((size_t)(op-ostart) >= maxCSize) { - zc->hufCTable_repeatMode = HUF_repeat_none; - return 0; - } } - - /* confirm repcodes */ - { int i; for (i=0; irep[i] = zc->repToConfirm[i]; } + { size_t const streamSize = ZSTD_encodeSequences(op, oend - op, + CTable_MatchLength, mlCodeTable, + CTable_OffsetBits, ofCodeTable, + CTable_LitLength, llCodeTable, + sequences, nbSeq, longOffsets); + if (ZSTD_isError(streamSize)) return streamSize; + op += streamSize; + } return op - ostart; } +MEM_STATIC size_t ZSTD_compressSequences(seqStore_t* seqStorePtr, + ZSTD_entropyCTables_t* entropy, + ZSTD_compressionParameters const* cParams, + void* dst, size_t dstCapacity, + size_t srcSize) +{ + size_t const cSize = ZSTD_compressSequences_internal(seqStorePtr, entropy, cParams, + dst, dstCapacity); + size_t const minGain = ZSTD_minGain(srcSize); + size_t const maxCSize = srcSize - minGain; + /* If the srcSize <= dstCapacity, then there is enough space to write a + * raw uncompressed block. Since we ran out of space, the block must not + * be compressible, so fall back to a raw uncompressed block. + */ + int const uncompressibleError = cSize == ERROR(dstSize_tooSmall) && srcSize <= dstCapacity; + + if (ZSTD_isError(cSize) && !uncompressibleError) + return cSize; + /* Check compressibility */ + if (cSize >= maxCSize || uncompressibleError) { + entropy->hufCTable_repeatMode = HUF_repeat_none; + entropy->offcode_repeatMode = FSE_repeat_none; + entropy->matchlength_repeatMode = FSE_repeat_none; + entropy->litlength_repeatMode = FSE_repeat_none; + return 0; + } + assert(!ZSTD_isError(cSize)); + + /* confirm repcodes */ + { int i; for (i=0; irep[i] = seqStorePtr->repToConfirm[i]; } + return cSize; +} + /*! ZSTD_storeSeq() : Store a sequence (literal length, literals, offset code and match length code) into seqStore_t. @@ -1475,7 +1483,7 @@ static void ZSTD_fillHashTable (ZSTD_CCtx* zc, const void* end, const U32 mls) } -FORCE_INLINE +FORCE_INLINE_TEMPLATE void ZSTD_compressBlock_fast_generic(ZSTD_CCtx* cctx, const void* src, size_t srcSize, const U32 mls) @@ -1491,7 +1499,7 @@ void ZSTD_compressBlock_fast_generic(ZSTD_CCtx* cctx, const BYTE* const lowest = base + lowestIndex; const BYTE* const iend = istart + srcSize; const BYTE* const ilimit = iend - HASH_READ_SIZE; - U32 offset_1=cctx->rep[0], offset_2=cctx->rep[1]; + U32 offset_1=seqStorePtr->rep[0], offset_2=seqStorePtr->rep[1]; U32 offsetSaved = 0; /* init */ @@ -1552,8 +1560,8 @@ void ZSTD_compressBlock_fast_generic(ZSTD_CCtx* cctx, } } } /* save reps for next block */ - cctx->repToConfirm[0] = offset_1 ? offset_1 : offsetSaved; - cctx->repToConfirm[1] = offset_2 ? offset_2 : offsetSaved; + seqStorePtr->repToConfirm[0] = offset_1 ? offset_1 : offsetSaved; + seqStorePtr->repToConfirm[1] = offset_2 ? offset_2 : offsetSaved; /* Last Literals */ { size_t const lastLLSize = iend - anchor; @@ -1601,7 +1609,7 @@ static void ZSTD_compressBlock_fast_extDict_generic(ZSTD_CCtx* ctx, const BYTE* const dictEnd = dictBase + dictLimit; const BYTE* const iend = istart + srcSize; const BYTE* const ilimit = iend - 8; - U32 offset_1=ctx->rep[0], offset_2=ctx->rep[1]; + U32 offset_1=seqStorePtr->rep[0], offset_2=seqStorePtr->rep[1]; /* Search Loop */ while (ip < ilimit) { /* < instead of <=, because (ip+1) */ @@ -1667,7 +1675,7 @@ static void ZSTD_compressBlock_fast_extDict_generic(ZSTD_CCtx* ctx, } } } /* save reps for next block */ - ctx->repToConfirm[0] = offset_1; ctx->repToConfirm[1] = offset_2; + seqStorePtr->repToConfirm[0] = offset_1; seqStorePtr->repToConfirm[1] = offset_2; /* Last Literals */ { size_t const lastLLSize = iend - anchor; @@ -1718,7 +1726,7 @@ static void ZSTD_fillDoubleHashTable (ZSTD_CCtx* cctx, const void* end, const U3 } -FORCE_INLINE +FORCE_INLINE_TEMPLATE void ZSTD_compressBlock_doubleFast_generic(ZSTD_CCtx* cctx, const void* src, size_t srcSize, const U32 mls) @@ -1736,7 +1744,7 @@ void ZSTD_compressBlock_doubleFast_generic(ZSTD_CCtx* cctx, const BYTE* const lowest = base + lowestIndex; const BYTE* const iend = istart + srcSize; const BYTE* const ilimit = iend - HASH_READ_SIZE; - U32 offset_1=cctx->rep[0], offset_2=cctx->rep[1]; + U32 offset_1=seqStorePtr->rep[0], offset_2=seqStorePtr->rep[1]; U32 offsetSaved = 0; /* init */ @@ -1823,8 +1831,8 @@ void ZSTD_compressBlock_doubleFast_generic(ZSTD_CCtx* cctx, } } } /* save reps for next block */ - cctx->repToConfirm[0] = offset_1 ? offset_1 : offsetSaved; - cctx->repToConfirm[1] = offset_2 ? offset_2 : offsetSaved; + seqStorePtr->repToConfirm[0] = offset_1 ? offset_1 : offsetSaved; + seqStorePtr->repToConfirm[1] = offset_2 ? offset_2 : offsetSaved; /* Last Literals */ { size_t const lastLLSize = iend - anchor; @@ -1873,7 +1881,7 @@ static void ZSTD_compressBlock_doubleFast_extDict_generic(ZSTD_CCtx* ctx, const BYTE* const dictEnd = dictBase + dictLimit; const BYTE* const iend = istart + srcSize; const BYTE* const ilimit = iend - 8; - U32 offset_1=ctx->rep[0], offset_2=ctx->rep[1]; + U32 offset_1=seqStorePtr->rep[0], offset_2=seqStorePtr->rep[1]; /* Search Loop */ while (ip < ilimit) { /* < instead of <=, because (ip+1) */ @@ -1973,7 +1981,7 @@ static void ZSTD_compressBlock_doubleFast_extDict_generic(ZSTD_CCtx* ctx, } } } /* save reps for next block */ - ctx->repToConfirm[0] = offset_1; ctx->repToConfirm[1] = offset_2; + seqStorePtr->repToConfirm[0] = offset_1; seqStorePtr->repToConfirm[1] = offset_2; /* Last Literals */ { size_t const lastLLSize = iend - anchor; @@ -2276,7 +2284,7 @@ static size_t ZSTD_BtFindBestMatch_selectMLS_extDict ( /* Update chains up to ip (excluded) Assumption : always within prefix (i.e. not within extDict) */ -FORCE_INLINE +FORCE_INLINE_TEMPLATE U32 ZSTD_insertAndFindFirstIndex (ZSTD_CCtx* zc, const BYTE* ip, U32 mls) { U32* const hashTable = zc->hashTable; @@ -2300,7 +2308,7 @@ U32 ZSTD_insertAndFindFirstIndex (ZSTD_CCtx* zc, const BYTE* ip, U32 mls) /* inlining is important to hardwire a hot branch (template emulation) */ -FORCE_INLINE +FORCE_INLINE_TEMPLATE size_t ZSTD_HcFindBestMatch_generic ( ZSTD_CCtx* zc, /* Index table will be updated */ const BYTE* const ip, const BYTE* const iLimit, @@ -2352,7 +2360,7 @@ size_t ZSTD_HcFindBestMatch_generic ( } -FORCE_INLINE size_t ZSTD_HcFindBestMatch_selectMLS ( +FORCE_INLINE_TEMPLATE size_t ZSTD_HcFindBestMatch_selectMLS ( ZSTD_CCtx* zc, const BYTE* ip, const BYTE* const iLimit, size_t* offsetPtr, @@ -2369,7 +2377,7 @@ FORCE_INLINE size_t ZSTD_HcFindBestMatch_selectMLS ( } -FORCE_INLINE size_t ZSTD_HcFindBestMatch_extDict_selectMLS ( +FORCE_INLINE_TEMPLATE size_t ZSTD_HcFindBestMatch_extDict_selectMLS ( ZSTD_CCtx* zc, const BYTE* ip, const BYTE* const iLimit, size_t* offsetPtr, @@ -2389,7 +2397,7 @@ FORCE_INLINE size_t ZSTD_HcFindBestMatch_extDict_selectMLS ( /* ******************************* * Common parser - lazy strategy *********************************/ -FORCE_INLINE +FORCE_INLINE_TEMPLATE void ZSTD_compressBlock_lazy_generic(ZSTD_CCtx* ctx, const void* src, size_t srcSize, const U32 searchMethod, const U32 depth) @@ -2409,7 +2417,7 @@ void ZSTD_compressBlock_lazy_generic(ZSTD_CCtx* ctx, size_t* offsetPtr, U32 maxNbAttempts, U32 matchLengthSearch); searchMax_f const searchMax = searchMethod ? ZSTD_BtFindBestMatch_selectMLS : ZSTD_HcFindBestMatch_selectMLS; - U32 offset_1 = ctx->rep[0], offset_2 = ctx->rep[1], savedOffset=0; + U32 offset_1 = seqStorePtr->rep[0], offset_2 = seqStorePtr->rep[1], savedOffset=0; /* init */ ip += (ip==base); @@ -2519,8 +2527,8 @@ _storeSequence: } } /* Save reps for next block */ - ctx->repToConfirm[0] = offset_1 ? offset_1 : savedOffset; - ctx->repToConfirm[1] = offset_2 ? offset_2 : savedOffset; + seqStorePtr->repToConfirm[0] = offset_1 ? offset_1 : savedOffset; + seqStorePtr->repToConfirm[1] = offset_2 ? offset_2 : savedOffset; /* Last Literals */ { size_t const lastLLSize = iend - anchor; @@ -2551,7 +2559,7 @@ static void ZSTD_compressBlock_greedy(ZSTD_CCtx* ctx, const void* src, size_t sr } -FORCE_INLINE +FORCE_INLINE_TEMPLATE void ZSTD_compressBlock_lazy_extDict_generic(ZSTD_CCtx* ctx, const void* src, size_t srcSize, const U32 searchMethod, const U32 depth) @@ -2578,7 +2586,7 @@ void ZSTD_compressBlock_lazy_extDict_generic(ZSTD_CCtx* ctx, U32 maxNbAttempts, U32 matchLengthSearch); searchMax_f searchMax = searchMethod ? ZSTD_BtFindBestMatch_selectMLS_extDict : ZSTD_HcFindBestMatch_extDict_selectMLS; - U32 offset_1 = ctx->rep[0], offset_2 = ctx->rep[1]; + U32 offset_1 = seqStorePtr->rep[0], offset_2 = seqStorePtr->rep[1]; /* init */ ctx->nextToUpdate3 = ctx->nextToUpdate; @@ -2714,7 +2722,7 @@ _storeSequence: } } /* Save reps for next block */ - ctx->repToConfirm[0] = offset_1; ctx->repToConfirm[1] = offset_2; + seqStorePtr->repToConfirm[0] = offset_1; seqStorePtr->repToConfirm[1] = offset_2; /* Last Literals */ { size_t const lastLLSize = iend - anchor; @@ -2823,7 +2831,7 @@ static size_t ZSTD_compressBlock_internal(ZSTD_CCtx* zc, void* dst, size_t dstCa if (current > zc->nextToUpdate + 384) zc->nextToUpdate = current - MIN(192, (U32)(current - zc->nextToUpdate - 384)); /* limited update after finding a very long match */ blockCompressor(zc, src, srcSize); - return ZSTD_compressSequences(zc, dst, dstCapacity, srcSize); + return ZSTD_compressSequences(&zc->seqStore, zc->entropy, &zc->appliedParams.cParams, dst, dstCapacity, srcSize); } @@ -3000,7 +3008,6 @@ static size_t ZSTD_compressContinue_internal (ZSTD_CCtx* cctx, return fhSize; } - size_t ZSTD_compressContinue (ZSTD_CCtx* cctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize) @@ -3106,13 +3113,14 @@ static size_t ZSTD_loadZstdDictionary(ZSTD_CCtx* cctx, const void* dict, size_t const BYTE* const dictEnd = dictPtr + dictSize; short offcodeNCount[MaxOff+1]; unsigned offcodeMaxValue = MaxOff; - BYTE scratchBuffer[1<entropy->workspace) >= (1<dictID = cctx->appliedParams.fParams.noDictIDFlag ? 0 : MEM_readLE32(dictPtr); dictPtr += 4; - { size_t const hufHeaderSize = HUF_readCTable(cctx->hufCTable, 255, dictPtr, dictEnd-dictPtr); + { size_t const hufHeaderSize = HUF_readCTable((HUF_CElt*)cctx->entropy->hufCTable, 255, dictPtr, dictEnd-dictPtr); if (HUF_isError(hufHeaderSize)) return ERROR(dictionary_corrupted); dictPtr += hufHeaderSize; } @@ -3122,7 +3130,7 @@ static size_t ZSTD_loadZstdDictionary(ZSTD_CCtx* cctx, const void* dict, size_t if (FSE_isError(offcodeHeaderSize)) return ERROR(dictionary_corrupted); if (offcodeLog > OffFSELog) return ERROR(dictionary_corrupted); /* Defer checking offcodeMaxValue because we need to know the size of the dictionary content */ - CHECK_E( FSE_buildCTable_wksp(cctx->offcodeCTable, offcodeNCount, offcodeMaxValue, offcodeLog, scratchBuffer, sizeof(scratchBuffer)), + CHECK_E( FSE_buildCTable_wksp(cctx->entropy->offcodeCTable, offcodeNCount, offcodeMaxValue, offcodeLog, cctx->entropy->workspace, sizeof(cctx->entropy->workspace)), dictionary_corrupted); dictPtr += offcodeHeaderSize; } @@ -3134,7 +3142,7 @@ static size_t ZSTD_loadZstdDictionary(ZSTD_CCtx* cctx, const void* dict, size_t if (matchlengthLog > MLFSELog) return ERROR(dictionary_corrupted); /* Every match length code must have non-zero probability */ CHECK_F( ZSTD_checkDictNCount(matchlengthNCount, matchlengthMaxValue, MaxML)); - CHECK_E( FSE_buildCTable_wksp(cctx->matchlengthCTable, matchlengthNCount, matchlengthMaxValue, matchlengthLog, scratchBuffer, sizeof(scratchBuffer)), + CHECK_E( FSE_buildCTable_wksp(cctx->entropy->matchlengthCTable, matchlengthNCount, matchlengthMaxValue, matchlengthLog, cctx->entropy->workspace, sizeof(cctx->entropy->workspace)), dictionary_corrupted); dictPtr += matchlengthHeaderSize; } @@ -3146,15 +3154,15 @@ static size_t ZSTD_loadZstdDictionary(ZSTD_CCtx* cctx, const void* dict, size_t if (litlengthLog > LLFSELog) return ERROR(dictionary_corrupted); /* Every literal length code must have non-zero probability */ CHECK_F( ZSTD_checkDictNCount(litlengthNCount, litlengthMaxValue, MaxLL)); - CHECK_E( FSE_buildCTable_wksp(cctx->litlengthCTable, litlengthNCount, litlengthMaxValue, litlengthLog, scratchBuffer, sizeof(scratchBuffer)), + CHECK_E( FSE_buildCTable_wksp(cctx->entropy->litlengthCTable, litlengthNCount, litlengthMaxValue, litlengthLog, cctx->entropy->workspace, sizeof(cctx->entropy->workspace)), dictionary_corrupted); dictPtr += litlengthHeaderSize; } if (dictPtr+12 > dictEnd) return ERROR(dictionary_corrupted); - cctx->rep[0] = MEM_readLE32(dictPtr+0); - cctx->rep[1] = MEM_readLE32(dictPtr+4); - cctx->rep[2] = MEM_readLE32(dictPtr+8); + cctx->seqStore.rep[0] = MEM_readLE32(dictPtr+0); + cctx->seqStore.rep[1] = MEM_readLE32(dictPtr+4); + cctx->seqStore.rep[2] = MEM_readLE32(dictPtr+8); dictPtr += 12; { size_t const dictContentSize = (size_t)(dictEnd - dictPtr); @@ -3168,12 +3176,14 @@ static size_t ZSTD_loadZstdDictionary(ZSTD_CCtx* cctx, const void* dict, size_t /* All repCodes must be <= dictContentSize and != 0*/ { U32 u; for (u=0; u<3; u++) { - if (cctx->rep[u] == 0) return ERROR(dictionary_corrupted); - if (cctx->rep[u] > dictContentSize) return ERROR(dictionary_corrupted); + if (cctx->seqStore.rep[u] == 0) return ERROR(dictionary_corrupted); + if (cctx->seqStore.rep[u] > dictContentSize) return ERROR(dictionary_corrupted); } } - cctx->fseCTables_ready = 1; - cctx->hufCTable_repeatMode = HUF_repeat_valid; + cctx->entropy->hufCTable_repeatMode = HUF_repeat_valid; + cctx->entropy->offcode_repeatMode = FSE_repeat_valid; + cctx->entropy->matchlength_repeatMode = FSE_repeat_valid; + cctx->entropy->litlength_repeatMode = FSE_repeat_valid; return ZSTD_loadDictionaryContent(cctx, dictPtr, dictContentSize); } } diff --git a/contrib/libzstd/include/zstd/compress/zstd_opt.h b/contrib/libzstd/include/zstd/compress/zstd_opt.h index e8e98915ea3..ae24732c7d8 100644 --- a/contrib/libzstd/include/zstd/compress/zstd_opt.h +++ b/contrib/libzstd/include/zstd/compress/zstd_opt.h @@ -1,10 +1,10 @@ -/** - * Copyright (c) 2016-present, Przemyslaw Skibinski, Yann Collet, Facebook, Inc. +/* + * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -22,173 +22,173 @@ /*-************************************* * Price functions for optimal parser ***************************************/ -FORCE_INLINE void ZSTD_setLog2Prices(seqStore_t* ssPtr) +static void ZSTD_setLog2Prices(optState_t* optPtr) { - ssPtr->log2matchLengthSum = ZSTD_highbit32(ssPtr->matchLengthSum+1); - ssPtr->log2litLengthSum = ZSTD_highbit32(ssPtr->litLengthSum+1); - ssPtr->log2litSum = ZSTD_highbit32(ssPtr->litSum+1); - ssPtr->log2offCodeSum = ZSTD_highbit32(ssPtr->offCodeSum+1); - ssPtr->factor = 1 + ((ssPtr->litSum>>5) / ssPtr->litLengthSum) + ((ssPtr->litSum<<1) / (ssPtr->litSum + ssPtr->matchSum)); + optPtr->log2matchLengthSum = ZSTD_highbit32(optPtr->matchLengthSum+1); + optPtr->log2litLengthSum = ZSTD_highbit32(optPtr->litLengthSum+1); + optPtr->log2litSum = ZSTD_highbit32(optPtr->litSum+1); + optPtr->log2offCodeSum = ZSTD_highbit32(optPtr->offCodeSum+1); + optPtr->factor = 1 + ((optPtr->litSum>>5) / optPtr->litLengthSum) + ((optPtr->litSum<<1) / (optPtr->litSum + optPtr->matchSum)); } -MEM_STATIC void ZSTD_rescaleFreqs(seqStore_t* ssPtr, const BYTE* src, size_t srcSize) +static void ZSTD_rescaleFreqs(optState_t* optPtr, const BYTE* src, size_t srcSize) { unsigned u; - ssPtr->cachedLiterals = NULL; - ssPtr->cachedPrice = ssPtr->cachedLitLength = 0; - ssPtr->staticPrices = 0; + optPtr->cachedLiterals = NULL; + optPtr->cachedPrice = optPtr->cachedLitLength = 0; + optPtr->staticPrices = 0; - if (ssPtr->litLengthSum == 0) { - if (srcSize <= 1024) ssPtr->staticPrices = 1; + if (optPtr->litLengthSum == 0) { + if (srcSize <= 1024) optPtr->staticPrices = 1; - assert(ssPtr->litFreq!=NULL); + assert(optPtr->litFreq!=NULL); for (u=0; u<=MaxLit; u++) - ssPtr->litFreq[u] = 0; + optPtr->litFreq[u] = 0; for (u=0; ulitFreq[src[u]]++; + optPtr->litFreq[src[u]]++; - ssPtr->litSum = 0; - ssPtr->litLengthSum = MaxLL+1; - ssPtr->matchLengthSum = MaxML+1; - ssPtr->offCodeSum = (MaxOff+1); - ssPtr->matchSum = (ZSTD_LITFREQ_ADD<litSum = 0; + optPtr->litLengthSum = MaxLL+1; + optPtr->matchLengthSum = MaxML+1; + optPtr->offCodeSum = (MaxOff+1); + optPtr->matchSum = (ZSTD_LITFREQ_ADD<litFreq[u] = 1 + (ssPtr->litFreq[u]>>ZSTD_FREQ_DIV); - ssPtr->litSum += ssPtr->litFreq[u]; + optPtr->litFreq[u] = 1 + (optPtr->litFreq[u]>>ZSTD_FREQ_DIV); + optPtr->litSum += optPtr->litFreq[u]; } for (u=0; u<=MaxLL; u++) - ssPtr->litLengthFreq[u] = 1; + optPtr->litLengthFreq[u] = 1; for (u=0; u<=MaxML; u++) - ssPtr->matchLengthFreq[u] = 1; + optPtr->matchLengthFreq[u] = 1; for (u=0; u<=MaxOff; u++) - ssPtr->offCodeFreq[u] = 1; + optPtr->offCodeFreq[u] = 1; } else { - ssPtr->matchLengthSum = 0; - ssPtr->litLengthSum = 0; - ssPtr->offCodeSum = 0; - ssPtr->matchSum = 0; - ssPtr->litSum = 0; + optPtr->matchLengthSum = 0; + optPtr->litLengthSum = 0; + optPtr->offCodeSum = 0; + optPtr->matchSum = 0; + optPtr->litSum = 0; for (u=0; u<=MaxLit; u++) { - ssPtr->litFreq[u] = 1 + (ssPtr->litFreq[u]>>(ZSTD_FREQ_DIV+1)); - ssPtr->litSum += ssPtr->litFreq[u]; + optPtr->litFreq[u] = 1 + (optPtr->litFreq[u]>>(ZSTD_FREQ_DIV+1)); + optPtr->litSum += optPtr->litFreq[u]; } for (u=0; u<=MaxLL; u++) { - ssPtr->litLengthFreq[u] = 1 + (ssPtr->litLengthFreq[u]>>(ZSTD_FREQ_DIV+1)); - ssPtr->litLengthSum += ssPtr->litLengthFreq[u]; + optPtr->litLengthFreq[u] = 1 + (optPtr->litLengthFreq[u]>>(ZSTD_FREQ_DIV+1)); + optPtr->litLengthSum += optPtr->litLengthFreq[u]; } for (u=0; u<=MaxML; u++) { - ssPtr->matchLengthFreq[u] = 1 + (ssPtr->matchLengthFreq[u]>>ZSTD_FREQ_DIV); - ssPtr->matchLengthSum += ssPtr->matchLengthFreq[u]; - ssPtr->matchSum += ssPtr->matchLengthFreq[u] * (u + 3); + optPtr->matchLengthFreq[u] = 1 + (optPtr->matchLengthFreq[u]>>ZSTD_FREQ_DIV); + optPtr->matchLengthSum += optPtr->matchLengthFreq[u]; + optPtr->matchSum += optPtr->matchLengthFreq[u] * (u + 3); } - ssPtr->matchSum *= ZSTD_LITFREQ_ADD; + optPtr->matchSum *= ZSTD_LITFREQ_ADD; for (u=0; u<=MaxOff; u++) { - ssPtr->offCodeFreq[u] = 1 + (ssPtr->offCodeFreq[u]>>ZSTD_FREQ_DIV); - ssPtr->offCodeSum += ssPtr->offCodeFreq[u]; + optPtr->offCodeFreq[u] = 1 + (optPtr->offCodeFreq[u]>>ZSTD_FREQ_DIV); + optPtr->offCodeSum += optPtr->offCodeFreq[u]; } } - ZSTD_setLog2Prices(ssPtr); + ZSTD_setLog2Prices(optPtr); } -FORCE_INLINE U32 ZSTD_getLiteralPrice(seqStore_t* ssPtr, U32 litLength, const BYTE* literals) +static U32 ZSTD_getLiteralPrice(optState_t* optPtr, U32 litLength, const BYTE* literals) { U32 price, u; - if (ssPtr->staticPrices) + if (optPtr->staticPrices) return ZSTD_highbit32((U32)litLength+1) + (litLength*6); if (litLength == 0) - return ssPtr->log2litLengthSum - ZSTD_highbit32(ssPtr->litLengthFreq[0]+1); + return optPtr->log2litLengthSum - ZSTD_highbit32(optPtr->litLengthFreq[0]+1); /* literals */ - if (ssPtr->cachedLiterals == literals) { - U32 const additional = litLength - ssPtr->cachedLitLength; - const BYTE* literals2 = ssPtr->cachedLiterals + ssPtr->cachedLitLength; - price = ssPtr->cachedPrice + additional * ssPtr->log2litSum; + if (optPtr->cachedLiterals == literals) { + U32 const additional = litLength - optPtr->cachedLitLength; + const BYTE* literals2 = optPtr->cachedLiterals + optPtr->cachedLitLength; + price = optPtr->cachedPrice + additional * optPtr->log2litSum; for (u=0; u < additional; u++) - price -= ZSTD_highbit32(ssPtr->litFreq[literals2[u]]+1); - ssPtr->cachedPrice = price; - ssPtr->cachedLitLength = litLength; + price -= ZSTD_highbit32(optPtr->litFreq[literals2[u]]+1); + optPtr->cachedPrice = price; + optPtr->cachedLitLength = litLength; } else { - price = litLength * ssPtr->log2litSum; + price = litLength * optPtr->log2litSum; for (u=0; u < litLength; u++) - price -= ZSTD_highbit32(ssPtr->litFreq[literals[u]]+1); + price -= ZSTD_highbit32(optPtr->litFreq[literals[u]]+1); if (litLength >= 12) { - ssPtr->cachedLiterals = literals; - ssPtr->cachedPrice = price; - ssPtr->cachedLitLength = litLength; + optPtr->cachedLiterals = literals; + optPtr->cachedPrice = price; + optPtr->cachedLitLength = litLength; } } /* literal Length */ { const BYTE LL_deltaCode = 19; const BYTE llCode = (litLength>63) ? (BYTE)ZSTD_highbit32(litLength) + LL_deltaCode : LL_Code[litLength]; - price += LL_bits[llCode] + ssPtr->log2litLengthSum - ZSTD_highbit32(ssPtr->litLengthFreq[llCode]+1); + price += LL_bits[llCode] + optPtr->log2litLengthSum - ZSTD_highbit32(optPtr->litLengthFreq[llCode]+1); } return price; } -FORCE_INLINE U32 ZSTD_getPrice(seqStore_t* seqStorePtr, U32 litLength, const BYTE* literals, U32 offset, U32 matchLength, const int ultra) +FORCE_INLINE_TEMPLATE U32 ZSTD_getPrice(optState_t* optPtr, U32 litLength, const BYTE* literals, U32 offset, U32 matchLength, const int ultra) { /* offset */ U32 price; BYTE const offCode = (BYTE)ZSTD_highbit32(offset+1); - if (seqStorePtr->staticPrices) - return ZSTD_getLiteralPrice(seqStorePtr, litLength, literals) + ZSTD_highbit32((U32)matchLength+1) + 16 + offCode; + if (optPtr->staticPrices) + return ZSTD_getLiteralPrice(optPtr, litLength, literals) + ZSTD_highbit32((U32)matchLength+1) + 16 + offCode; - price = offCode + seqStorePtr->log2offCodeSum - ZSTD_highbit32(seqStorePtr->offCodeFreq[offCode]+1); + price = offCode + optPtr->log2offCodeSum - ZSTD_highbit32(optPtr->offCodeFreq[offCode]+1); if (!ultra && offCode >= 20) price += (offCode-19)*2; /* match Length */ { const BYTE ML_deltaCode = 36; const BYTE mlCode = (matchLength>127) ? (BYTE)ZSTD_highbit32(matchLength) + ML_deltaCode : ML_Code[matchLength]; - price += ML_bits[mlCode] + seqStorePtr->log2matchLengthSum - ZSTD_highbit32(seqStorePtr->matchLengthFreq[mlCode]+1); + price += ML_bits[mlCode] + optPtr->log2matchLengthSum - ZSTD_highbit32(optPtr->matchLengthFreq[mlCode]+1); } - return price + ZSTD_getLiteralPrice(seqStorePtr, litLength, literals) + seqStorePtr->factor; + return price + ZSTD_getLiteralPrice(optPtr, litLength, literals) + optPtr->factor; } -MEM_STATIC void ZSTD_updatePrice(seqStore_t* seqStorePtr, U32 litLength, const BYTE* literals, U32 offset, U32 matchLength) +static void ZSTD_updatePrice(optState_t* optPtr, U32 litLength, const BYTE* literals, U32 offset, U32 matchLength) { U32 u; /* literals */ - seqStorePtr->litSum += litLength*ZSTD_LITFREQ_ADD; + optPtr->litSum += litLength*ZSTD_LITFREQ_ADD; for (u=0; u < litLength; u++) - seqStorePtr->litFreq[literals[u]] += ZSTD_LITFREQ_ADD; + optPtr->litFreq[literals[u]] += ZSTD_LITFREQ_ADD; /* literal Length */ { const BYTE LL_deltaCode = 19; const BYTE llCode = (litLength>63) ? (BYTE)ZSTD_highbit32(litLength) + LL_deltaCode : LL_Code[litLength]; - seqStorePtr->litLengthFreq[llCode]++; - seqStorePtr->litLengthSum++; + optPtr->litLengthFreq[llCode]++; + optPtr->litLengthSum++; } /* match offset */ { BYTE const offCode = (BYTE)ZSTD_highbit32(offset+1); - seqStorePtr->offCodeSum++; - seqStorePtr->offCodeFreq[offCode]++; + optPtr->offCodeSum++; + optPtr->offCodeFreq[offCode]++; } /* match Length */ { const BYTE ML_deltaCode = 36; const BYTE mlCode = (matchLength>127) ? (BYTE)ZSTD_highbit32(matchLength) + ML_deltaCode : ML_Code[matchLength]; - seqStorePtr->matchLengthFreq[mlCode]++; - seqStorePtr->matchLengthSum++; + optPtr->matchLengthFreq[mlCode]++; + optPtr->matchLengthSum++; } - ZSTD_setLog2Prices(seqStorePtr); + ZSTD_setLog2Prices(optPtr); } @@ -203,7 +203,7 @@ MEM_STATIC void ZSTD_updatePrice(seqStore_t* seqStorePtr, U32 litLength, const B /* function safe only for comparisons */ -MEM_STATIC U32 ZSTD_readMINMATCH(const void* memPtr, U32 length) +static U32 ZSTD_readMINMATCH(const void* memPtr, U32 length) { switch (length) { @@ -219,7 +219,7 @@ MEM_STATIC U32 ZSTD_readMINMATCH(const void* memPtr, U32 length) /* Update hashTable3 up to ip (excluded) Assumption : always within prefix (i.e. not within extDict) */ -FORCE_INLINE +static U32 ZSTD_insertAndFindFirstIndexHash3 (ZSTD_CCtx* zc, const BYTE* ip) { U32* const hashTable3 = zc->hashTable3; @@ -412,11 +412,12 @@ static U32 ZSTD_BtGetAllMatches_selectMLS_extDict ( /*-******************************* * Optimal parser *********************************/ -FORCE_INLINE +FORCE_INLINE_TEMPLATE void ZSTD_compressBlock_opt_generic(ZSTD_CCtx* ctx, const void* src, size_t srcSize, const int ultra) { seqStore_t* seqStorePtr = &(ctx->seqStore); + optState_t* optStatePtr = &(ctx->optState); const BYTE* const istart = (const BYTE*)src; const BYTE* ip = istart; const BYTE* anchor = istart; @@ -430,16 +431,16 @@ void ZSTD_compressBlock_opt_generic(ZSTD_CCtx* ctx, const U32 mls = ctx->appliedParams.cParams.searchLength; const U32 minMatch = (ctx->appliedParams.cParams.searchLength == 3) ? 3 : 4; - ZSTD_optimal_t* opt = seqStorePtr->priceTable; - ZSTD_match_t* matches = seqStorePtr->matchTable; + ZSTD_optimal_t* opt = optStatePtr->priceTable; + ZSTD_match_t* matches = optStatePtr->matchTable; const BYTE* inr; U32 offset, rep[ZSTD_REP_NUM]; /* init */ ctx->nextToUpdate3 = ctx->nextToUpdate; - ZSTD_rescaleFreqs(seqStorePtr, (const BYTE*)src, srcSize); + ZSTD_rescaleFreqs(optStatePtr, (const BYTE*)src, srcSize); ip += (ip==prefixStart); - { U32 i; for (i=0; irep[i]; } + { U32 i; for (i=0; irep[i]; } /* Match Loop */ while (ip < ilimit) { @@ -462,7 +463,7 @@ void ZSTD_compressBlock_opt_generic(ZSTD_CCtx* ctx, } best_off = i - (ip == anchor); do { - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); if (mlen > last_pos || price < opt[mlen].price) SET_PRICE(mlen, mlen, i, litlen, price); /* note : macro modifies last_pos */ mlen--; @@ -487,7 +488,7 @@ void ZSTD_compressBlock_opt_generic(ZSTD_CCtx* ctx, mlen = (u>0) ? matches[u-1].len+1 : best_mlen; best_mlen = matches[u].len; while (mlen <= best_mlen) { - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); if (mlen > last_pos || price < opt[mlen].price) SET_PRICE(mlen, mlen, matches[u].off, litlen, price); /* note : macro modifies last_pos */ mlen++; @@ -507,12 +508,12 @@ void ZSTD_compressBlock_opt_generic(ZSTD_CCtx* ctx, if (opt[cur-1].mlen == 1) { litlen = opt[cur-1].litlen + 1; if (cur > litlen) { - price = opt[cur - litlen].price + ZSTD_getLiteralPrice(seqStorePtr, litlen, inr-litlen); + price = opt[cur - litlen].price + ZSTD_getLiteralPrice(optStatePtr, litlen, inr-litlen); } else - price = ZSTD_getLiteralPrice(seqStorePtr, litlen, anchor); + price = ZSTD_getLiteralPrice(optStatePtr, litlen, anchor); } else { litlen = 1; - price = opt[cur - 1].price + ZSTD_getLiteralPrice(seqStorePtr, litlen, inr-1); + price = opt[cur - 1].price + ZSTD_getLiteralPrice(optStatePtr, litlen, inr-1); } if (cur > last_pos || price <= opt[cur].price) @@ -554,12 +555,12 @@ void ZSTD_compressBlock_opt_generic(ZSTD_CCtx* ctx, if (opt[cur].mlen == 1) { litlen = opt[cur].litlen; if (cur > litlen) { - price = opt[cur - litlen].price + ZSTD_getPrice(seqStorePtr, litlen, inr-litlen, best_off, mlen - MINMATCH, ultra); + price = opt[cur - litlen].price + ZSTD_getPrice(optStatePtr, litlen, inr-litlen, best_off, mlen - MINMATCH, ultra); } else - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); } else { litlen = 0; - price = opt[cur].price + ZSTD_getPrice(seqStorePtr, 0, NULL, best_off, mlen - MINMATCH, ultra); + price = opt[cur].price + ZSTD_getPrice(optStatePtr, 0, NULL, best_off, mlen - MINMATCH, ultra); } if (cur + mlen > last_pos || price <= opt[cur + mlen].price) @@ -586,12 +587,12 @@ void ZSTD_compressBlock_opt_generic(ZSTD_CCtx* ctx, if (opt[cur].mlen == 1) { litlen = opt[cur].litlen; if (cur > litlen) - price = opt[cur - litlen].price + ZSTD_getPrice(seqStorePtr, litlen, ip+cur-litlen, matches[u].off-1, mlen - MINMATCH, ultra); + price = opt[cur - litlen].price + ZSTD_getPrice(optStatePtr, litlen, ip+cur-litlen, matches[u].off-1, mlen - MINMATCH, ultra); else - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); } else { litlen = 0; - price = opt[cur].price + ZSTD_getPrice(seqStorePtr, 0, NULL, matches[u].off-1, mlen - MINMATCH, ultra); + price = opt[cur].price + ZSTD_getPrice(optStatePtr, 0, NULL, matches[u].off-1, mlen - MINMATCH, ultra); } if (cur + mlen > last_pos || (price < opt[cur + mlen].price)) @@ -645,13 +646,13 @@ _storeSequence: /* cur, last_pos, best_mlen, best_off have to be set */ if (litLength==0) offset--; } - ZSTD_updatePrice(seqStorePtr, litLength, anchor, offset, mlen-MINMATCH); + ZSTD_updatePrice(optStatePtr, litLength, anchor, offset, mlen-MINMATCH); ZSTD_storeSeq(seqStorePtr, litLength, anchor, offset, mlen-MINMATCH); anchor = ip = ip + mlen; } } /* for (cur=0; cur < last_pos; ) */ /* Save reps for next block */ - { int i; for (i=0; irepToConfirm[i] = rep[i]; } + { int i; for (i=0; irepToConfirm[i] = rep[i]; } /* Last Literals */ { size_t const lastLLSize = iend - anchor; @@ -661,11 +662,12 @@ _storeSequence: /* cur, last_pos, best_mlen, best_off have to be set */ } -FORCE_INLINE +FORCE_INLINE_TEMPLATE void ZSTD_compressBlock_opt_extDict_generic(ZSTD_CCtx* ctx, const void* src, size_t srcSize, const int ultra) { seqStore_t* seqStorePtr = &(ctx->seqStore); + optState_t* optStatePtr = &(ctx->optState); const BYTE* const istart = (const BYTE*)src; const BYTE* ip = istart; const BYTE* anchor = istart; @@ -683,16 +685,16 @@ void ZSTD_compressBlock_opt_extDict_generic(ZSTD_CCtx* ctx, const U32 mls = ctx->appliedParams.cParams.searchLength; const U32 minMatch = (ctx->appliedParams.cParams.searchLength == 3) ? 3 : 4; - ZSTD_optimal_t* opt = seqStorePtr->priceTable; - ZSTD_match_t* matches = seqStorePtr->matchTable; + ZSTD_optimal_t* opt = optStatePtr->priceTable; + ZSTD_match_t* matches = optStatePtr->matchTable; const BYTE* inr; /* init */ U32 offset, rep[ZSTD_REP_NUM]; - { U32 i; for (i=0; irep[i]; } + { U32 i; for (i=0; irep[i]; } ctx->nextToUpdate3 = ctx->nextToUpdate; - ZSTD_rescaleFreqs(seqStorePtr, (const BYTE*)src, srcSize); + ZSTD_rescaleFreqs(optStatePtr, (const BYTE*)src, srcSize); ip += (ip==prefixStart); /* Match Loop */ @@ -726,7 +728,7 @@ void ZSTD_compressBlock_opt_extDict_generic(ZSTD_CCtx* ctx, best_off = i - (ip==anchor); litlen = opt[0].litlen; do { - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); if (mlen > last_pos || price < opt[mlen].price) SET_PRICE(mlen, mlen, i, litlen, price); /* note : macro modifies last_pos */ mlen--; @@ -756,7 +758,7 @@ void ZSTD_compressBlock_opt_extDict_generic(ZSTD_CCtx* ctx, best_mlen = matches[u].len; litlen = opt[0].litlen; while (mlen <= best_mlen) { - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); if (mlen > last_pos || price < opt[mlen].price) SET_PRICE(mlen, mlen, matches[u].off, litlen, price); mlen++; @@ -773,12 +775,12 @@ void ZSTD_compressBlock_opt_extDict_generic(ZSTD_CCtx* ctx, if (opt[cur-1].mlen == 1) { litlen = opt[cur-1].litlen + 1; if (cur > litlen) { - price = opt[cur - litlen].price + ZSTD_getLiteralPrice(seqStorePtr, litlen, inr-litlen); + price = opt[cur - litlen].price + ZSTD_getLiteralPrice(optStatePtr, litlen, inr-litlen); } else - price = ZSTD_getLiteralPrice(seqStorePtr, litlen, anchor); + price = ZSTD_getLiteralPrice(optStatePtr, litlen, anchor); } else { litlen = 1; - price = opt[cur - 1].price + ZSTD_getLiteralPrice(seqStorePtr, litlen, inr-1); + price = opt[cur - 1].price + ZSTD_getLiteralPrice(optStatePtr, litlen, inr-1); } if (cur > last_pos || price <= opt[cur].price) @@ -826,12 +828,12 @@ void ZSTD_compressBlock_opt_extDict_generic(ZSTD_CCtx* ctx, if (opt[cur].mlen == 1) { litlen = opt[cur].litlen; if (cur > litlen) { - price = opt[cur - litlen].price + ZSTD_getPrice(seqStorePtr, litlen, inr-litlen, best_off, mlen - MINMATCH, ultra); + price = opt[cur - litlen].price + ZSTD_getPrice(optStatePtr, litlen, inr-litlen, best_off, mlen - MINMATCH, ultra); } else - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, best_off, mlen - MINMATCH, ultra); } else { litlen = 0; - price = opt[cur].price + ZSTD_getPrice(seqStorePtr, 0, NULL, best_off, mlen - MINMATCH, ultra); + price = opt[cur].price + ZSTD_getPrice(optStatePtr, 0, NULL, best_off, mlen - MINMATCH, ultra); } if (cur + mlen > last_pos || price <= opt[cur + mlen].price) @@ -858,12 +860,12 @@ void ZSTD_compressBlock_opt_extDict_generic(ZSTD_CCtx* ctx, if (opt[cur].mlen == 1) { litlen = opt[cur].litlen; if (cur > litlen) - price = opt[cur - litlen].price + ZSTD_getPrice(seqStorePtr, litlen, ip+cur-litlen, matches[u].off-1, mlen - MINMATCH, ultra); + price = opt[cur - litlen].price + ZSTD_getPrice(optStatePtr, litlen, ip+cur-litlen, matches[u].off-1, mlen - MINMATCH, ultra); else - price = ZSTD_getPrice(seqStorePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); + price = ZSTD_getPrice(optStatePtr, litlen, anchor, matches[u].off-1, mlen - MINMATCH, ultra); } else { litlen = 0; - price = opt[cur].price + ZSTD_getPrice(seqStorePtr, 0, NULL, matches[u].off-1, mlen - MINMATCH, ultra); + price = opt[cur].price + ZSTD_getPrice(optStatePtr, 0, NULL, matches[u].off-1, mlen - MINMATCH, ultra); } if (cur + mlen > last_pos || (price < opt[cur + mlen].price)) @@ -918,13 +920,13 @@ _storeSequence: /* cur, last_pos, best_mlen, best_off have to be set */ if (litLength==0) offset--; } - ZSTD_updatePrice(seqStorePtr, litLength, anchor, offset, mlen-MINMATCH); + ZSTD_updatePrice(optStatePtr, litLength, anchor, offset, mlen-MINMATCH); ZSTD_storeSeq(seqStorePtr, litLength, anchor, offset, mlen-MINMATCH); anchor = ip = ip + mlen; } } /* for (cur=0; cur < last_pos; ) */ /* Save reps for next block */ - { int i; for (i=0; irepToConfirm[i] = rep[i]; } + { int i; for (i=0; irepToConfirm[i] = rep[i]; } /* Last Literals */ { size_t lastLLSize = iend - anchor; diff --git a/contrib/libzstd/include/zstd/compress/zstdmt_compress.c b/contrib/libzstd/include/zstd/compress/zstdmt_compress.c index 0cee01eacb8..8564bc43921 100644 --- a/contrib/libzstd/include/zstd/compress/zstdmt_compress.c +++ b/contrib/libzstd/include/zstd/compress/zstdmt_compress.c @@ -1,15 +1,16 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ /* ====== Tuning parameters ====== */ -#define ZSTDMT_NBTHREADS_MAX 128 +#define ZSTDMT_NBTHREADS_MAX 256 +#define ZSTDMT_OVERLAPLOG_DEFAULT 6 /* ====== Compiler specifics ====== */ @@ -73,6 +74,7 @@ static unsigned long long GetCurrentClockTimeMicroseconds(void) /* ===== Buffer Pool ===== */ +/* a single Buffer Pool can be invoked from multiple threads in parallel */ typedef struct buffer_s { void* start; @@ -82,6 +84,8 @@ typedef struct buffer_s { static const buffer_t g_nullBuffer = { NULL, 0 }; typedef struct ZSTDMT_bufferPool_s { + pthread_mutex_t poolMutex; + size_t bufferSize; unsigned totalBuffers; unsigned nbBuffers; ZSTD_customMem cMem; @@ -90,10 +94,15 @@ typedef struct ZSTDMT_bufferPool_s { static ZSTDMT_bufferPool* ZSTDMT_createBufferPool(unsigned nbThreads, ZSTD_customMem cMem) { - unsigned const maxNbBuffers = 2*nbThreads + 2; + unsigned const maxNbBuffers = 2*nbThreads + 3; ZSTDMT_bufferPool* const bufPool = (ZSTDMT_bufferPool*)ZSTD_calloc( sizeof(ZSTDMT_bufferPool) + (maxNbBuffers-1) * sizeof(buffer_t), cMem); if (bufPool==NULL) return NULL; + if (pthread_mutex_init(&bufPool->poolMutex, NULL)) { + ZSTD_free(bufPool, cMem); + return NULL; + } + bufPool->bufferSize = 64 KB; bufPool->totalBuffers = maxNbBuffers; bufPool->nbBuffers = 0; bufPool->cMem = cMem; @@ -106,6 +115,7 @@ static void ZSTDMT_freeBufferPool(ZSTDMT_bufferPool* bufPool) if (!bufPool) return; /* compatibility with free on NULL */ for (u=0; utotalBuffers; u++) ZSTD_free(bufPool->bTable[u].start, bufPool->cMem); + pthread_mutex_destroy(&bufPool->poolMutex); ZSTD_free(bufPool, bufPool->cMem); } @@ -116,65 +126,85 @@ static size_t ZSTDMT_sizeof_bufferPool(ZSTDMT_bufferPool* bufPool) + (bufPool->totalBuffers - 1) * sizeof(buffer_t); unsigned u; size_t totalBufferSize = 0; + pthread_mutex_lock(&bufPool->poolMutex); for (u=0; utotalBuffers; u++) totalBufferSize += bufPool->bTable[u].size; + pthread_mutex_unlock(&bufPool->poolMutex); return poolSize + totalBufferSize; } -/** ZSTDMT_getBuffer() : - * assumption : invocation from main thread only ! */ -static buffer_t ZSTDMT_getBuffer(ZSTDMT_bufferPool* pool, size_t bSize) +static void ZSTDMT_setBufferSize(ZSTDMT_bufferPool* bufPool, size_t bSize) { - if (pool->nbBuffers) { /* try to use an existing buffer */ - buffer_t const buf = pool->bTable[--(pool->nbBuffers)]; + bufPool->bufferSize = bSize; +} + +/** ZSTDMT_getBuffer() : + * assumption : bufPool must be valid */ +static buffer_t ZSTDMT_getBuffer(ZSTDMT_bufferPool* bufPool) +{ + size_t const bSize = bufPool->bufferSize; + DEBUGLOG(5, "ZSTDMT_getBuffer"); + pthread_mutex_lock(&bufPool->poolMutex); + if (bufPool->nbBuffers) { /* try to use an existing buffer */ + buffer_t const buf = bufPool->bTable[--(bufPool->nbBuffers)]; size_t const availBufferSize = buf.size; - if ((availBufferSize >= bSize) & (availBufferSize <= 10*bSize)) + if ((availBufferSize >= bSize) & (availBufferSize <= 10*bSize)) { /* large enough, but not too much */ + pthread_mutex_unlock(&bufPool->poolMutex); return buf; + } /* size conditions not respected : scratch this buffer, create new one */ - ZSTD_free(buf.start, pool->cMem); + DEBUGLOG(5, "existing buffer does not meet size conditions => freeing"); + ZSTD_free(buf.start, bufPool->cMem); } + pthread_mutex_unlock(&bufPool->poolMutex); /* create new buffer */ + DEBUGLOG(5, "create a new buffer"); { buffer_t buffer; - void* const start = ZSTD_malloc(bSize, pool->cMem); - if (start==NULL) bSize = 0; + void* const start = ZSTD_malloc(bSize, bufPool->cMem); buffer.start = start; /* note : start can be NULL if malloc fails ! */ - buffer.size = bSize; + buffer.size = (start==NULL) ? 0 : bSize; return buffer; } } /* store buffer for later re-use, up to pool capacity */ -static void ZSTDMT_releaseBuffer(ZSTDMT_bufferPool* pool, buffer_t buf) +static void ZSTDMT_releaseBuffer(ZSTDMT_bufferPool* bufPool, buffer_t buf) { - if (buf.start == NULL) return; /* release on NULL */ - if (pool->nbBuffers < pool->totalBuffers) { - pool->bTable[pool->nbBuffers++] = buf; /* store for later re-use */ + if (buf.start == NULL) return; /* compatible with release on NULL */ + DEBUGLOG(5, "ZSTDMT_releaseBuffer"); + pthread_mutex_lock(&bufPool->poolMutex); + if (bufPool->nbBuffers < bufPool->totalBuffers) { + bufPool->bTable[bufPool->nbBuffers++] = buf; /* stored for later use */ + pthread_mutex_unlock(&bufPool->poolMutex); return; } + pthread_mutex_unlock(&bufPool->poolMutex); /* Reached bufferPool capacity (should not happen) */ - ZSTD_free(buf.start, pool->cMem); + DEBUGLOG(5, "buffer pool capacity reached => freeing "); + ZSTD_free(buf.start, bufPool->cMem); } /* ===== CCtx Pool ===== */ +/* a single CCtx Pool can be invoked from multiple threads in parallel */ typedef struct { + pthread_mutex_t poolMutex; unsigned totalCCtx; unsigned availCCtx; ZSTD_customMem cMem; ZSTD_CCtx* cctx[1]; /* variable size */ } ZSTDMT_CCtxPool; -/* assumption : CCtxPool invocation only from main thread */ - /* note : all CCtx borrowed from the pool should be released back to the pool _before_ freeing the pool */ static void ZSTDMT_freeCCtxPool(ZSTDMT_CCtxPool* pool) { unsigned u; for (u=0; utotalCCtx; u++) ZSTD_freeCCtx(pool->cctx[u]); /* note : compatible with free on NULL */ + pthread_mutex_destroy(&pool->poolMutex); ZSTD_free(pool, pool->cMem); } @@ -186,6 +216,10 @@ static ZSTDMT_CCtxPool* ZSTDMT_createCCtxPool(unsigned nbThreads, ZSTDMT_CCtxPool* const cctxPool = (ZSTDMT_CCtxPool*) ZSTD_calloc( sizeof(ZSTDMT_CCtxPool) + (nbThreads-1)*sizeof(ZSTD_CCtx*), cMem); if (!cctxPool) return NULL; + if (pthread_mutex_init(&cctxPool->poolMutex, NULL)) { + ZSTD_free(cctxPool, cMem); + return NULL; + } cctxPool->cMem = cMem; cctxPool->totalCCtx = nbThreads; cctxPool->availCCtx = 1; /* at least one cctx for single-thread mode */ @@ -198,50 +232,57 @@ static ZSTDMT_CCtxPool* ZSTDMT_createCCtxPool(unsigned nbThreads, /* only works during initialization phase, not during compression */ static size_t ZSTDMT_sizeof_CCtxPool(ZSTDMT_CCtxPool* cctxPool) { - unsigned const nbThreads = cctxPool->totalCCtx; - size_t const poolSize = sizeof(*cctxPool) - + (nbThreads-1)*sizeof(ZSTD_CCtx*); - unsigned u; - size_t totalCCtxSize = 0; - for (u=0; ucctx[u]); - - return poolSize + totalCCtxSize; + pthread_mutex_lock(&cctxPool->poolMutex); + { unsigned const nbThreads = cctxPool->totalCCtx; + size_t const poolSize = sizeof(*cctxPool) + + (nbThreads-1)*sizeof(ZSTD_CCtx*); + unsigned u; + size_t totalCCtxSize = 0; + for (u=0; ucctx[u]); + } + pthread_mutex_unlock(&cctxPool->poolMutex); + return poolSize + totalCCtxSize; + } } -static ZSTD_CCtx* ZSTDMT_getCCtx(ZSTDMT_CCtxPool* pool) +static ZSTD_CCtx* ZSTDMT_getCCtx(ZSTDMT_CCtxPool* cctxPool) { - if (pool->availCCtx) { - pool->availCCtx--; - return pool->cctx[pool->availCCtx]; - } - return ZSTD_createCCtx(); /* note : can be NULL, when creation fails ! */ + DEBUGLOG(5, "ZSTDMT_getCCtx"); + pthread_mutex_lock(&cctxPool->poolMutex); + if (cctxPool->availCCtx) { + cctxPool->availCCtx--; + { ZSTD_CCtx* const cctx = cctxPool->cctx[cctxPool->availCCtx]; + pthread_mutex_unlock(&cctxPool->poolMutex); + return cctx; + } } + pthread_mutex_unlock(&cctxPool->poolMutex); + DEBUGLOG(5, "create one more CCtx"); + return ZSTD_createCCtx_advanced(cctxPool->cMem); /* note : can be NULL, when creation fails ! */ } static void ZSTDMT_releaseCCtx(ZSTDMT_CCtxPool* pool, ZSTD_CCtx* cctx) { if (cctx==NULL) return; /* compatibility with release on NULL */ + pthread_mutex_lock(&pool->poolMutex); if (pool->availCCtx < pool->totalCCtx) pool->cctx[pool->availCCtx++] = cctx; - else + else { /* pool overflow : should not happen, since totalCCtx==nbThreads */ + DEBUGLOG(5, "CCtx pool overflow : free cctx"); ZSTD_freeCCtx(cctx); + } + pthread_mutex_unlock(&pool->poolMutex); } /* ===== Thread worker ===== */ typedef struct { - buffer_t buffer; - size_t filled; -} inBuff_t; - -typedef struct { - ZSTD_CCtx* cctx; buffer_t src; const void* srcStart; - size_t srcSize; size_t dictSize; + size_t srcSize; buffer_t dstBuff; size_t cSize; size_t dstFlushed; @@ -253,6 +294,8 @@ typedef struct { pthread_cond_t* jobCompleted_cond; ZSTD_parameters params; const ZSTD_CDict* cdict; + ZSTDMT_CCtxPool* cctxPool; + ZSTDMT_bufferPool* bufPool; unsigned long long fullFrameSize; } ZSTDMT_jobDescription; @@ -260,37 +303,56 @@ typedef struct { void ZSTDMT_compressChunk(void* jobDescription) { ZSTDMT_jobDescription* const job = (ZSTDMT_jobDescription*)jobDescription; + ZSTD_CCtx* cctx = ZSTDMT_getCCtx(job->cctxPool); const void* const src = (const char*)job->srcStart + job->dictSize; - buffer_t const dstBuff = job->dstBuff; + buffer_t dstBuff = job->dstBuff; DEBUGLOG(5, "job (first:%u) (last:%u) : dictSize %u, srcSize %u", job->firstChunk, job->lastChunk, (U32)job->dictSize, (U32)job->srcSize); + + if (cctx==NULL) { + job->cSize = ERROR(memory_allocation); + goto _endJob; + } + + if (dstBuff.start == NULL) { + dstBuff = ZSTDMT_getBuffer(job->bufPool); + if (dstBuff.start==NULL) { + job->cSize = ERROR(memory_allocation); + goto _endJob; + } + job->dstBuff = dstBuff; + } + if (job->cdict) { /* should only happen for first segment */ - size_t const initError = ZSTD_compressBegin_usingCDict_advanced(job->cctx, job->cdict, job->params.fParams, job->fullFrameSize); + size_t const initError = ZSTD_compressBegin_usingCDict_advanced(cctx, job->cdict, job->params.fParams, job->fullFrameSize); DEBUGLOG(5, "using CDict"); if (ZSTD_isError(initError)) { job->cSize = initError; goto _endJob; } } else { /* srcStart points at reloaded section */ if (!job->firstChunk) job->params.fParams.contentSizeFlag = 0; /* ensure no srcSize control */ - { size_t const dictModeError = ZSTD_setCCtxParameter(job->cctx, ZSTD_p_forceRawDict, 1); /* Force loading dictionary in "content-only" mode (no header analysis) */ - size_t const initError = ZSTD_compressBegin_advanced(job->cctx, job->srcStart, job->dictSize, job->params, job->fullFrameSize); + { size_t const dictModeError = ZSTD_setCCtxParameter(cctx, ZSTD_p_forceRawDict, 1); /* Force loading dictionary in "content-only" mode (no header analysis) */ + size_t const initError = ZSTD_compressBegin_advanced(cctx, job->srcStart, job->dictSize, job->params, job->fullFrameSize); if (ZSTD_isError(initError) || ZSTD_isError(dictModeError)) { job->cSize = initError; goto _endJob; } - ZSTD_setCCtxParameter(job->cctx, ZSTD_p_forceWindow, 1); + ZSTD_setCCtxParameter(cctx, ZSTD_p_forceWindow, 1); } } if (!job->firstChunk) { /* flush and overwrite frame header when it's not first segment */ - size_t const hSize = ZSTD_compressContinue(job->cctx, dstBuff.start, dstBuff.size, src, 0); + size_t const hSize = ZSTD_compressContinue(cctx, dstBuff.start, dstBuff.size, src, 0); if (ZSTD_isError(hSize)) { job->cSize = hSize; goto _endJob; } - ZSTD_invalidateRepCodes(job->cctx); + ZSTD_invalidateRepCodes(cctx); } DEBUGLOG(5, "Compressing : "); DEBUG_PRINTHEX(4, job->srcStart, 12); job->cSize = (job->lastChunk) ? - ZSTD_compressEnd (job->cctx, dstBuff.start, dstBuff.size, src, job->srcSize) : - ZSTD_compressContinue(job->cctx, dstBuff.start, dstBuff.size, src, job->srcSize); + ZSTD_compressEnd (cctx, dstBuff.start, dstBuff.size, src, job->srcSize) : + ZSTD_compressContinue(cctx, dstBuff.start, dstBuff.size, src, job->srcSize); DEBUGLOG(5, "compressed %u bytes into %u bytes (first:%u) (last:%u)", (unsigned)job->srcSize, (unsigned)job->cSize, job->firstChunk, job->lastChunk); DEBUGLOG(5, "dstBuff.size : %u ; => %s", (U32)dstBuff.size, ZSTD_getErrorName(job->cSize)); _endJob: + ZSTDMT_releaseCCtx(job->cctxPool, cctx); + ZSTDMT_releaseBuffer(job->bufPool, job->src); + job->src = g_nullBuffer; job->srcStart = NULL; PTHREAD_MUTEX_LOCK(job->jobCompleted_mutex); job->jobCompleted = 1; job->jobScanned = 0; @@ -303,15 +365,19 @@ _endJob: /* ===== Multi-threaded compression ===== */ /* ------------------------------------------ */ +typedef struct { + buffer_t buffer; + size_t filled; +} inBuff_t; + struct ZSTDMT_CCtx_s { POOL_ctx* factory; ZSTDMT_jobDescription* jobs; - ZSTDMT_bufferPool* buffPool; + ZSTDMT_bufferPool* bufPool; ZSTDMT_CCtxPool* cctxPool; pthread_mutex_t jobCompleted_mutex; pthread_cond_t jobCompleted_cond; size_t targetSectionSize; - size_t marginSize; size_t inBuffSize; size_t dictSize; size_t targetDictSize; @@ -324,7 +390,7 @@ struct ZSTDMT_CCtx_s { unsigned nextJobID; unsigned frameEnded; unsigned allJobsCompleted; - unsigned overlapRLog; + unsigned overlapLog; unsigned long long frameContentSize; size_t sectionSize; ZSTD_customMem cMem; @@ -347,7 +413,8 @@ ZSTDMT_CCtx* ZSTDMT_createCCtx_advanced(unsigned nbThreads, ZSTD_customMem cMem) U32 nbJobs = nbThreads + 2; DEBUGLOG(3, "ZSTDMT_createCCtx_advanced"); - if ((nbThreads < 1) | (nbThreads > ZSTDMT_NBTHREADS_MAX)) return NULL; + if (nbThreads < 1) return NULL; + nbThreads = MIN(nbThreads , ZSTDMT_NBTHREADS_MAX); if ((cMem.customAlloc!=NULL) ^ (cMem.customFree!=NULL)) /* invalid custom allocator */ return NULL; @@ -358,18 +425,24 @@ ZSTDMT_CCtx* ZSTDMT_createCCtx_advanced(unsigned nbThreads, ZSTD_customMem cMem) mtctx->nbThreads = nbThreads; mtctx->allJobsCompleted = 1; mtctx->sectionSize = 0; - mtctx->overlapRLog = 3; - mtctx->factory = POOL_create(nbThreads, 1); + mtctx->overlapLog = ZSTDMT_OVERLAPLOG_DEFAULT; + mtctx->factory = POOL_create(nbThreads, 0); mtctx->jobs = ZSTDMT_allocJobsTable(&nbJobs, cMem); mtctx->jobIDMask = nbJobs - 1; - mtctx->buffPool = ZSTDMT_createBufferPool(nbThreads, cMem); + mtctx->bufPool = ZSTDMT_createBufferPool(nbThreads, cMem); mtctx->cctxPool = ZSTDMT_createCCtxPool(nbThreads, cMem); - if (!mtctx->factory | !mtctx->jobs | !mtctx->buffPool | !mtctx->cctxPool) { + if (!mtctx->factory | !mtctx->jobs | !mtctx->bufPool | !mtctx->cctxPool) { + ZSTDMT_freeCCtx(mtctx); + return NULL; + } + if (pthread_mutex_init(&mtctx->jobCompleted_mutex, NULL)) { + ZSTDMT_freeCCtx(mtctx); + return NULL; + } + if (pthread_cond_init(&mtctx->jobCompleted_cond, NULL)) { ZSTDMT_freeCCtx(mtctx); return NULL; } - pthread_mutex_init(&mtctx->jobCompleted_mutex, NULL); /* Todo : check init function return */ - pthread_cond_init(&mtctx->jobCompleted_cond, NULL); DEBUGLOG(3, "mt_cctx created, for %u threads", nbThreads); return mtctx; } @@ -386,15 +459,13 @@ static void ZSTDMT_releaseAllJobResources(ZSTDMT_CCtx* mtctx) unsigned jobID; DEBUGLOG(3, "ZSTDMT_releaseAllJobResources"); for (jobID=0; jobID <= mtctx->jobIDMask; jobID++) { - ZSTDMT_releaseBuffer(mtctx->buffPool, mtctx->jobs[jobID].dstBuff); + ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->jobs[jobID].dstBuff); mtctx->jobs[jobID].dstBuff = g_nullBuffer; - ZSTDMT_releaseBuffer(mtctx->buffPool, mtctx->jobs[jobID].src); + ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->jobs[jobID].src); mtctx->jobs[jobID].src = g_nullBuffer; - ZSTDMT_releaseCCtx(mtctx->cctxPool, mtctx->jobs[jobID].cctx); - mtctx->jobs[jobID].cctx = NULL; } memset(mtctx->jobs, 0, (mtctx->jobIDMask+1)*sizeof(ZSTDMT_jobDescription)); - ZSTDMT_releaseBuffer(mtctx->buffPool, mtctx->inBuff.buffer); + ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->inBuff.buffer); mtctx->inBuff.buffer = g_nullBuffer; mtctx->allJobsCompleted = 1; } @@ -404,7 +475,7 @@ size_t ZSTDMT_freeCCtx(ZSTDMT_CCtx* mtctx) if (mtctx==NULL) return 0; /* compatible with free on NULL */ POOL_free(mtctx->factory); if (!mtctx->allJobsCompleted) ZSTDMT_releaseAllJobResources(mtctx); /* stop workers first */ - ZSTDMT_freeBufferPool(mtctx->buffPool); /* release job resources into pools first */ + ZSTDMT_freeBufferPool(mtctx->bufPool); /* release job resources into pools first */ ZSTD_free(mtctx->jobs, mtctx->cMem); ZSTDMT_freeCCtxPool(mtctx->cctxPool); ZSTD_freeCDict(mtctx->cdictLocal); @@ -418,11 +489,11 @@ size_t ZSTDMT_sizeof_CCtx(ZSTDMT_CCtx* mtctx) { if (mtctx == NULL) return 0; /* supports sizeof NULL */ return sizeof(*mtctx) - + POOL_sizeof(mtctx->factory) - + ZSTDMT_sizeof_bufferPool(mtctx->buffPool) - + (mtctx->jobIDMask+1) * sizeof(ZSTDMT_jobDescription) - + ZSTDMT_sizeof_CCtxPool(mtctx->cctxPool) - + ZSTD_sizeof_CDict(mtctx->cdictLocal); + + POOL_sizeof(mtctx->factory) + + ZSTDMT_sizeof_bufferPool(mtctx->bufPool) + + (mtctx->jobIDMask+1) * sizeof(ZSTDMT_jobDescription) + + ZSTDMT_sizeof_CCtxPool(mtctx->cctxPool) + + ZSTD_sizeof_CDict(mtctx->cdictLocal); } size_t ZSTDMT_setMTCtxParameter(ZSTDMT_CCtx* mtctx, ZSDTMT_parameter parameter, unsigned value) @@ -434,10 +505,10 @@ size_t ZSTDMT_setMTCtxParameter(ZSTDMT_CCtx* mtctx, ZSDTMT_parameter parameter, return 0; case ZSTDMT_p_overlapSectionLog : DEBUGLOG(5, "ZSTDMT_p_overlapSectionLog : %u", value); - mtctx->overlapRLog = (value >= 9) ? 0 : 9 - value; + mtctx->overlapLog = (value >= 9) ? 9 : value; return 0; default : - return ERROR(compressionParameter_unsupported); + return ERROR(parameter_unsupported); } } @@ -459,12 +530,13 @@ static unsigned computeNbChunks(size_t srcSize, unsigned windowLog, unsigned nbT size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, - void* dst, size_t dstCapacity, - const void* src, size_t srcSize, - const ZSTD_CDict* cdict, - ZSTD_parameters const params, - unsigned overlapRLog) + void* dst, size_t dstCapacity, + const void* src, size_t srcSize, + const ZSTD_CDict* cdict, + ZSTD_parameters const params, + unsigned overlapLog) { + unsigned const overlapRLog = (overlapLog>9) ? 0 : 9-overlapLog; size_t const overlapSize = (overlapRLog>=9) ? 0 : (size_t)1 << (params.cParams.windowLog - overlapRLog); unsigned nbChunks = computeNbChunks(srcSize, params.cParams.windowLog, mtctx->nbThreads); size_t const proposedChunkSize = (srcSize + (nbChunks-1)) / nbChunks; @@ -473,6 +545,7 @@ size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, size_t remainingSrcSize = srcSize; unsigned const compressWithinDst = (dstCapacity >= ZSTD_compressBound(srcSize)) ? nbChunks : (unsigned)(dstCapacity / ZSTD_compressBound(avgChunkSize)); /* presumes avgChunkSize >= 256 KB, which should be the case */ size_t frameStartPos = 0, dstBufferPos = 0; + XXH64_state_t xxh64; DEBUGLOG(4, "nbChunks : %2u (chunkSize : %u bytes) ", nbChunks, (U32)avgChunkSize); if (nbChunks==1) { /* fallback to single-thread mode */ @@ -480,7 +553,9 @@ size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, if (cdict) return ZSTD_compress_usingCDict_advanced(cctx, dst, dstCapacity, src, srcSize, cdict, params.fParams); return ZSTD_compress_advanced(cctx, dst, dstCapacity, src, srcSize, NULL, 0, params); } - assert(avgChunkSize >= 256 KB); /* condition for ZSTD_compressBound(A) + ZSTD_compressBound(B) <= ZSTD_compressBound(A+B), which is useful to avoid allocating extra buffers */ + assert(avgChunkSize >= 256 KB); /* condition for ZSTD_compressBound(A) + ZSTD_compressBound(B) <= ZSTD_compressBound(A+B), which is required for compressWithinDst */ + ZSTDMT_setBufferSize(mtctx->bufPool, ZSTD_compressBound(avgChunkSize) ); + XXH64_reset(&xxh64, 0); if (nbChunks > mtctx->jobIDMask+1) { /* enlarge job table */ U32 nbJobs = nbChunks; @@ -496,17 +571,10 @@ size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, size_t const chunkSize = MIN(remainingSrcSize, avgChunkSize); size_t const dstBufferCapacity = ZSTD_compressBound(chunkSize); buffer_t const dstAsBuffer = { (char*)dst + dstBufferPos, dstBufferCapacity }; - buffer_t const dstBuffer = u < compressWithinDst ? dstAsBuffer : ZSTDMT_getBuffer(mtctx->buffPool, dstBufferCapacity); - ZSTD_CCtx* const cctx = ZSTDMT_getCCtx(mtctx->cctxPool); + buffer_t const dstBuffer = u < compressWithinDst ? dstAsBuffer : g_nullBuffer; size_t dictSize = u ? overlapSize : 0; - if ((cctx==NULL) || (dstBuffer.start==NULL)) { - mtctx->jobs[u].cSize = ERROR(memory_allocation); /* job result */ - mtctx->jobs[u].jobCompleted = 1; - nbChunks = u+1; /* only wait and free u jobs, instead of initially expected nbChunks ones */ - break; /* let's wait for previous jobs to complete, but don't start new ones */ - } - + mtctx->jobs[u].src = g_nullBuffer; mtctx->jobs[u].srcStart = srcStart + frameStartPos - dictSize; mtctx->jobs[u].dictSize = dictSize; mtctx->jobs[u].srcSize = chunkSize; @@ -516,13 +584,18 @@ size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, /* do not calculate checksum within sections, but write it in header for first section */ if (u!=0) mtctx->jobs[u].params.fParams.checksumFlag = 0; mtctx->jobs[u].dstBuff = dstBuffer; - mtctx->jobs[u].cctx = cctx; + mtctx->jobs[u].cctxPool = mtctx->cctxPool; + mtctx->jobs[u].bufPool = mtctx->bufPool; mtctx->jobs[u].firstChunk = (u==0); mtctx->jobs[u].lastChunk = (u==nbChunks-1); mtctx->jobs[u].jobCompleted = 0; mtctx->jobs[u].jobCompleted_mutex = &mtctx->jobCompleted_mutex; mtctx->jobs[u].jobCompleted_cond = &mtctx->jobCompleted_cond; + if (params.fParams.checksumFlag) { + XXH64_update(&xxh64, srcStart + frameStartPos, chunkSize); + } + DEBUGLOG(5, "posting job %u (%u bytes)", u, (U32)chunkSize); DEBUG_PRINTHEX(6, mtctx->jobs[u].srcStart, 12); POOL_add(mtctx->factory, ZSTDMT_compressChunk, &mtctx->jobs[u]); @@ -533,8 +606,8 @@ size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, } } /* collect result */ - { unsigned chunkID; - size_t error = 0, dstPos = 0; + { size_t error = 0, dstPos = 0; + unsigned chunkID; for (chunkID=0; chunkIDjobCompleted_mutex); @@ -545,8 +618,6 @@ size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, pthread_mutex_unlock(&mtctx->jobCompleted_mutex); DEBUGLOG(5, "ready to write chunk %u ", chunkID); - ZSTDMT_releaseCCtx(mtctx->cctxPool, mtctx->jobs[chunkID].cctx); - mtctx->jobs[chunkID].cctx = NULL; mtctx->jobs[chunkID].srcStart = NULL; { size_t const cSize = mtctx->jobs[chunkID].cSize; if (ZSTD_isError(cSize)) error = cSize; @@ -556,13 +627,25 @@ size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, memmove((char*)dst + dstPos, mtctx->jobs[chunkID].dstBuff.start, cSize); /* may overlap when chunk compressed within dst */ if (chunkID >= compressWithinDst) { /* chunk compressed into its own buffer, which must be released */ DEBUGLOG(5, "releasing buffer %u>=%u", chunkID, compressWithinDst); - ZSTDMT_releaseBuffer(mtctx->buffPool, mtctx->jobs[chunkID].dstBuff); + ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->jobs[chunkID].dstBuff); } mtctx->jobs[chunkID].dstBuff = g_nullBuffer; } dstPos += cSize ; } - } + } /* for (chunkID=0; chunkID dstCapacity) { + error = ERROR(dstSize_tooSmall); + } else { + DEBUGLOG(4, "writing checksum : %08X \n", checksum); + MEM_writeLE32((char*)dst + dstPos, checksum); + dstPos += 4; + } } + if (!error) DEBUGLOG(4, "compressed size : %u ", (U32)dstPos); return error ? error : dstPos; } @@ -574,10 +657,10 @@ size_t ZSTDMT_compressCCtx(ZSTDMT_CCtx* mtctx, const void* src, size_t srcSize, int compressionLevel) { - U32 const overlapRLog = (compressionLevel >= ZSTD_maxCLevel()) ? 0 : 3; + U32 const overlapLog = (compressionLevel >= ZSTD_maxCLevel()) ? 9 : ZSTDMT_OVERLAPLOG_DEFAULT; ZSTD_parameters params = ZSTD_getParams(compressionLevel, srcSize, 0); params.fParams.contentSizeFlag = 1; - return ZSTDMT_compress_advanced(mtctx, dst, dstCapacity, src, srcSize, NULL, params, overlapRLog); + return ZSTDMT_compress_advanced(mtctx, dst, dstCapacity, src, srcSize, NULL, params, overlapLog); } @@ -615,8 +698,8 @@ size_t ZSTDMT_initCStream_internal(ZSTDMT_CCtx* zcs, if (zcs->nbThreads==1) { DEBUGLOG(4, "single thread mode"); return ZSTD_initCStream_internal(zcs->cctxPool->cctx[0], - dict, dictSize, cdict, - params, pledgedSrcSize); + dict, dictSize, cdict, + params, pledgedSrcSize); } if (zcs->allJobsCompleted == 0) { /* previous compression not correctly finished */ @@ -642,18 +725,16 @@ size_t ZSTDMT_initCStream_internal(ZSTDMT_CCtx* zcs, zcs->cdict = cdict; } - zcs->targetDictSize = (zcs->overlapRLog>=9) ? 0 : (size_t)1 << (zcs->params.cParams.windowLog - zcs->overlapRLog); - DEBUGLOG(4, "overlapRLog : %u ", zcs->overlapRLog); + zcs->targetDictSize = (zcs->overlapLog==0) ? 0 : (size_t)1 << (zcs->params.cParams.windowLog - (9 - zcs->overlapLog)); + DEBUGLOG(4, "overlapLog : %u ", zcs->overlapLog); DEBUGLOG(4, "overlap Size : %u KB", (U32)(zcs->targetDictSize>>10)); zcs->targetSectionSize = zcs->sectionSize ? zcs->sectionSize : (size_t)1 << (zcs->params.cParams.windowLog + 2); zcs->targetSectionSize = MAX(ZSTDMT_SECTION_SIZE_MIN, zcs->targetSectionSize); zcs->targetSectionSize = MAX(zcs->targetDictSize, zcs->targetSectionSize); DEBUGLOG(4, "Section Size : %u KB", (U32)(zcs->targetSectionSize>>10)); - zcs->marginSize = zcs->targetSectionSize >> 2; - zcs->inBuffSize = zcs->targetDictSize + zcs->targetSectionSize + zcs->marginSize; - zcs->inBuff.buffer = ZSTDMT_getBuffer(zcs->buffPool, zcs->inBuffSize); - if (zcs->inBuff.buffer.start == NULL) return ERROR(memory_allocation); - zcs->inBuff.filled = 0; + zcs->inBuffSize = zcs->targetDictSize + zcs->targetSectionSize; + ZSTDMT_setBufferSize(zcs->bufPool, MAX(zcs->inBuffSize, ZSTD_compressBound(zcs->targetSectionSize)) ); + zcs->inBuff.buffer = g_nullBuffer; zcs->dictSize = 0; zcs->doneJobID = 0; zcs->nextJobID = 0; @@ -664,8 +745,9 @@ size_t ZSTDMT_initCStream_internal(ZSTDMT_CCtx* zcs, } size_t ZSTDMT_initCStream_advanced(ZSTDMT_CCtx* mtctx, - const void* dict, size_t dictSize, - ZSTD_parameters params, unsigned long long pledgedSrcSize) + const void* dict, size_t dictSize, + ZSTD_parameters params, + unsigned long long pledgedSrcSize) { DEBUGLOG(5, "ZSTDMT_initCStream_advanced"); return ZSTDMT_initCStream_internal(mtctx, dict, dictSize, NULL, params, pledgedSrcSize); @@ -701,19 +783,8 @@ size_t ZSTDMT_initCStream(ZSTDMT_CCtx* zcs, int compressionLevel) { static size_t ZSTDMT_createCompressionJob(ZSTDMT_CCtx* zcs, size_t srcSize, unsigned endFrame) { - size_t const dstBufferCapacity = ZSTD_compressBound(srcSize); - buffer_t const dstBuffer = ZSTDMT_getBuffer(zcs->buffPool, dstBufferCapacity); - ZSTD_CCtx* const cctx = ZSTDMT_getCCtx(zcs->cctxPool); unsigned const jobID = zcs->nextJobID & zcs->jobIDMask; - if ((cctx==NULL) || (dstBuffer.start==NULL)) { - zcs->jobs[jobID].jobCompleted = 1; - zcs->nextJobID++; - ZSTDMT_waitForAllJobsCompleted(zcs); - ZSTDMT_releaseAllJobResources(zcs); - return ERROR(memory_allocation); - } - DEBUGLOG(4, "preparing job %u to compress %u bytes with %u preload ", zcs->nextJobID, (U32)srcSize, (U32)zcs->dictSize); zcs->jobs[jobID].src = zcs->inBuff.buffer; @@ -726,8 +797,9 @@ static size_t ZSTDMT_createCompressionJob(ZSTDMT_CCtx* zcs, size_t srcSize, unsi if (zcs->nextJobID) zcs->jobs[jobID].params.fParams.checksumFlag = 0; zcs->jobs[jobID].cdict = zcs->nextJobID==0 ? zcs->cdict : NULL; zcs->jobs[jobID].fullFrameSize = zcs->frameContentSize; - zcs->jobs[jobID].dstBuff = dstBuffer; - zcs->jobs[jobID].cctx = cctx; + zcs->jobs[jobID].dstBuff = g_nullBuffer; + zcs->jobs[jobID].cctxPool = zcs->cctxPool; + zcs->jobs[jobID].bufPool = zcs->bufPool; zcs->jobs[jobID].firstChunk = (zcs->nextJobID==0); zcs->jobs[jobID].lastChunk = endFrame; zcs->jobs[jobID].jobCompleted = 0; @@ -735,11 +807,13 @@ static size_t ZSTDMT_createCompressionJob(ZSTDMT_CCtx* zcs, size_t srcSize, unsi zcs->jobs[jobID].jobCompleted_mutex = &zcs->jobCompleted_mutex; zcs->jobs[jobID].jobCompleted_cond = &zcs->jobCompleted_cond; + if (zcs->params.fParams.checksumFlag) + XXH64_update(&zcs->xxhState, (const char*)zcs->inBuff.buffer.start + zcs->dictSize, srcSize); + /* get a new buffer for next input */ if (!endFrame) { size_t const newDictSize = MIN(srcSize + zcs->dictSize, zcs->targetDictSize); - DEBUGLOG(5, "ZSTDMT_createCompressionJob::endFrame = %u", endFrame); - zcs->inBuff.buffer = ZSTDMT_getBuffer(zcs->buffPool, zcs->inBuffSize); + zcs->inBuff.buffer = ZSTDMT_getBuffer(zcs->bufPool); if (zcs->inBuff.buffer.start == NULL) { /* not enough memory to allocate next input buffer */ zcs->jobs[jobID].jobCompleted = 1; zcs->nextJobID++; @@ -747,26 +821,20 @@ static size_t ZSTDMT_createCompressionJob(ZSTDMT_CCtx* zcs, size_t srcSize, unsi ZSTDMT_releaseAllJobResources(zcs); return ERROR(memory_allocation); } - DEBUGLOG(5, "inBuff currently filled to %u", (U32)zcs->inBuff.filled); zcs->inBuff.filled -= srcSize + zcs->dictSize - newDictSize; - DEBUGLOG(5, "new job : inBuff filled to %u, with %u dict and %u src", - (U32)zcs->inBuff.filled, (U32)newDictSize, - (U32)(zcs->inBuff.filled - newDictSize)); memmove(zcs->inBuff.buffer.start, (const char*)zcs->jobs[jobID].srcStart + zcs->dictSize + srcSize - newDictSize, zcs->inBuff.filled); - DEBUGLOG(5, "new inBuff pre-filled"); zcs->dictSize = newDictSize; } else { /* if (endFrame==1) */ - DEBUGLOG(5, "ZSTDMT_createCompressionJob::endFrame = %u", endFrame); zcs->inBuff.buffer = g_nullBuffer; zcs->inBuff.filled = 0; zcs->dictSize = 0; zcs->frameEnded = 1; - if (zcs->nextJobID == 0) + if (zcs->nextJobID == 0) { /* single chunk exception : checksum is calculated directly within worker thread */ zcs->params.fParams.checksumFlag = 0; - } + } } DEBUGLOG(4, "posting job %u : %u bytes (end:%u) (note : doneJob = %u=>%u)", zcs->nextJobID, @@ -804,11 +872,8 @@ static size_t ZSTDMT_flushNextJob(ZSTDMT_CCtx* zcs, ZSTD_outBuffer* output, unsi ZSTDMT_releaseAllJobResources(zcs); return job.cSize; } - ZSTDMT_releaseCCtx(zcs->cctxPool, job.cctx); - zcs->jobs[wJobID].cctx = NULL; DEBUGLOG(5, "zcs->params.fParams.checksumFlag : %u ", zcs->params.fParams.checksumFlag); if (zcs->params.fParams.checksumFlag) { - XXH64_update(&zcs->xxhState, (const char*)job.srcStart + job.dictSize, job.srcSize); if (zcs->frameEnded && (zcs->doneJobID+1 == zcs->nextJobID)) { /* write checksum at end of last section */ U32 const checksum = (U32)XXH64_digest(&zcs->xxhState); DEBUGLOG(5, "writing checksum : %08X \n", checksum); @@ -816,9 +881,6 @@ static size_t ZSTDMT_flushNextJob(ZSTDMT_CCtx* zcs, ZSTD_outBuffer* output, unsi job.cSize += 4; zcs->jobs[wJobID].cSize += 4; } } - ZSTDMT_releaseBuffer(zcs->buffPool, job.src); - zcs->jobs[wJobID].srcStart = NULL; - zcs->jobs[wJobID].src = g_nullBuffer; zcs->jobs[wJobID].jobScanned = 1; } { size_t const toWrite = MIN(job.cSize - job.dstFlushed, output->size - output->pos); @@ -828,7 +890,7 @@ static size_t ZSTDMT_flushNextJob(ZSTDMT_CCtx* zcs, ZSTD_outBuffer* output, unsi job.dstFlushed += toWrite; } if (job.dstFlushed == job.cSize) { /* output buffer fully flushed => move to next one */ - ZSTDMT_releaseBuffer(zcs->buffPool, job.dstBuff); + ZSTDMT_releaseBuffer(zcs->bufPool, job.dstBuff); zcs->jobs[wJobID].dstBuff = g_nullBuffer; zcs->jobs[wJobID].jobCompleted = 0; zcs->doneJobID++; @@ -852,18 +914,18 @@ size_t ZSTDMT_compressStream_generic(ZSTDMT_CCtx* mtctx, ZSTD_inBuffer* input, ZSTD_EndDirective endOp) { - size_t const newJobThreshold = mtctx->dictSize + mtctx->targetSectionSize + mtctx->marginSize; + size_t const newJobThreshold = mtctx->dictSize + mtctx->targetSectionSize; assert(output->pos <= output->size); assert(input->pos <= input->size); if ((mtctx->frameEnded) && (endOp==ZSTD_e_continue)) { /* current frame being ended. Only flush/end are allowed. Or start new frame with init */ return ERROR(stage_wrong); } - if (mtctx->nbThreads==1) { + if (mtctx->nbThreads==1) { /* delegate to single-thread (synchronous) */ return ZSTD_compressStream_generic(mtctx->cctxPool->cctx[0], output, input, endOp); } - /* single-pass shortcut (note : this is blocking-mode) */ + /* single-pass shortcut (note : this is synchronous-mode) */ if ( (mtctx->nextJobID==0) /* just started */ && (mtctx->inBuff.filled==0) /* nothing buffered */ && (endOp==ZSTD_e_end) /* end order */ @@ -871,24 +933,29 @@ size_t ZSTDMT_compressStream_generic(ZSTDMT_CCtx* mtctx, size_t const cSize = ZSTDMT_compress_advanced(mtctx, (char*)output->dst + output->pos, output->size - output->pos, (const char*)input->src + input->pos, input->size - input->pos, - mtctx->cdict, mtctx->params, mtctx->overlapRLog); + mtctx->cdict, mtctx->params, mtctx->overlapLog); if (ZSTD_isError(cSize)) return cSize; input->pos = input->size; output->pos += cSize; - ZSTDMT_releaseBuffer(mtctx->buffPool, mtctx->inBuff.buffer); /* was allocated in initStream */ + ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->inBuff.buffer); /* was allocated in initStream */ mtctx->allJobsCompleted = 1; mtctx->frameEnded = 1; return 0; } /* fill input buffer */ - if ((input->src) && (mtctx->inBuff.buffer.start)) { /* support NULL input */ - size_t const toLoad = MIN(input->size - input->pos, mtctx->inBuffSize - mtctx->inBuff.filled); - DEBUGLOG(2, "inBuff:%08X; inBuffSize=%u; ToCopy=%u", (U32)(size_t)mtctx->inBuff.buffer.start, (U32)mtctx->inBuffSize, (U32)toLoad); - memcpy((char*)mtctx->inBuff.buffer.start + mtctx->inBuff.filled, (const char*)input->src + input->pos, toLoad); - input->pos += toLoad; - mtctx->inBuff.filled += toLoad; - } + if (input->size > input->pos) { /* support NULL input */ + if (mtctx->inBuff.buffer.start == NULL) { + mtctx->inBuff.buffer = ZSTDMT_getBuffer(mtctx->bufPool); + if (mtctx->inBuff.buffer.start == NULL) return ERROR(memory_allocation); + mtctx->inBuff.filled = 0; + } + { size_t const toLoad = MIN(input->size - input->pos, mtctx->inBuffSize - mtctx->inBuff.filled); + DEBUGLOG(5, "inBuff:%08X; inBuffSize=%u; ToCopy=%u", (U32)(size_t)mtctx->inBuff.buffer.start, (U32)mtctx->inBuffSize, (U32)toLoad); + memcpy((char*)mtctx->inBuff.buffer.start + mtctx->inBuff.filled, (const char*)input->src + input->pos, toLoad); + input->pos += toLoad; + mtctx->inBuff.filled += toLoad; + } } if ( (mtctx->inBuff.filled >= newJobThreshold) /* filled enough : let's compress */ && (mtctx->nextJobID <= mtctx->doneJobID + mtctx->jobIDMask) ) { /* avoid overwriting job round buffer */ diff --git a/contrib/libzstd/include/zstd/compress/zstdmt_compress.h b/contrib/libzstd/include/zstd/compress/zstdmt_compress.h index fad63b6d861..0f0fc2b03fc 100644 --- a/contrib/libzstd/include/zstd/compress/zstdmt_compress.h +++ b/contrib/libzstd/include/zstd/compress/zstdmt_compress.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTDMT_COMPRESS_H @@ -15,10 +15,11 @@ #endif -/* Note : All prototypes defined in this file are labelled experimental. - * No guarantee of API continuity is provided on any of them. - * In fact, the expectation is that these prototypes will be replaced - * by ZSTD_compress_generic() API in the near future */ +/* Note : This is an internal API. + * Some methods are still exposed (ZSTDLIB_API), + * because it used to be the only way to invoke MT compression. + * Now, it's recommended to use ZSTD_compress_generic() instead. + * These methods will stop being exposed in a future version */ /* === Dependencies === */ #include /* size_t */ @@ -67,7 +68,7 @@ ZSTDLIB_API size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx, const void* src, size_t srcSize, const ZSTD_CDict* cdict, ZSTD_parameters const params, - unsigned overlapRLog); + unsigned overlapLog); ZSTDLIB_API size_t ZSTDMT_initCStream_advanced(ZSTDMT_CCtx* mtctx, const void* dict, size_t dictSize, /* dict can be released after init, a local copy is preserved within zcs */ diff --git a/contrib/libzstd/include/zstd/decompress/huf_decompress.c b/contrib/libzstd/include/zstd/decompress/huf_decompress.c index 2a1b70ea5ef..79ded96bf61 100644 --- a/contrib/libzstd/include/zstd/decompress/huf_decompress.c +++ b/contrib/libzstd/include/zstd/decompress/huf_decompress.c @@ -32,38 +32,22 @@ - Public forum : https://groups.google.com/forum/#!forum/lz4c ****************************************************************** */ -/* ************************************************************** -* Compiler specifics -****************************************************************/ -#ifdef _MSC_VER /* Visual Studio */ -# define FORCE_INLINE static __forceinline -# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ -#else -# if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ -# ifdef __GNUC__ -# define FORCE_INLINE static inline __attribute__((always_inline)) -# else -# define FORCE_INLINE static inline -# endif -# else -# define FORCE_INLINE static -# endif /* __STDC_VERSION__ */ -#endif - - /* ************************************************************** * Dependencies ****************************************************************/ #include /* memcpy, memset */ #include "bitstream.h" /* BIT_* */ +#include "compiler.h" #include "fse.h" /* header compression */ #define HUF_STATIC_LINKING_ONLY #include "huf.h" +#include "error_private.h" /* ************************************************************** * Error Management ****************************************************************/ +#define HUF_isError ERR_isError #define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ @@ -180,7 +164,7 @@ static BYTE HUF_decodeSymbolX2(BIT_DStream_t* Dstream, const HUF_DEltX2* dt, con if (MEM_64bits()) \ HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) -FORCE_INLINE size_t HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX2* const dt, const U32 dtLog) +HINT_INLINE size_t HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX2* const dt, const U32 dtLog) { BYTE* const pStart = p; @@ -639,7 +623,7 @@ static U32 HUF_decodeLastSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DE if (MEM_64bits()) \ ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog) -FORCE_INLINE size_t HUF_decodeStreamX4(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const HUF_DEltX4* const dt, const U32 dtLog) +HINT_INLINE size_t HUF_decodeStreamX4(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const HUF_DEltX4* const dt, const U32 dtLog) { BYTE* const pStart = p; @@ -917,11 +901,11 @@ static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, qu * Tells which decoder is likely to decode faster, * based on a set of pre-determined metrics. * @return : 0==HUF_decompress4X2, 1==HUF_decompress4X4 . -* Assumption : 0 < cSrcSize < dstSize <= 128 KB */ +* Assumption : 0 < cSrcSize, dstSize <= 128 KB */ U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize) { /* decoder timing evaluation */ - U32 const Q = (U32)(cSrcSize * 16 / dstSize); /* Q < 16 since dstSize > cSrcSize */ + U32 const Q = cSrcSize >= dstSize ? 15 : (U32)(cSrcSize * 16 / dstSize); /* Q < 16 */ U32 const D256 = (U32)(dstSize >> 8); U32 const DTime0 = algoTime[Q][0].tableTime + (algoTime[Q][0].decode256Time * D256); U32 DTime1 = algoTime[Q][1].tableTime + (algoTime[Q][1].decode256Time * D256); @@ -977,7 +961,7 @@ size_t HUF_decompress4X_hufOnly_wksp(HUF_DTable* dctx, void* dst, { /* validation checks */ if (dstSize == 0) return ERROR(dstSize_tooSmall); - if ((cSrcSize >= dstSize) || (cSrcSize <= 1)) return ERROR(corruption_detected); /* invalid */ + if (cSrcSize == 0) return ERROR(corruption_detected); { U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize); return algoNb ? HUF_decompress4X4_DCtx_wksp(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize): diff --git a/contrib/libzstd/include/zstd/decompress/zstd_decompress.c b/contrib/libzstd/include/zstd/decompress/zstd_decompress.c index 003d703a5eb..d2bc545e52c 100644 --- a/contrib/libzstd/include/zstd/decompress/zstd_decompress.c +++ b/contrib/libzstd/include/zstd/decompress/zstd_decompress.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -53,15 +53,6 @@ # include "zstd_legacy.h" #endif -#if defined(_MSC_VER) && !defined(_M_IA64) /* _mm_prefetch() is not defined for ia64 */ -# include /* https://msdn.microsoft.com/fr-fr/library/84szxsww(v=vs.90).aspx */ -# define ZSTD_PREFETCH(ptr) _mm_prefetch((const char*)ptr, _MM_HINT_T0) -#elif defined(__GNUC__) -# define ZSTD_PREFETCH(ptr) __builtin_prefetch(ptr, 0, 0) -#else -# define ZSTD_PREFETCH(ptr) /* disabled */ -#endif - /*-************************************* * Errors @@ -95,7 +86,7 @@ typedef struct { HUF_DTable hufTable[HUF_DTABLE_SIZE(HufLog)]; /* can accommodate HUF_decompress4X */ U32 workspace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32]; U32 rep[ZSTD_REP_NUM]; -} ZSTD_entropyTables_t; +} ZSTD_entropyDTables_t; struct ZSTD_DCtx_s { @@ -103,7 +94,7 @@ struct ZSTD_DCtx_s const FSE_DTable* MLTptr; const FSE_DTable* OFTptr; const HUF_DTable* HUFptr; - ZSTD_entropyTables_t entropy; + ZSTD_entropyDTables_t entropy; const void* previousDstEnd; /* detect continuity */ const void* base; /* start of current segment */ const void* vBase; /* virtual start of previous segment if it was just before current one */ @@ -304,15 +295,18 @@ size_t ZSTD_getFrameHeader(ZSTD_frameHeader* zfhPtr, const void* src, size_t src return ZSTD_skippableHeaderSize; /* magic number + frame length */ memset(zfhPtr, 0, sizeof(*zfhPtr)); zfhPtr->frameContentSize = MEM_readLE32((const char *)src + 4); - zfhPtr->windowSize = 0; /* windowSize==0 means a frame is skippable */ + zfhPtr->frameType = ZSTD_skippableFrame; + zfhPtr->windowSize = 0; return 0; } return ERROR(prefix_unknown); } /* ensure there is enough `srcSize` to fully read/decode frame header */ - { size_t const fhsize = ZSTD_frameHeaderSize(src, srcSize); - if (srcSize < fhsize) return fhsize; } + { size_t const fhsize = ZSTD_frameHeaderSize(src, srcSize); + if (srcSize < fhsize) return fhsize; + zfhPtr->headerSize = (U32)fhsize; + } { BYTE const fhdByte = ip[4]; size_t pos = 5; @@ -320,24 +314,23 @@ size_t ZSTD_getFrameHeader(ZSTD_frameHeader* zfhPtr, const void* src, size_t src U32 const checksumFlag = (fhdByte>>2)&1; U32 const singleSegment = (fhdByte>>5)&1; U32 const fcsID = fhdByte>>6; - U32 const windowSizeMax = 1U << ZSTD_WINDOWLOG_MAX; - U32 windowSize = 0; + U64 windowSize = 0; U32 dictID = 0; - U64 frameContentSize = 0; + U64 frameContentSize = ZSTD_CONTENTSIZE_UNKNOWN; if ((fhdByte & 0x08) != 0) - return ERROR(frameParameter_unsupported); /* reserved bits, must be zero */ + return ERROR(frameParameter_unsupported); /* reserved bits, must be zero */ + if (!singleSegment) { BYTE const wlByte = ip[pos++]; U32 const windowLog = (wlByte >> 3) + ZSTD_WINDOWLOG_ABSOLUTEMIN; if (windowLog > ZSTD_WINDOWLOG_MAX) return ERROR(frameParameter_windowTooLarge); - windowSize = (1U << windowLog); + windowSize = (1ULL << windowLog); windowSize += (windowSize >> 3) * (wlByte&7); } - switch(dictIDSizeCode) { - default: /* impossible */ + default: assert(0); /* impossible */ case 0 : break; case 1 : dictID = ip[pos]; pos++; break; case 2 : dictID = MEM_readLE16(ip+pos); pos+=2; break; @@ -345,14 +338,15 @@ size_t ZSTD_getFrameHeader(ZSTD_frameHeader* zfhPtr, const void* src, size_t src } switch(fcsID) { - default: /* impossible */ + default: assert(0); /* impossible */ case 0 : if (singleSegment) frameContentSize = ip[pos]; break; case 1 : frameContentSize = MEM_readLE16(ip+pos)+256; break; case 2 : frameContentSize = MEM_readLE32(ip+pos); break; case 3 : frameContentSize = MEM_readLE64(ip+pos); break; } - if (!windowSize) windowSize = (U32)frameContentSize; - if (windowSize > windowSizeMax) return ERROR(frameParameter_windowTooLarge); + if (singleSegment) windowSize = frameContentSize; + + zfhPtr->frameType = ZSTD_frame; zfhPtr->frameContentSize = frameContentSize; zfhPtr->windowSize = windowSize; zfhPtr->dictID = dictID; @@ -362,10 +356,10 @@ size_t ZSTD_getFrameHeader(ZSTD_frameHeader* zfhPtr, const void* src, size_t src } /** ZSTD_getFrameContentSize() : -* compatible with legacy mode -* @return : decompressed size of the single frame pointed to be `src` if known, otherwise -* - ZSTD_CONTENTSIZE_UNKNOWN if the size cannot be determined -* - ZSTD_CONTENTSIZE_ERROR if an error occurred (e.g. invalid magic number, srcSize too small) */ + * compatible with legacy mode + * @return : decompressed size of the single frame pointed to be `src` if known, otherwise + * - ZSTD_CONTENTSIZE_UNKNOWN if the size cannot be determined + * - ZSTD_CONTENTSIZE_ERROR if an error occurred (e.g. invalid magic number, srcSize too small) */ unsigned long long ZSTD_getFrameContentSize(const void *src, size_t srcSize) { #if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT >= 1) @@ -374,17 +368,14 @@ unsigned long long ZSTD_getFrameContentSize(const void *src, size_t srcSize) return ret == 0 ? ZSTD_CONTENTSIZE_UNKNOWN : ret; } #endif - { ZSTD_frameHeader fParams; - if (ZSTD_getFrameHeader(&fParams, src, srcSize) != 0) return ZSTD_CONTENTSIZE_ERROR; - if (fParams.windowSize == 0) { - /* Either skippable or empty frame, size == 0 either way */ + { ZSTD_frameHeader zfh; + if (ZSTD_getFrameHeader(&zfh, src, srcSize) != 0) + return ZSTD_CONTENTSIZE_ERROR; + if (zfh.frameType == ZSTD_skippableFrame) { return 0; - } else if (fParams.frameContentSize != 0) { - return fParams.frameContentSize; } else { - return ZSTD_CONTENTSIZE_UNKNOWN; - } - } + return zfh.frameContentSize; + } } } /** ZSTD_findDecompressedSize() : @@ -442,7 +433,8 @@ unsigned long long ZSTD_findDecompressedSize(const void* src, size_t srcSize) * compatible with legacy mode * @return : decompressed size if known, 0 otherwise note : 0 can mean any of the following : - - decompressed size is not present within frame header + - frame content is empty + - decompressed size field is not present in frame header - frame header unknown / not supported - frame header not complete (`srcSize` too small) */ unsigned long long ZSTD_getDecompressedSize(const void* src, size_t srcSize) @@ -460,19 +452,13 @@ static size_t ZSTD_decodeFrameHeader(ZSTD_DCtx* dctx, const void* src, size_t he size_t const result = ZSTD_getFrameHeader(&(dctx->fParams), src, headerSize); if (ZSTD_isError(result)) return result; /* invalid header */ if (result>0) return ERROR(srcSize_wrong); /* headerSize too small */ - if (dctx->fParams.dictID && (dctx->dictID != dctx->fParams.dictID)) return ERROR(dictionary_wrong); + if (dctx->fParams.dictID && (dctx->dictID != dctx->fParams.dictID)) + return ERROR(dictionary_wrong); if (dctx->fParams.checksumFlag) XXH64_reset(&dctx->xxhState, 0); return 0; } -typedef struct -{ - blockType_e blockType; - U32 lastBlock; - U32 origSize; -} blockProperties_t; - /*! ZSTD_getcBlockSize() : * Provides the size of compressed block from block header `src` */ size_t ZSTD_getcBlockSize(const void* src, size_t srcSize, @@ -958,7 +944,7 @@ static seq_t ZSTD_decodeSequence(seqState_t* seqState) } -FORCE_INLINE +HINT_INLINE size_t ZSTD_execSequence(BYTE* op, BYTE* const oend, seq_t sequence, const BYTE** litPtr, const BYTE* const litLimit, @@ -1102,7 +1088,7 @@ static size_t ZSTD_decompressSequences( } -FORCE_INLINE seq_t ZSTD_decodeSequenceLong_generic(seqState_t* seqState, int const longOffsets) +FORCE_INLINE_TEMPLATE seq_t ZSTD_decodeSequenceLong_generic(seqState_t* seqState, int const longOffsets) { seq_t seq; @@ -1202,7 +1188,7 @@ static seq_t ZSTD_decodeSequenceLong(seqState_t* seqState, unsigned const window } } -FORCE_INLINE +HINT_INLINE size_t ZSTD_execSequenceLong(BYTE* op, BYTE* const oend, seq_t sequence, const BYTE** litPtr, const BYTE* const litLimit, @@ -1338,7 +1324,7 @@ static size_t ZSTD_decompressSequencesLong( seq_t const sequence = ZSTD_decodeSequenceLong(&seqState, windowSize32); size_t const oneSeqSize = ZSTD_execSequenceLong(op, oend, sequences[(seqNb-ADVANCED_SEQS) & STOSEQ_MASK], &litPtr, litEnd, base, vBase, dictEnd); if (ZSTD_isError(oneSeqSize)) return oneSeqSize; - ZSTD_PREFETCH(sequence.match); + PREFETCH(sequence.match); sequences[seqNb&STOSEQ_MASK] = sequence; op += oneSeqSize; } @@ -1440,28 +1426,26 @@ size_t ZSTD_generateNxBytes(void* dst, size_t dstCapacity, BYTE byte, size_t len size_t ZSTD_findFrameCompressedSize(const void *src, size_t srcSize) { #if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT >= 1) - if (ZSTD_isLegacy(src, srcSize)) return ZSTD_findFrameCompressedSizeLegacy(src, srcSize); + if (ZSTD_isLegacy(src, srcSize)) + return ZSTD_findFrameCompressedSizeLegacy(src, srcSize); #endif - if (srcSize >= ZSTD_skippableHeaderSize && - (MEM_readLE32(src) & 0xFFFFFFF0U) == ZSTD_MAGIC_SKIPPABLE_START) { + if ( (srcSize >= ZSTD_skippableHeaderSize) + && (MEM_readLE32(src) & 0xFFFFFFF0U) == ZSTD_MAGIC_SKIPPABLE_START ) { return ZSTD_skippableHeaderSize + MEM_readLE32((const BYTE*)src + 4); } else { const BYTE* ip = (const BYTE*)src; const BYTE* const ipstart = ip; size_t remainingSize = srcSize; - ZSTD_frameHeader fParams; + ZSTD_frameHeader zfh; - size_t const headerSize = ZSTD_frameHeaderSize(ip, remainingSize); - if (ZSTD_isError(headerSize)) return headerSize; - - /* Frame Header */ - { size_t const ret = ZSTD_getFrameHeader(&fParams, ip, remainingSize); + /* Extract Frame Header */ + { size_t const ret = ZSTD_getFrameHeader(&zfh, src, srcSize); if (ZSTD_isError(ret)) return ret; if (ret > 0) return ERROR(srcSize_wrong); } - ip += headerSize; - remainingSize -= headerSize; + ip += zfh.headerSize; + remainingSize -= zfh.headerSize; /* Loop on each block */ while (1) { @@ -1469,7 +1453,8 @@ size_t ZSTD_findFrameCompressedSize(const void *src, size_t srcSize) size_t const cBlockSize = ZSTD_getcBlockSize(ip, remainingSize, &blockProperties); if (ZSTD_isError(cBlockSize)) return cBlockSize; - if (ZSTD_blockHeaderSize + cBlockSize > remainingSize) return ERROR(srcSize_wrong); + if (ZSTD_blockHeaderSize + cBlockSize > remainingSize) + return ERROR(srcSize_wrong); ip += ZSTD_blockHeaderSize + cBlockSize; remainingSize -= ZSTD_blockHeaderSize + cBlockSize; @@ -1477,7 +1462,7 @@ size_t ZSTD_findFrameCompressedSize(const void *src, size_t srcSize) if (blockProperties.lastBlock) break; } - if (fParams.checksumFlag) { /* Frame content checksum */ + if (zfh.checksumFlag) { /* Final frame content checksum */ if (remainingSize < 4) return ERROR(srcSize_wrong); ip += 4; remainingSize -= 4; @@ -1490,8 +1475,8 @@ size_t ZSTD_findFrameCompressedSize(const void *src, size_t srcSize) /*! ZSTD_decompressFrame() : * @dctx must be properly initialized */ static size_t ZSTD_decompressFrame(ZSTD_DCtx* dctx, - void* dst, size_t dstCapacity, - const void** srcPtr, size_t *srcSizePtr) + void* dst, size_t dstCapacity, + const void** srcPtr, size_t *srcSizePtr) { const BYTE* ip = (const BYTE*)(*srcPtr); BYTE* const ostart = (BYTE* const)dst; @@ -1500,13 +1485,15 @@ static size_t ZSTD_decompressFrame(ZSTD_DCtx* dctx, size_t remainingSize = *srcSizePtr; /* check */ - if (remainingSize < ZSTD_frameHeaderSize_min+ZSTD_blockHeaderSize) return ERROR(srcSize_wrong); + if (remainingSize < ZSTD_frameHeaderSize_min+ZSTD_blockHeaderSize) + return ERROR(srcSize_wrong); /* Frame Header */ { size_t const frameHeaderSize = ZSTD_frameHeaderSize(ip, ZSTD_frameHeaderSize_prefix); if (ZSTD_isError(frameHeaderSize)) return frameHeaderSize; - if (remainingSize < frameHeaderSize+ZSTD_blockHeaderSize) return ERROR(srcSize_wrong); - CHECK_F(ZSTD_decodeFrameHeader(dctx, ip, frameHeaderSize)); + if (remainingSize < frameHeaderSize+ZSTD_blockHeaderSize) + return ERROR(srcSize_wrong); + CHECK_F( ZSTD_decodeFrameHeader(dctx, ip, frameHeaderSize) ); ip += frameHeaderSize; remainingSize -= frameHeaderSize; } @@ -1538,14 +1525,15 @@ static size_t ZSTD_decompressFrame(ZSTD_DCtx* dctx, } if (ZSTD_isError(decodedSize)) return decodedSize; - if (dctx->fParams.checksumFlag) XXH64_update(&dctx->xxhState, op, decodedSize); + if (dctx->fParams.checksumFlag) + XXH64_update(&dctx->xxhState, op, decodedSize); op += decodedSize; ip += cBlockSize; remainingSize -= cBlockSize; if (blockProperties.lastBlock) break; } - if (dctx->fParams.checksumFlag) { /* Frame content checksum verification */ + if (dctx->fParams.checksumFlag) { /* Frame content checksum verification */ U32 const checkCalc = (U32)XXH64_digest(&dctx->xxhState); U32 checkRead; if (remainingSize<4) return ERROR(checksum_wrong); @@ -1567,17 +1555,13 @@ static size_t ZSTD_DDictDictSize(const ZSTD_DDict* ddict); static size_t ZSTD_decompressMultiFrame(ZSTD_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize, - const void *dict, size_t dictSize, + const void* dict, size_t dictSize, const ZSTD_DDict* ddict) { void* const dststart = dst; + assert(dict==NULL || ddict==NULL); /* either dict or ddict set, not both */ if (ddict) { - if (dict) { - /* programmer error, these two cases should be mutually exclusive */ - return ERROR(GENERIC); - } - dict = ZSTD_DDictDictContent(ddict); dictSize = ZSTD_DDictDictSize(ddict); } @@ -1590,7 +1574,7 @@ static size_t ZSTD_decompressMultiFrame(ZSTD_DCtx* dctx, size_t decodedSize; size_t const frameSize = ZSTD_findFrameCompressedSizeLegacy(src, srcSize); if (ZSTD_isError(frameSize)) return frameSize; - /* legacy support is incompatible with static dctx */ + /* legacy support is not compatible with static dctx */ if (dctx->staticSize) return ERROR(memory_allocation); decodedSize = ZSTD_decompressLegacy(dst, dstCapacity, src, frameSize, dict, dictSize); @@ -1613,16 +1597,13 @@ static size_t ZSTD_decompressMultiFrame(ZSTD_DCtx* dctx, return ERROR(srcSize_wrong); skippableSize = MEM_readLE32((const BYTE *)src + 4) + ZSTD_skippableHeaderSize; - if (srcSize < skippableSize) { - return ERROR(srcSize_wrong); - } + if (srcSize < skippableSize) return ERROR(srcSize_wrong); src = (const BYTE *)src + skippableSize; srcSize -= skippableSize; continue; - } else { - return ERROR(prefix_unknown); } + return ERROR(prefix_unknown); } if (ddict) { @@ -1638,12 +1619,11 @@ static size_t ZSTD_decompressMultiFrame(ZSTD_DCtx* dctx, { const size_t res = ZSTD_decompressFrame(dctx, dst, dstCapacity, &src, &srcSize); if (ZSTD_isError(res)) return res; - /* don't need to bounds check this, ZSTD_decompressFrame will have - * already */ + /* no need to bound check, ZSTD_decompressFrame already has */ dst = (BYTE*)dst + res; dstCapacity -= res; } - } + } /* while (srcSize >= ZSTD_frameHeaderSize_prefix) */ if (srcSize) return ERROR(srcSize_wrong); /* input not entirely consumed */ @@ -1742,7 +1722,7 @@ size_t ZSTD_decompressContinue(ZSTD_DCtx* dctx, void* dst, size_t dstCapacity, c return 0; } dctx->expected = 0; /* not necessary to copy more */ - + /* fall-through */ case ZSTDds_decodeFrameHeader: assert(src != NULL); memcpy(dctx->headerBuffer + ZSTD_frameHeaderSize_prefix, src, dctx->expected); @@ -1853,7 +1833,7 @@ static size_t ZSTD_refDictContent(ZSTD_DCtx* dctx, const void* dict, size_t dict /* ZSTD_loadEntropy() : * dict : must point at beginning of a valid zstd dictionary * @return : size of entropy tables read */ -static size_t ZSTD_loadEntropy(ZSTD_entropyTables_t* entropy, const void* const dict, size_t const dictSize) +static size_t ZSTD_loadEntropy(ZSTD_entropyDTables_t* entropy, const void* const dict, size_t const dictSize) { const BYTE* dictPtr = (const BYTE*)dict; const BYTE* const dictEnd = dictPtr + dictSize; @@ -1931,8 +1911,9 @@ static size_t ZSTD_decompress_insertDictionary(ZSTD_DCtx* dctx, const void* dict size_t ZSTD_decompressBegin_usingDict(ZSTD_DCtx* dctx, const void* dict, size_t dictSize) { - CHECK_F(ZSTD_decompressBegin(dctx)); - if (dict && dictSize) CHECK_E(ZSTD_decompress_insertDictionary(dctx, dict, dictSize), dictionary_corrupted); + CHECK_F( ZSTD_decompressBegin(dctx) ); + if (dict && dictSize) + CHECK_E(ZSTD_decompress_insertDictionary(dctx, dict, dictSize), dictionary_corrupted); return 0; } @@ -1943,7 +1924,7 @@ struct ZSTD_DDict_s { void* dictBuffer; const void* dictContent; size_t dictSize; - ZSTD_entropyTables_t entropy; + ZSTD_entropyDTables_t entropy; U32 dictID; U32 entropyPresent; ZSTD_customMem cMem; @@ -1961,7 +1942,7 @@ static size_t ZSTD_DDictDictSize(const ZSTD_DDict* ddict) size_t ZSTD_decompressBegin_usingDDict(ZSTD_DCtx* dstDCtx, const ZSTD_DDict* ddict) { - CHECK_F(ZSTD_decompressBegin(dstDCtx)); + CHECK_F( ZSTD_decompressBegin(dstDCtx) ); if (ddict) { /* support begin on NULL */ dstDCtx->dictID = ddict->dictID; dstDCtx->base = ddict->dictContent; @@ -2142,7 +2123,7 @@ unsigned ZSTD_getDictID_fromDDict(const ZSTD_DDict* ddict) * ZSTD_getFrameHeader(), which will provide a more precise error code. */ unsigned ZSTD_getDictID_fromFrame(const void* src, size_t srcSize) { - ZSTD_frameHeader zfp = { 0 , 0 , 0 , 0 }; + ZSTD_frameHeader zfp = { 0, 0, ZSTD_frame, 0, 0, 0 }; size_t const hError = ZSTD_getFrameHeader(&zfp, src, srcSize); if (ZSTD_isError(hError)) return 0; return zfp.dictID; @@ -2237,7 +2218,7 @@ size_t ZSTD_setDStreamParameter(ZSTD_DStream* zds, { switch(paramType) { - default : return ERROR(parameter_unknown); + default : return ERROR(parameter_unsupported); case DStream_p_maxWindowSize : zds->maxWindowSize = paramValue ? paramValue : (U32)(-1); break; } return 0; @@ -2254,16 +2235,19 @@ size_t ZSTD_estimateDStreamSize(size_t windowSize) size_t const blockSize = MIN(windowSize, ZSTD_BLOCKSIZE_MAX); size_t const inBuffSize = blockSize; /* no block can be larger */ size_t const outBuffSize = windowSize + blockSize + (WILDCOPY_OVERLENGTH * 2); - return sizeof(ZSTD_DStream) + ZSTD_estimateDCtxSize() + inBuffSize + outBuffSize; + return ZSTD_estimateDCtxSize() + inBuffSize + outBuffSize; } ZSTDLIB_API size_t ZSTD_estimateDStreamSize_fromFrame(const void* src, size_t srcSize) { - ZSTD_frameHeader fh; - size_t const err = ZSTD_getFrameHeader(&fh, src, srcSize); + U32 const windowSizeMax = 1U << ZSTD_WINDOWLOG_MAX; + ZSTD_frameHeader zfh; + size_t const err = ZSTD_getFrameHeader(&zfh, src, srcSize); if (ZSTD_isError(err)) return err; if (err>0) return ERROR(srcSize_wrong); - return ZSTD_estimateDStreamSize(fh.windowSize); + if (zfh.windowSize > windowSizeMax) + return ERROR(frameParameter_windowTooLarge); + return ZSTD_estimateDStreamSize((size_t)zfh.windowSize); } @@ -2314,16 +2298,14 @@ size_t ZSTD_decompressStream(ZSTD_DStream* zds, ZSTD_outBuffer* output, ZSTD_inB size_t const dictSize = zds->ddict ? zds->ddict->dictSize : 0; /* legacy support is incompatible with static dctx */ if (zds->staticSize) return ERROR(memory_allocation); - CHECK_F(ZSTD_initLegacyStream(&zds->legacyContext, zds->previousLegacyVersion, legacyVersion, - dict, dictSize)); + CHECK_F(ZSTD_initLegacyStream(&zds->legacyContext, + zds->previousLegacyVersion, legacyVersion, + dict, dictSize)); zds->legacyVersion = zds->previousLegacyVersion = legacyVersion; - return ZSTD_decompressLegacyStream(zds->legacyContext, zds->legacyVersion, output, input); - } else { - return hSize; /* error */ + return ZSTD_decompressLegacyStream(zds->legacyContext, legacyVersion, output, input); } -#else - return hSize; #endif + return hSize; /* error */ } if (hSize != 0) { /* need more input */ size_t const toLoad = hSize - zds->lhSize; /* if hSize!=0, hSize > zds->lhSize */ @@ -2374,8 +2356,8 @@ size_t ZSTD_decompressStream(ZSTD_DStream* zds, ZSTD_outBuffer* output, ZSTD_inB if (zds->fParams.windowSize > zds->maxWindowSize) return ERROR(frameParameter_windowTooLarge); /* Adapt buffer sizes to frame header instructions */ - { size_t const blockSize = MIN(zds->fParams.windowSize, ZSTD_BLOCKSIZE_MAX); - size_t const neededOutSize = zds->fParams.windowSize + blockSize + WILDCOPY_OVERLENGTH * 2; + { size_t const blockSize = (size_t)(MIN(zds->fParams.windowSize, ZSTD_BLOCKSIZE_MAX)); + size_t const neededOutSize = (size_t)(zds->fParams.windowSize + blockSize + WILDCOPY_OVERLENGTH * 2); zds->blockSize = blockSize; if ((zds->inBuffSize < blockSize) || (zds->outBuffSize < neededOutSize)) { size_t const bufferSize = blockSize + neededOutSize; @@ -2400,7 +2382,7 @@ size_t ZSTD_decompressStream(ZSTD_DStream* zds, ZSTD_outBuffer* output, ZSTD_inB zds->outBuffSize = neededOutSize; } } zds->streamStage = zdss_read; - /* pass-through */ + /* fall-through */ case zdss_read: DEBUGLOG(5, "stage zdss_read"); @@ -2425,8 +2407,7 @@ size_t ZSTD_decompressStream(ZSTD_DStream* zds, ZSTD_outBuffer* output, ZSTD_inB } } if (ip==iend) { someMoreWork = 0; break; } /* no more input */ zds->streamStage = zdss_load; - /* pass-through */ - + /* fall-through */ case zdss_load: { size_t const neededInSize = ZSTD_nextSrcSizeToDecompress(zds); size_t const toLoad = neededInSize - zds->inPos; /* should always be <= remaining space within inBuff */ @@ -2448,8 +2429,7 @@ size_t ZSTD_decompressStream(ZSTD_DStream* zds, ZSTD_outBuffer* output, ZSTD_inB zds->outEnd = zds->outStart + decodedSize; } } zds->streamStage = zdss_flush; - /* pass-through */ - + /* fall-through */ case zdss_flush: { size_t const toFlushSize = zds->outEnd - zds->outStart; size_t const flushedSize = ZSTD_limitCopy(op, oend-op, zds->outBuff + zds->outStart, toFlushSize); diff --git a/contrib/libzstd/include/zstd/deprecated/zbuff.h b/contrib/libzstd/include/zstd/deprecated/zbuff.h index f62091976c7..e6ea84ad3b4 100644 --- a/contrib/libzstd/include/zstd/deprecated/zbuff.h +++ b/contrib/libzstd/include/zstd/deprecated/zbuff.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ /* *************************************************************** diff --git a/contrib/libzstd/include/zstd/deprecated/zbuff_common.c b/contrib/libzstd/include/zstd/deprecated/zbuff_common.c index 9fff6eb2095..2de45bec170 100644 --- a/contrib/libzstd/include/zstd/deprecated/zbuff_common.c +++ b/contrib/libzstd/include/zstd/deprecated/zbuff_common.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ /*-************************************* @@ -23,4 +23,3 @@ unsigned ZBUFF_isError(size_t errorCode) { return ERR_isError(errorCode); } /*! ZBUFF_getErrorName() : * provides error code string from function result (useful for debugging) */ const char* ZBUFF_getErrorName(size_t errorCode) { return ERR_getErrorName(errorCode); } - diff --git a/contrib/libzstd/include/zstd/deprecated/zbuff_compress.c b/contrib/libzstd/include/zstd/deprecated/zbuff_compress.c index 5a37a0027ae..4444e95d8fd 100644 --- a/contrib/libzstd/include/zstd/deprecated/zbuff_compress.c +++ b/contrib/libzstd/include/zstd/deprecated/zbuff_compress.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ diff --git a/contrib/libzstd/include/zstd/deprecated/zbuff_decompress.c b/contrib/libzstd/include/zstd/deprecated/zbuff_decompress.c index d9c155e08eb..a819d7f4042 100644 --- a/contrib/libzstd/include/zstd/deprecated/zbuff_decompress.c +++ b/contrib/libzstd/include/zstd/deprecated/zbuff_decompress.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ diff --git a/contrib/libzstd/include/zstd/dictBuilder/cover.c b/contrib/libzstd/include/zstd/dictBuilder/cover.c index 06c1b9fadb7..3d445ae8b81 100644 --- a/contrib/libzstd/include/zstd/dictBuilder/cover.c +++ b/contrib/libzstd/include/zstd/dictBuilder/cover.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ /* ***************************************************************************** @@ -714,11 +714,9 @@ typedef struct COVER_best_s { * Initialize the `COVER_best_t`. */ static void COVER_best_init(COVER_best_t *best) { - if (!best) { - return; - } - pthread_mutex_init(&best->mutex, NULL); - pthread_cond_init(&best->cond, NULL); + if (best==NULL) return; /* compatible with init on NULL */ + (void)pthread_mutex_init(&best->mutex, NULL); + (void)pthread_cond_init(&best->cond, NULL); best->liveJobs = 0; best->dict = NULL; best->dictSize = 0; diff --git a/contrib/libzstd/include/zstd/dictBuilder/zdict.c b/contrib/libzstd/include/zstd/dictBuilder/zdict.c index 742586eacdd..c2871c2ccfb 100644 --- a/contrib/libzstd/include/zstd/dictBuilder/zdict.c +++ b/contrib/libzstd/include/zstd/dictBuilder/zdict.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -695,7 +695,7 @@ static size_t ZDICT_analyzeEntropy(void* dstBuffer, size_t maxDstSize, DISPLAYLEVEL(1, "Not enough memory \n"); goto _cleanup; } - if (offcodeMax>OFFCODE_MAX) { eSize = ERROR(dictionary_wrong); goto _cleanup; } /* too large dictionary */ + if (offcodeMax>OFFCODE_MAX) { eSize = ERROR(dictionaryCreation_failed); goto _cleanup; } /* too large dictionary */ for (u=0; u<256; u++) countLit[u] = 1; /* any character must be described */ for (u=0; u<=offcodeMax; u++) offcodeCount[u] = 1; for (u=0; u<=MaxML; u++) matchLengthCount[u] = 1; diff --git a/contrib/libzstd/include/zstd/dictBuilder/zdict.h b/contrib/libzstd/include/zstd/dictBuilder/zdict.h index 7bfbb351a1d..3d72a465e5e 100644 --- a/contrib/libzstd/include/zstd/dictBuilder/zdict.h +++ b/contrib/libzstd/include/zstd/dictBuilder/zdict.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef DICTBUILDER_H_001 diff --git a/contrib/libzstd/include/zstd/legacy/zstd_legacy.h b/contrib/libzstd/include/zstd/legacy/zstd_legacy.h index 3c9798f880e..1126e24669f 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_legacy.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_legacy.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTD_LEGACY_H @@ -123,6 +123,7 @@ MEM_STATIC size_t ZSTD_decompressLegacy( const void* dict,size_t dictSize) { U32 const version = ZSTD_isLegacy(src, compressedSize); + (void)dst; (void)dstCapacity; (void)dict; (void)dictSize; /* unused when ZSTD_LEGACY_SUPPORT >= 8 */ switch(version) { #if (ZSTD_LEGACY_SUPPORT <= 1) @@ -223,6 +224,7 @@ MEM_STATIC size_t ZSTD_freeLegacyStreamContext(void* legacyContext, U32 version) case 1 : case 2 : case 3 : + (void)legacyContext; return ERROR(version_unsupported); #if (ZSTD_LEGACY_SUPPORT <= 4) case 4 : return ZBUFFv04_freeDCtx((ZBUFFv04_DCtx*)legacyContext); @@ -250,6 +252,7 @@ MEM_STATIC size_t ZSTD_initLegacyStream(void** legacyContext, U32 prevVersion, U case 1 : case 2 : case 3 : + (void)dict; (void)dictSize; return 0; #if (ZSTD_LEGACY_SUPPORT <= 4) case 4 : @@ -306,6 +309,7 @@ MEM_STATIC size_t ZSTD_decompressLegacyStream(void* legacyContext, U32 version, case 1 : case 2 : case 3 : + (void)legacyContext; (void)output; (void)input; return ERROR(version_unsupported); #if (ZSTD_LEGACY_SUPPORT <= 4) case 4 : diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v01.c b/contrib/libzstd/include/zstd/legacy/zstd_v01.c index cf5354d6a9b..45f421ae6f2 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v01.c +++ b/contrib/libzstd/include/zstd/legacy/zstd_v01.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v01.h b/contrib/libzstd/include/zstd/legacy/zstd_v01.h index 13cb3acfdc3..a91c6a133ba 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v01.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_v01.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTD_V01_H_28739879432 diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v02.c b/contrib/libzstd/include/zstd/legacy/zstd_v02.c index 3cf8f477825..dc1ec0e7c67 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v02.c +++ b/contrib/libzstd/include/zstd/legacy/zstd_v02.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v02.h b/contrib/libzstd/include/zstd/legacy/zstd_v02.h index d14f0293cbe..63cb3b8d578 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v02.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_v02.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTD_V02_H_4174539423 diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v03.c b/contrib/libzstd/include/zstd/legacy/zstd_v03.c index f438330a469..8257de7e6cd 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v03.c +++ b/contrib/libzstd/include/zstd/legacy/zstd_v03.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v03.h b/contrib/libzstd/include/zstd/legacy/zstd_v03.h index 07f7597bb7c..e38e0109b13 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v03.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_v03.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTD_V03_H_298734209782 diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v04.c b/contrib/libzstd/include/zstd/legacy/zstd_v04.c index 8b8e23cb09c..951561a6cc8 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v04.c +++ b/contrib/libzstd/include/zstd/legacy/zstd_v04.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -2776,7 +2776,7 @@ static size_t ZSTD_decodeFrameHeader_Part2(ZSTD_DCtx* zc, const void* src, size_ size_t result; if (srcSize != zc->headerSize) return ERROR(srcSize_wrong); result = ZSTD_getFrameParams(&(zc->params), src, srcSize); - if ((MEM_32bits()) && (zc->params.windowLog > 25)) return ERROR(frameParameter_unsupportedBy32bits); + if ((MEM_32bits()) && (zc->params.windowLog > 25)) return ERROR(frameParameter_unsupported); return result; } diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v04.h b/contrib/libzstd/include/zstd/legacy/zstd_v04.h index 1b5439d3924..a7d6623305f 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v04.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_v04.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTD_V04_H_91868324769238 diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v05.c b/contrib/libzstd/include/zstd/legacy/zstd_v05.c index e929618a3bf..4a1d4d4bd93 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v05.c +++ b/contrib/libzstd/include/zstd/legacy/zstd_v05.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -2888,7 +2888,7 @@ static size_t ZSTDv05_decodeFrameHeader_Part2(ZSTDv05_DCtx* zc, const void* src, if (srcSize != zc->headerSize) return ERROR(srcSize_wrong); result = ZSTDv05_getFrameParams(&(zc->params), src, srcSize); - if ((MEM_32bits()) && (zc->params.windowLog > 25)) return ERROR(frameParameter_unsupportedBy32bits); + if ((MEM_32bits()) && (zc->params.windowLog > 25)) return ERROR(frameParameter_unsupported); return result; } diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v05.h b/contrib/libzstd/include/zstd/legacy/zstd_v05.h index 8ce662fd9fc..a333bd127bd 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v05.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_v05.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTDv05_H diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v06.c b/contrib/libzstd/include/zstd/legacy/zstd_v06.c index 26f0929da6f..a285a09016f 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v06.c +++ b/contrib/libzstd/include/zstd/legacy/zstd_v06.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ @@ -3084,7 +3084,7 @@ size_t ZSTDv06_getFrameParams(ZSTDv06_frameParams* fparamsPtr, const void* src, static size_t ZSTDv06_decodeFrameHeader(ZSTDv06_DCtx* zc, const void* src, size_t srcSize) { size_t const result = ZSTDv06_getFrameParams(&(zc->fParams), src, srcSize); - if ((MEM_32bits()) && (zc->fParams.windowLog > 25)) return ERROR(frameParameter_unsupportedBy32bits); + if ((MEM_32bits()) && (zc->fParams.windowLog > 25)) return ERROR(frameParameter_unsupported); return result; } diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v06.h b/contrib/libzstd/include/zstd/legacy/zstd_v06.h index 10c9c772597..ee043a17909 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v06.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_v06.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTDv06_H diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v07.c b/contrib/libzstd/include/zstd/legacy/zstd_v07.c index 6669b71cea4..ad392e90b61 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v07.c +++ b/contrib/libzstd/include/zstd/legacy/zstd_v07.c @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ diff --git a/contrib/libzstd/include/zstd/legacy/zstd_v07.h b/contrib/libzstd/include/zstd/legacy/zstd_v07.h index cc95c661bc9..68d18e9636b 100644 --- a/contrib/libzstd/include/zstd/legacy/zstd_v07.h +++ b/contrib/libzstd/include/zstd/legacy/zstd_v07.h @@ -1,10 +1,10 @@ -/** +/* * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ #ifndef ZSTDv07_H_235446 diff --git a/contrib/libzstd/include/zstd/zstd.h b/contrib/libzstd/include/zstd/zstd.h index 58e9a5606db..13b4563fd69 100644 --- a/contrib/libzstd/include/zstd/zstd.h +++ b/contrib/libzstd/include/zstd/zstd.h @@ -2,11 +2,10 @@ * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. * All rights reserved. * - * This source code is licensed under the BSD-style license found in the - * LICENSE file in the root directory of this source tree. An additional grant - * of patent rights can be found in the PATENTS file in the same directory. + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). */ - #if defined (__cplusplus) extern "C" { #endif @@ -59,7 +58,7 @@ extern "C" { /*------ Version ------*/ #define ZSTD_VERSION_MAJOR 1 #define ZSTD_VERSION_MINOR 3 -#define ZSTD_VERSION_RELEASE 0 +#define ZSTD_VERSION_RELEASE 1 #define ZSTD_VERSION_NUMBER (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE) ZSTDLIB_API unsigned ZSTD_versionNumber(void); /**< useful to check dll version */ @@ -425,13 +424,6 @@ typedef struct { ZSTD_frameParameters fParams; } ZSTD_parameters; -typedef struct { - unsigned long long frameContentSize; - size_t windowSize; - unsigned dictID; - unsigned checksumFlag; -} ZSTD_frameHeader; - /*= Custom memory allocation functions */ typedef void* (*ZSTD_allocFunction) (void* opaque, size_t size); typedef void (*ZSTD_freeFunction) (void* opaque, void* address); @@ -809,7 +801,6 @@ ZSTDLIB_API size_t ZSTD_compressContinue(ZSTD_CCtx* cctx, void* dst, size_t dstC ZSTDLIB_API size_t ZSTD_compressEnd(ZSTD_CCtx* cctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize); - /*- Buffer-less streaming decompression (synchronous mode) @@ -874,6 +865,15 @@ ZSTDLIB_API size_t ZSTD_compressEnd(ZSTD_CCtx* cctx, void* dst, size_t dstCapaci */ /*===== Buffer-less streaming decompression functions =====*/ +typedef enum { ZSTD_frame, ZSTD_skippableFrame } ZSTD_frameType_e; +typedef struct { + unsigned long long frameContentSize; /* ZSTD_CONTENTSIZE_UNKNOWN means this field is not available. 0 means "empty" */ + unsigned long long windowSize; /* can be very large, up to <= frameContentSize */ + ZSTD_frameType_e frameType; /* if == ZSTD_skippableFrame, frameContentSize is the size of skippable content */ + unsigned headerSize; + unsigned dictID; + unsigned checksumFlag; +} ZSTD_frameHeader; ZSTDLIB_API size_t ZSTD_getFrameHeader(ZSTD_frameHeader* zfhPtr, const void* src, size_t srcSize); /**< doesn't consume input */ ZSTDLIB_API size_t ZSTD_decompressBegin(ZSTD_DCtx* dctx); ZSTDLIB_API size_t ZSTD_decompressBegin_usingDict(ZSTD_DCtx* dctx, const void* dict, size_t dictSize); @@ -953,7 +953,9 @@ typedef enum { * Special: value 0 means "do not change strategy". */ /* frame parameters */ - ZSTD_p_contentSizeFlag=200, /* Content size is written into frame header _whenever known_ (default:1) */ + ZSTD_p_contentSizeFlag=200, /* Content size is written into frame header _whenever known_ (default:1) + * note that content size must be known at the beginning, + * it is sent using ZSTD_CCtx_setPledgedSrcSize() */ ZSTD_p_checksumFlag, /* A 32-bits checksum of content is written at end of frame (default:0) */ ZSTD_p_dictIDFlag, /* When applicable, dictID of dictionary is provided in frame header (default:1) */ From 88e2dd9d74a74b291f2e221159803e1321ca9ac5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Sep 2017 06:47:18 +0300 Subject: [PATCH 039/108] Added missing file [#CLICKHOUSE-3282]. --- .../libzstd/include/zstd/common/compiler.h | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) create mode 100644 contrib/libzstd/include/zstd/common/compiler.h diff --git a/contrib/libzstd/include/zstd/common/compiler.h b/contrib/libzstd/include/zstd/common/compiler.h new file mode 100644 index 00000000000..d7225c443e9 --- /dev/null +++ b/contrib/libzstd/include/zstd/common/compiler.h @@ -0,0 +1,85 @@ +/* + * Copyright (c) 2016-present, Yann Collet, Facebook, Inc. + * All rights reserved. + * + * This source code is licensed under both the BSD-style license (found in the + * LICENSE file in the root directory of this source tree) and the GPLv2 (found + * in the COPYING file in the root directory of this source tree). + */ + +#ifndef ZSTD_COMPILER_H +#define ZSTD_COMPILER_H + +/*-******************************************************* +* Compiler specifics +*********************************************************/ +/* force inlining */ +#if defined (__GNUC__) || defined(__cplusplus) || defined(__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ +# define INLINE_KEYWORD inline +#else +# define INLINE_KEYWORD +#endif + +#if defined(__GNUC__) +# define FORCE_INLINE_ATTR __attribute__((always_inline)) +#elif defined(_MSC_VER) +# define FORCE_INLINE_ATTR __forceinline +#else +# define FORCE_INLINE_ATTR +#endif + +/** + * FORCE_INLINE_TEMPLATE is used to define C "templates", which take constant + * parameters. They must be inlined for the compiler to elimininate the constant + * branches. + */ +#define FORCE_INLINE_TEMPLATE static INLINE_KEYWORD FORCE_INLINE_ATTR +/** + * HINT_INLINE is used to help the compiler generate better code. It is *not* + * used for "templates", so it can be tweaked based on the compilers + * performance. + * + * gcc-4.8 and gcc-4.9 have been shown to benefit from leaving off the + * always_inline attribute. + * + * clang up to 5.0.0 (trunk) benefit tremendously from the always_inline + * attribute. + */ +#if !defined(__clang__) && defined(__GNUC__) && __GNUC__ >= 4 && __GNUC_MINOR__ >= 8 && __GNUC__ < 5 +# define HINT_INLINE static INLINE_KEYWORD +#else +# define HINT_INLINE static INLINE_KEYWORD FORCE_INLINE_ATTR +#endif + +/* force no inlining */ +#ifdef _MSC_VER +# define FORCE_NOINLINE static __declspec(noinline) +#else +# ifdef __GNUC__ +# define FORCE_NOINLINE static __attribute__((__noinline__)) +# else +# define FORCE_NOINLINE static +# endif +#endif + +/* prefetch */ +#if defined(_MSC_VER) && (defined(_M_X64) || defined(_M_I86)) /* _mm_prefetch() is not defined outside of x86/x64 */ +# include /* https://msdn.microsoft.com/fr-fr/library/84szxsww(v=vs.90).aspx */ +# define PREFETCH(ptr) _mm_prefetch((const char*)ptr, _MM_HINT_T0) +#elif defined(__GNUC__) +# define PREFETCH(ptr) __builtin_prefetch(ptr, 0, 0) +#else +# define PREFETCH(ptr) /* disabled */ +#endif + +/* disable warnings */ +#ifdef _MSC_VER /* Visual Studio */ +# include /* For Visual 2005 */ +# pragma warning(disable : 4100) /* disable: C4100: unreferenced formal parameter */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4204) /* disable: C4204: non-constant aggregate initializer */ +# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ +# pragma warning(disable : 4324) /* disable: C4324: padded structure */ +#endif + +#endif /* ZSTD_COMPILER_H */ From 657e333dba99106dae8790dfa42abd9e81b01c0a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Sep 2017 07:02:29 +0300 Subject: [PATCH 040/108] Added "Uptime" asynchronous metric [#CLICKHOUSE-3282]. --- dbms/src/Interpreters/AsynchronousMetrics.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index 12a9c6dc8ca..823e6eab2e3 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -126,6 +126,8 @@ void AsynchronousMetrics::update() } } + set("Uptime", context.getUptimeSeconds()); + { auto databases = context.getDatabases(); From 4017622ffb4cfd3a7a870a6c19a6d388459d3e96 Mon Sep 17 00:00:00 2001 From: Vadim Skipin Date: Thu, 7 Sep 2017 17:38:35 +0300 Subject: [PATCH 041/108] CLICKHOUSE-3289: Do not use Application singleton --- dbms/src/Common/isLocalAddress.cpp | 3 +-- dbms/src/Common/isLocalAddress.h | 2 +- .../ClickHouseDictionarySource.cpp | 2 +- dbms/src/Functions/FunctionsMiscellaneous.cpp | 2 +- dbms/src/Interpreters/Cluster.cpp | 24 +++++++++---------- dbms/src/Interpreters/Cluster.h | 4 ++-- dbms/src/Interpreters/DDLWorker.cpp | 8 +++---- .../TableFunctions/TableFunctionRemote.cpp | 2 +- 8 files changed, 23 insertions(+), 24 deletions(-) diff --git a/dbms/src/Common/isLocalAddress.cpp b/dbms/src/Common/isLocalAddress.cpp index a1b24f2e0ce..d78ac869d5f 100644 --- a/dbms/src/Common/isLocalAddress.cpp +++ b/dbms/src/Common/isLocalAddress.cpp @@ -9,9 +9,8 @@ namespace DB { -bool isLocalAddress(const Poco::Net::SocketAddress & address) +bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_port) { - const UInt16 clickhouse_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0); static auto interfaces = Poco::Net::NetworkInterface::list(); if (clickhouse_port == address.port()) diff --git a/dbms/src/Common/isLocalAddress.h b/dbms/src/Common/isLocalAddress.h index d63b42dbb97..6e15490eeed 100644 --- a/dbms/src/Common/isLocalAddress.h +++ b/dbms/src/Common/isLocalAddress.h @@ -20,6 +20,6 @@ namespace DB * - only the first address is taken for each network interface; * - the routing rules that affect which network interface we go to the specified address are not checked. */ - bool isLocalAddress(const Poco::Net::SocketAddress & address); + bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_port); } diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index 894d3207094..535d09586c6 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -45,7 +45,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( where{config.getString(config_prefix + ".where", "")}, query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks}, sample_block{sample_block}, context(context), - is_local{isLocalAddress({ host, port })}, + is_local{isLocalAddress({ host, port }, config.getInt("tcp_port", 0))}, pool{is_local ? nullptr : createPool(host, port, db, user, password)}, load_all_query{query_builder.composeLoadAllQuery()} {} diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index 7ff48b85bb9..c18bbe9dc93 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1709,7 +1709,7 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & else { std::vector> host_names = {{ host_name }}; - auto cluster = std::make_shared(global_context.getSettings(), host_names, !user_name.empty() ? user_name : "default", password); + auto cluster = std::make_shared(global_context.getSettings(), host_names, !user_name.empty() ? user_name : "default", password, global_context.getTCPPort()); auto names_and_types_list = std::make_shared(getStructureOfRemoteTable(*cluster, database_name, table_name, global_context)); const auto & names = names_and_types_list->getNames(); has_column = std::find(names.begin(), names.end(), column_name) != names.end(); diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index d1196cf7d4e..304a0dd32d6 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -29,7 +29,7 @@ namespace /// Default shard weight. static constexpr UInt32 default_weight = 1; -inline bool isLocal(const Cluster::Address & address) +inline bool isLocal(const Cluster::Address & address, UInt16 clickhouse_port) { /// If there is replica, for which: /// - its port is the same that the server is listening; @@ -41,7 +41,7 @@ inline bool isLocal(const Cluster::Address & address) /// Also, replica is considered non-local, if it has default database set /// (only reason is to avoid query rewrite). - return address.default_database.empty() && isLocalAddress(address.resolved_address); + return address.default_database.empty() && isLocalAddress(address.resolved_address, clickhouse_port); } @@ -61,23 +61,23 @@ Poco::Net::SocketAddress resolveSocketAddress(const String & host_and_port) Cluster::Address::Address(Poco::Util::AbstractConfiguration & config, const String & config_prefix) { + UInt16 clickhouse_port = config.getInt("tcp_port", 0); + host_name = config.getString(config_prefix + ".host"); port = static_cast(config.getInt(config_prefix + ".port")); resolved_address = resolveSocketAddress(host_name, port); user = config.getString(config_prefix + ".user", "default"); password = config.getString(config_prefix + ".password", ""); default_database = config.getString(config_prefix + ".default_database", ""); - is_local = isLocal(*this); + is_local = isLocal(*this, clickhouse_port); } -Cluster::Address::Address(const String & host_port_, const String & user_, const String & password_) +Cluster::Address::Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port) : user(user_), password(password_) { - UInt16 default_port = static_cast(Poco::Util::Application::instance().config().getInt("tcp_port", 0)); - /// It's like that 'host_port_' string contains port. If condition is met, it doesn't necessarily mean that port exists (example: [::]). - if ((nullptr != strchr(host_port_.c_str(), ':')) || !default_port) + if ((nullptr != strchr(host_port_.c_str(), ':')) || !clickhouse_port) { resolved_address = resolveSocketAddress(host_port_); host_name = host_port_.substr(0, host_port_.find(':')); @@ -85,11 +85,11 @@ Cluster::Address::Address(const String & host_port_, const String & user_, const } else { - resolved_address = resolveSocketAddress(host_port_, default_port); + resolved_address = resolveSocketAddress(host_port_, clickhouse_port); host_name = host_port_; - port = default_port; + port = clickhouse_port; } - is_local = isLocal(*this); + is_local = isLocal(*this, clickhouse_port); } @@ -333,7 +333,7 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se Cluster::Cluster(const Settings & settings, const std::vector> & names, - const String & username, const String & password) + const String & username, const String & password, UInt16 clickhouse_port) { UInt32 current_shard_num = 1; @@ -341,7 +341,7 @@ Cluster::Cluster(const Settings & settings, const std::vector> & names, - const String & username, const String & password); + const String & username, const String & password, UInt16 clickhouse_port); Cluster(const Cluster &) = delete; Cluster & operator=(const Cluster &) = delete; @@ -59,7 +59,7 @@ public: Address() = default; Address(Poco::Util::AbstractConfiguration & config, const String & config_prefix); - Address(const String & host_port_, const String & user_, const String & password_); + Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port); /// Returns 'escaped_host_name:port' String toString() const; diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 481e7045bbe..73dd4a9e686 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -82,11 +82,11 @@ struct HostID return host_name + ":" + DB::toString(port); } - bool isLocalAddress() const + bool isLocalAddress(UInt16 clickhouse_port) const { try { - return DB::isLocalAddress(Poco::Net::SocketAddress(DNSCache::instance().resolveHost(host_name), port)); + return DB::isLocalAddress(Poco::Net::SocketAddress(host_name, port), clickhouse_port); } catch (const Poco::Exception & e) { @@ -287,7 +287,7 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason) bool host_in_hostlist = false; for (const HostID & host : task->entry.hosts) { - if (!host.isLocalAddress()) + if (!host.isLocalAddress(context.getTCPPort())) continue; if (host_in_hostlist) @@ -464,7 +464,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) { const Cluster::Address & address = shards[shard_num][replica_num]; - if (isLocalAddress(address.resolved_address)) + if (isLocalAddress(address.resolved_address, context.getTCPPort())) { if (found_via_resolving) { diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index d1240c93832..8ea29059488 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -274,7 +274,7 @@ StoragePtr TableFunctionRemote::execute(const ASTPtr & ast_function, const Conte if (names.empty()) throw Exception("Shard list is empty after parsing first argument", ErrorCodes::BAD_ARGUMENTS); - auto cluster = std::make_shared(context.getSettings(), names, username, password); + auto cluster = std::make_shared(context.getSettings(), names, username, password, context.getTCPPort()); auto res = StorageDistributed::createWithOwnCluster( getName(), From 39a4401f054e6e3d06745947b8bea3bda7273511 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Sep 2017 20:55:02 +0300 Subject: [PATCH 042/108] Update Cluster.h --- dbms/src/Interpreters/Cluster.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 6d75286636d..230d357d7e2 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -19,6 +19,8 @@ public: Cluster(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name); /// Construct a cluster by the names of shards and replicas. Local are treated as well as remote ones. + /// 'clickhouse_port' - port that this server instance listen for queries. + /// This parameter is needed only to check that some address is local (points to ourself). Cluster(const Settings & settings, const std::vector> & names, const String & username, const String & password, UInt16 clickhouse_port); From 45eda97e3914be2b0e96e2e7f375cd6d133fd135 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Sep 2017 20:55:55 +0300 Subject: [PATCH 043/108] Fixed bug in usage of ZSTD [#CLICKHOUSE-3282]. --- dbms/src/IO/CompressedReadBufferBase.cpp | 4 ++-- dbms/src/IO/CompressedWriteBuffer.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index 1c51a3e1900..26038c1a198 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -54,8 +54,8 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, size_t & size_compressed = size_compressed_without_checksum; if (method == static_cast(CompressionMethodByte::LZ4) || - method == static_cast(CompressionMethodByte::ZSTD) || - method == static_cast(CompressionMethodByte::NONE)) + method == static_cast(CompressionMethodByte::ZSTD) || + method == static_cast(CompressionMethodByte::NONE)) { size_compressed = unalignedLoad(&own_compressed_buffer[1]); size_decompressed = unalignedLoad(&own_compressed_buffer[5]); diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/IO/CompressedWriteBuffer.cpp index 4575278b12c..25bf1b4c967 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/IO/CompressedWriteBuffer.cpp @@ -80,7 +80,7 @@ void CompressedWriteBuffer::nextImpl() size_t res = ZSTD_compress( &compressed_buffer[header_size], - compressed_buffer.size(), + compressed_buffer.size() - header_size, working_buffer.begin(), uncompressed_size, 1); From 0dcee75180920c502d4b69323caeba97f2fd2a73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Sep 2017 22:13:37 +0300 Subject: [PATCH 044/108] Supporessed useless false positive errors found by cppcheck [#CLICKHOUSE-3300]. --- CMakeLists.txt | 3 +++ dbms/src/Functions/FunctionsArray.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 5 ++--- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index fe0838b2ff5..2ec52c7f427 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -27,6 +27,9 @@ endif () cmake_policy (SET CMP0014 OLD) # Ignore warning about CMakeLists.txt in each directory cmake_policy (SET CMP0012 NEW) # Don't dereference TRUE and FALSE +# Write compile_commands.json +set(CMAKE_EXPORT_COMPILE_COMMANDS 1) + find_program (CCACHE_FOUND ccache) if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CMAKE_CXX_COMPILER MATCHES "ccache") set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "ccache") diff --git a/dbms/src/Functions/FunctionsArray.cpp b/dbms/src/Functions/FunctionsArray.cpp index 9e756922400..e66788aa6b6 100644 --- a/dbms/src/Functions/FunctionsArray.cpp +++ b/dbms/src/Functions/FunctionsArray.cpp @@ -1071,7 +1071,7 @@ DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments) void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) { /// Check nullability. - bool is_nullable; + bool is_nullable = false; const ColumnArray * col_array = nullptr; const ColumnConst * col_const_array = nullptr; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index f8e835d071d..a35052db311 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2402,7 +2402,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, LOG_DEBUG(log, "Doing ALTER"); - int new_columns_version; + int new_columns_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck. String new_columns_str; zkutil::Stat stat; @@ -2427,8 +2427,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, new_columns_str = ColumnsDescription{ new_columns, new_materialized_columns, - new_alias_columns, new_column_defaults - }.toString(); + new_alias_columns, new_column_defaults}.toString(); /// Do ALTER. getZooKeeper()->set(zookeeper_path + "/columns", new_columns_str, -1, &stat); From 8b313ab99ec6330d26d64596666725f717a29e51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 00:04:48 +0300 Subject: [PATCH 045/108] Fixed warnings found by cppcheck [#CLICKHOUSE-3300]. --- dbms/src/Columns/ColumnFixedString.cpp | 2 +- dbms/src/Columns/ColumnString.cpp | 2 +- dbms/src/Columns/ColumnsCommon.cpp | 4 +- dbms/src/Common/Exception.cpp | 2 +- dbms/src/Common/Exception.h | 15 +-- dbms/src/Common/NetException.h | 8 +- dbms/src/Common/SharedLibrary.h | 9 +- dbms/src/Common/ZooKeeper/SingleBarrier.cpp | 3 +- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 11 +- .../tests/zkutil_expiration_test.cpp | 3 +- dbms/src/Common/tests/compact_array.cpp | 2 +- dbms/src/Common/tests/int_hashes_perf.cpp | 20 ++-- dbms/src/Common/tests/lru_cache.cpp | 2 +- dbms/src/Core/tests/rvo_test.cpp | 4 +- ...ggregatedMemoryEfficientBlockInputStream.h | 2 +- .../SummingSortedBlockInputStream.cpp | 16 +-- dbms/src/DataTypes/DataTypeArray.cpp | 2 +- dbms/src/DataTypes/DataTypeNumberBase.cpp | 6 +- dbms/src/DataTypes/tests/data_type_string.cpp | 10 +- dbms/src/Databases/DatabaseCloud.cpp | 4 +- .../Dictionaries/LibraryDictionarySource.cpp | 21 +++- .../Dictionaries/LibraryDictionarySource.h | 2 + .../Conditional/StringArrayEvaluator.cpp | 6 +- dbms/src/Functions/FunctionsArray.cpp | 2 +- dbms/src/Functions/FunctionsCharset.cpp | 2 +- dbms/src/Functions/FunctionsMiscellaneous.cpp | 4 +- .../Functions/tests/functions_arithmetic.cpp | 3 +- .../tests/logical_functions_performance.cpp | 4 +- dbms/src/IO/BufferWithOwnMemory.h | 2 +- dbms/src/IO/MemoryReadWriteBuffer.cpp | 20 ++-- dbms/src/IO/ReadHelpers.cpp | 1 + dbms/src/IO/WriteHelpers.cpp | 24 +--- dbms/src/IO/tests/compressed_buffer.cpp | 1 - dbms/src/IO/tests/read_buffer_aio.cpp | 4 +- dbms/src/IO/tests/write_buffer_aio.cpp | 4 +- dbms/src/Interpreters/Aggregator.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 2 +- dbms/src/Interpreters/DDLWorker.cpp | 2 +- .../src/Interpreters/ExternalDictionaries.cpp | 9 +- dbms/src/Interpreters/ProcessList.cpp | 2 +- dbms/src/Interpreters/Users.cpp | 4 +- dbms/src/Interpreters/sortBlock.cpp | 4 +- .../Interpreters/tests/hash_map_string.cpp | 2 +- .../tests/hash_map_string_small.cpp | 2 +- dbms/src/Parsers/ASTSelectQuery.cpp | 2 +- dbms/src/Parsers/ParserCase.cpp | 4 +- dbms/src/Server/InterserverIOHTTPHandler.h | 2 +- dbms/src/Server/PerformanceTest.cpp | 9 +- dbms/src/Server/ReplicasStatusHandler.h | 2 +- dbms/src/Server/RootRequestHandler.h | 2 +- dbms/src/Server/StatusFile.h | 2 +- dbms/src/Server/TCPHandlerFactory.h | 2 +- .../MergeTree/MergeTreeBaseBlockInputStream.h | 1 - .../MergeTree/MergeTreeBlockInputStream.cpp | 2 - .../MergeTree/MergeTreeBlockInputStream.h | 2 + .../MergeTreeThreadBlockInputStream.cpp | 2 - dbms/src/Storages/MergeTree/PKCondition.cpp | 2 +- .../Storages/MergeTree/ReshardingWorker.cpp | 4 +- .../MergeTree/SimpleMergeSelector.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 106 +++++++++--------- dbms/src/Storages/StorageCloud.cpp | 2 + dbms/src/Storages/StorageFile.cpp | 2 +- dbms/src/Storages/StorageLog.cpp | 2 +- dbms/src/Storages/StorageMemory.cpp | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 2 +- dbms/src/Storages/StorageStripeLog.cpp | 2 +- dbms/src/Storages/StorageTinyLog.cpp | 2 +- dbms/src/Storages/StorageTrivialBuffer.cpp | 2 +- .../dictionary_library/dictionary_library.cpp | 5 +- libs/libdaemon/src/BaseDaemon.cpp | 8 +- libs/libmysqlxx/src/Pool.cpp | 16 +-- utils/iotest/iotest.cpp | 24 +--- utils/iotest/iotest_aio.cpp | 44 +------- utils/test-data-generator/MarkovModel.h | 2 +- 74 files changed, 219 insertions(+), 298 deletions(-) diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 7e76195980e..946763f3f7a 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -102,7 +102,7 @@ template struct ColumnFixedString::less { const ColumnFixedString & parent; - less(const ColumnFixedString & parent_) : parent(parent_) {} + explicit less(const ColumnFixedString & parent_) : parent(parent_) {} bool operator()(size_t lhs, size_t rhs) const { /// TODO: memcmp slows down. diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index cfe0097f189..17fb9b53667 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -163,7 +163,7 @@ template struct ColumnString::less { const ColumnString & parent; - less(const ColumnString & parent_) : parent(parent_) {} + explicit less(const ColumnString & parent_) : parent(parent_) {} bool operator()(size_t lhs, size_t rhs) const { size_t left_len = parent.sizeAt(lhs); diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index 441a4eb1acd..4812e5ac24e 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -103,7 +103,7 @@ namespace IColumn::Offsets_t & res_offsets; IColumn::Offset_t current_src_offset = 0; - ResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) : res_offsets(*res_offsets_) {} + explicit ResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) : res_offsets(*res_offsets_) {} void reserve(ssize_t result_size_hint, size_t src_size) { @@ -147,7 +147,7 @@ namespace struct NoResultOffsetsBuilder { - NoResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) {} + explicit NoResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) {} void reserve(ssize_t result_size_hint, size_t src_size) {} void insertOne(size_t array_size) {} diff --git a/dbms/src/Common/Exception.cpp b/dbms/src/Common/Exception.cpp index d1176f73d07..8a512f48f89 100644 --- a/dbms/src/Common/Exception.cpp +++ b/dbms/src/Common/Exception.cpp @@ -30,7 +30,6 @@ void throwFromErrno(const std::string & s, int code, int e) const size_t buf_size = 128; char buf[buf_size]; #ifndef _GNU_SOURCE - const char * unknown_message = "Unknown error "; int rc = strerror_r(e, buf, buf_size); #ifdef __APPLE__ if (rc != 0 && rc != EINVAL) @@ -40,6 +39,7 @@ void throwFromErrno(const std::string & s, int code, int e) { std::string tmp = std::to_string(code); const char * code = tmp.c_str(); + const char * unknown_message = "Unknown error "; strcpy(buf, unknown_message); strcpy(buf + strlen(unknown_message), code); } diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index ecf3790d790..a978ba4a303 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -17,19 +17,12 @@ namespace DB class Exception : public Poco::Exception { public: - Exception(int code = 0) : Poco::Exception(code) {} + Exception() {} /// For deferred initialization. Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {} Exception(const std::string & msg, const std::string & arg, int code = 0) : Poco::Exception(msg, arg, code) {} Exception(const std::string & msg, const Exception & exc, int code = 0) : Poco::Exception(msg, exc, code), trace(exc.trace) {} - Exception(const Exception & exc) : Poco::Exception(exc), trace(exc.trace) {} explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText()) {} - ~Exception() throw() override {} - Exception & operator = (const Exception & exc) - { - Poco::Exception::operator=(exc); - trace = exc.trace; - return *this; - } + const char * name() const throw() override { return "DB::Exception"; } const char * className() const throw() override { return "DB::Exception"; } DB::Exception * clone() const override { return new DB::Exception(*this); } @@ -49,16 +42,12 @@ private: class ErrnoException : public Exception { public: - ErrnoException(int code = 0, int saved_errno_ = 0) - : Exception(code), saved_errno(saved_errno_) {} ErrnoException(const std::string & msg, int code = 0, int saved_errno_ = 0) : Exception(msg, code), saved_errno(saved_errno_) {} ErrnoException(const std::string & msg, const std::string & arg, int code = 0, int saved_errno_ = 0) : Exception(msg, arg, code), saved_errno(saved_errno_) {} ErrnoException(const std::string & msg, const Exception & exc, int code = 0, int saved_errno_ = 0) : Exception(msg, exc, code), saved_errno(saved_errno_) {} - ErrnoException(const ErrnoException & exc) - : Exception(exc), saved_errno(exc.saved_errno) {} int getErrno() const { return saved_errno; } diff --git a/dbms/src/Common/NetException.h b/dbms/src/Common/NetException.h index aef98bcd079..b2b12e85648 100644 --- a/dbms/src/Common/NetException.h +++ b/dbms/src/Common/NetException.h @@ -5,23 +5,21 @@ namespace DB { + class NetException : public DB::Exception { public: - explicit NetException(int code = 0) : DB::Exception(code) {} NetException(const std::string & msg, int code = 0) : DB::Exception(msg, code) {} NetException(const std::string & msg, const std::string & arg, int code = 0) : DB::Exception(msg, arg, code) {} NetException(const std::string & msg, const DB::Exception & exc, int code = 0) : DB::Exception(msg, exc, code) {} explicit NetException(const DB::Exception & exc) : DB::Exception(exc) {} explicit NetException(const Poco::Exception & exc) : DB::Exception(exc.displayText()) {} - NetException(const DB::NetException & exc) = default; - - ~NetException() throw() override {} const char * name() const throw() override { return "DB::NetException"; } const char * className() const throw() override { return "DB::NetException"; } DB::NetException * clone() const override { return new DB::NetException(*this); } void rethrow() const override { throw *this; } }; -} \ No newline at end of file + +} diff --git a/dbms/src/Common/SharedLibrary.h b/dbms/src/Common/SharedLibrary.h index 2b55b396f12..96c8f6fe025 100644 --- a/dbms/src/Common/SharedLibrary.h +++ b/dbms/src/Common/SharedLibrary.h @@ -2,16 +2,18 @@ #include #include -#include +#include + namespace DB { -/** Allows you to open a dynamic library and get a pointer to a function from it. + + /** Allows you to open a dynamic library and get a pointer to a function from it. */ class SharedLibrary : private boost::noncopyable { public: - SharedLibrary(const std::string & path); + explicit SharedLibrary(const std::string & path); ~SharedLibrary(); @@ -33,4 +35,5 @@ private: }; using SharedLibraryPtr = std::shared_ptr; + } diff --git a/dbms/src/Common/ZooKeeper/SingleBarrier.cpp b/dbms/src/Common/ZooKeeper/SingleBarrier.cpp index ba6e21e0797..54fa2c3d2d0 100644 --- a/dbms/src/Common/ZooKeeper/SingleBarrier.cpp +++ b/dbms/src/Common/ZooKeeper/SingleBarrier.cpp @@ -67,8 +67,6 @@ void SingleBarrier::enter(UInt64 timeout) { __sync_synchronize(); - bool is_first_crossing = true; - RWLock lock{get_zookeeper, path + "/lock"}; try @@ -92,6 +90,7 @@ void SingleBarrier::enter(UInt64 timeout) throw KeeperException{code}; Stopwatch watch; + bool is_first_crossing = true; while (true) { diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 29f42ac2a30..05c5c16d3d2 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -33,17 +33,14 @@ const int CreateMode::Ephemeral = ZOO_EPHEMERAL; const int CreateMode::EphemeralSequential = ZOO_EPHEMERAL | ZOO_SEQUENCE; const int CreateMode::PersistentSequential = ZOO_SEQUENCE; -void check(int32_t code, const std::string path = "") + +static void check(int32_t code, const std::string & path) { if (code != ZOK) - { - if (path.size()) - throw KeeperException(code, path); - else - throw KeeperException(code); - } + throw KeeperException(code, path); } + struct WatchContext { /// ZooKeeper instance exists for the entire WatchContext lifetime. diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp index 4ee2b147447..6855e63ce3c 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp @@ -39,8 +39,7 @@ int main(int argc, char ** argv) int code; try { - code = zk.tryMulti(ops);std::string unused; - //code = zk.tryCreate("/test", "", zkutil::CreateMode::Persistent, unused); + code = zk.tryMulti(ops); } catch (zkutil::KeeperException & e) { diff --git a/dbms/src/Common/tests/compact_array.cpp b/dbms/src/Common/tests/compact_array.cpp index d7c09850ec3..e83496514a1 100644 --- a/dbms/src/Common/tests/compact_array.cpp +++ b/dbms/src/Common/tests/compact_array.cpp @@ -54,7 +54,7 @@ struct Test wb.write(reinterpret_cast(&store), sizeof(store)); const unsigned char * p = reinterpret_cast(&store); for (size_t i = 0; i < sizeof(store); ++i) - ++p; + ++*p; } { diff --git a/dbms/src/Common/tests/int_hashes_perf.cpp b/dbms/src/Common/tests/int_hashes_perf.cpp index 990292e4e66..ae64c84f53c 100644 --- a/dbms/src/Common/tests/int_hashes_perf.cpp +++ b/dbms/src/Common/tests/int_hashes_perf.cpp @@ -306,16 +306,16 @@ int main(int argc, char ** argv) setAffinity(); - if (!method || method == 0) test (n, &data[0], "0: identity"); - if (!method || method == 1) test (n, &data[0], "1: intHash32"); - if (!method || method == 2) test (n, &data[0], "2: intHash64"); - if (!method || method == 3) test (n, &data[0], "3: two rounds"); - if (!method || method == 4) test (n, &data[0], "4: two rounds and two variables"); - if (!method || method == 5) test (n, &data[0], "5: two rounds with less ops"); - if (!method || method == 6) test (n, &data[0], "6: murmur64 mixer"); - if (!method || method == 7) test (n, &data[0], "7: mulShift"); - if (!method || method == 8) test(n, &data[0], "8: tabulation"); - if (!method || method == 9) test (n, &data[0], "9: crc32"); + if (!method || method == 1) test (n, &data[0], "0: identity"); + if (!method || method == 2) test (n, &data[0], "1: intHash32"); + if (!method || method == 3) test (n, &data[0], "2: intHash64"); + if (!method || method == 4) test (n, &data[0], "3: two rounds"); + if (!method || method == 5) test (n, &data[0], "4: two rounds and two variables"); + if (!method || method == 6) test (n, &data[0], "5: two rounds with less ops"); + if (!method || method == 7) test (n, &data[0], "6: murmur64 mixer"); + if (!method || method == 8) test (n, &data[0], "7: mulShift"); + if (!method || method == 9) test(n, &data[0], "8: tabulation"); + if (!method || method == 10) test (n, &data[0], "9: crc32"); return 0; } diff --git a/dbms/src/Common/tests/lru_cache.cpp b/dbms/src/Common/tests/lru_cache.cpp index c311b88494c..90de5903673 100644 --- a/dbms/src/Common/tests/lru_cache.cpp +++ b/dbms/src/Common/tests/lru_cache.cpp @@ -46,7 +46,7 @@ void run() } } -void runTest(unsigned int num, const std::function func) +void runTest(unsigned int num, const std::function & func) { bool ok; diff --git a/dbms/src/Core/tests/rvo_test.cpp b/dbms/src/Core/tests/rvo_test.cpp index 143b511c816..aa6cc1a3a6b 100644 --- a/dbms/src/Core/tests/rvo_test.cpp +++ b/dbms/src/Core/tests/rvo_test.cpp @@ -8,7 +8,7 @@ struct C { volatile int data; - C(int n = 0) : data(n) {} + explicit C(int n = 0) : data(n) {} C(const C & x) { @@ -140,7 +140,7 @@ C f10() C f11() { - return rand() % 2 ? f1() : f2(); + return (rand() % 2) ? f1() : f2(); } C f12() diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index 17655e0edbf..1b1de6f90af 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -146,7 +146,7 @@ private: /// An event by which the main thread is telling merging threads that it is possible to process the next group of blocks. std::condition_variable have_space; - ParallelMergeData(size_t max_threads) : pool(max_threads) {} + explicit ParallelMergeData(size_t max_threads) : pool(max_threads) {} }; std::unique_ptr parallel_merge_data; diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index 2aa27c37a4d..e1610d3a68d 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -241,7 +241,7 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std: /// If it is zero, and without it the output stream will be empty, we will write it anyway. if (!current_row_is_zero || !output_is_non_empty) { - ++merged_rows; + ++merged_rows; /// Dead store (result is unused). Left for clarity. insertCurrentRow(merged_columns); } @@ -257,15 +257,15 @@ class FieldVisitorSum : public StaticVisitor private: const Field & rhs; public: - FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {} + explicit FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {} - bool operator() (UInt64 & x) const { x += get(rhs); return x != 0; } - bool operator() (Int64 & x) const { x += get(rhs); return x != 0; } - bool operator() (Float64 & x) const { x += get(rhs); return x != 0; } + bool operator() (UInt64 & x) const { x += get(rhs); return x != 0; } + bool operator() (Int64 & x) const { x += get(rhs); return x != 0; } + bool operator() (Float64 & x) const { x += get(rhs); return x != 0; } - bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); } - bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } - bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); } + bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); } + bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } + bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); } }; diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index c6609656c46..fb13f6bab1e 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -230,11 +230,11 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r IColumn & nested_column = column_array.getData(); size_t size = 0; - bool first = true; assertChar('[', istr); try { + bool first = true; while (!istr.eof() && *istr.position() != ']') { if (!first) diff --git a/dbms/src/DataTypes/DataTypeNumberBase.cpp b/dbms/src/DataTypes/DataTypeNumberBase.cpp index 6b2515c0274..b7fe9dbd14c 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.cpp +++ b/dbms/src/DataTypes/DataTypeNumberBase.cpp @@ -113,9 +113,6 @@ void DataTypeNumberBase::serializeTextJSON(const IColumn & column, size_t row template void DataTypeNumberBase::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const { - static constexpr bool is_uint8 = std::is_same::value; - static constexpr bool is_int8 = std::is_same::value; - bool has_quote = false; if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without. { @@ -135,6 +132,9 @@ void DataTypeNumberBase::deserializeTextJSON(IColumn & column, ReadBuffer & i } else { + static constexpr bool is_uint8 = std::is_same::value; + static constexpr bool is_int8 = std::is_same::value; + if (is_uint8 || is_int8) { // extra conditions to parse true/false strings into 1/0 diff --git a/dbms/src/DataTypes/tests/data_type_string.cpp b/dbms/src/DataTypes/tests/data_type_string.cpp index 9da44127e3c..d5e6624a958 100644 --- a/dbms/src/DataTypes/tests/data_type_string.cpp +++ b/dbms/src/DataTypes/tests/data_type_string.cpp @@ -5,8 +5,8 @@ #include -#include -#include +#include +#include #include #include @@ -35,8 +35,7 @@ try offsets[i] = (i + 1) * size; } - std::ofstream ostr("test"); - WriteBufferFromFileDescriptor out_buf(STDOUT_FILENO); + WriteBufferFromFile out_buf("test"); stopwatch.restart(); data_type.serializeBinaryBulk(*column, out_buf, 0, 0); @@ -48,8 +47,7 @@ try { std::shared_ptr column = std::make_shared(); - std::ifstream istr("test"); - ReadBufferFromFileDescriptor in_buf(STDIN_FILENO); + ReadBufferFromFile in_buf("test"); stopwatch.restart(); data_type.deserializeBinaryBulk(*column, in_buf, n, 0); diff --git a/dbms/src/Databases/DatabaseCloud.cpp b/dbms/src/Databases/DatabaseCloud.cpp index 62c7914b0b2..8ec75a0b92a 100644 --- a/dbms/src/Databases/DatabaseCloud.cpp +++ b/dbms/src/Databases/DatabaseCloud.cpp @@ -176,7 +176,7 @@ struct TableSet using Container = std::map; Container map; - TableSet(const String & data) + explicit TableSet(const String & data) { ReadBufferFromString in(data); read(in); @@ -466,7 +466,7 @@ private: } public: - DatabaseCloudIterator(DatabasePtr database) + explicit DatabaseCloudIterator(DatabasePtr database) : owned_database(database), zookeeper(parent().context.getZooKeeper()), zookeeper_path(parent().zookeeper_path + "/tables/" + parent().name), diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index d571a71fae8..3b8d35dfc93 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -14,13 +14,12 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; } -const std::string lib_config_settings = ".settings"; class CStringsHolder { public: - using strings_type = std::vector; - CStringsHolder(strings_type strings_pass) + using Container = std::vector; + explicit CStringsHolder(const Container & strings_pass) { strings_holder = strings_pass; strings.size = strings_holder.size(); @@ -38,15 +37,22 @@ public: private: std::unique_ptr ptr_holder = nullptr; - strings_type strings_holder; + Container strings_holder; }; + +namespace +{ + +const std::string lib_config_settings = ".settings"; + + CStringsHolder getLibSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_root) { Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(config_root, config_keys); - CStringsHolder::strings_type strings; + CStringsHolder::Container strings; for (const auto & key : config_keys) { std::string key_name = key; @@ -59,10 +65,12 @@ CStringsHolder getLibSettings(const Poco::Util::AbstractConfiguration & config, return CStringsHolder(strings); } + bool dataToBlock(const void * data, Block & block) { if (!data) return true; + auto columns_received = static_cast(data); std::vector columns(block.columns()); for (const auto i : ext::range(0, columns.size())) @@ -82,6 +90,9 @@ bool dataToBlock(const void * data, Block & block) return false; } +} + + LibraryDictionarySource::LibraryDictionarySource(const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.h b/dbms/src/Dictionaries/LibraryDictionarySource.h index fdde72ab7cc..4835ba02afe 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.h +++ b/dbms/src/Dictionaries/LibraryDictionarySource.h @@ -7,6 +7,7 @@ #include + namespace Poco { class Logger; @@ -15,6 +16,7 @@ class Logger; namespace DB { + class CStringsHolder; /// Allows loading dictionaries from dynamic libraries (.so) diff --git a/dbms/src/Functions/Conditional/StringArrayEvaluator.cpp b/dbms/src/Functions/Conditional/StringArrayEvaluator.cpp index b01f19e6a21..2977914897d 100644 --- a/dbms/src/Functions/Conditional/StringArrayEvaluator.cpp +++ b/dbms/src/Functions/Conditional/StringArrayEvaluator.cpp @@ -44,7 +44,7 @@ using VarCallback = std::function(arg) && !checkDataType(arg) && - !checkDataType(arg) & + !checkDataType(arg) && !checkDataType(arg)) { throw Exception{ diff --git a/dbms/src/Functions/FunctionsCharset.cpp b/dbms/src/Functions/FunctionsCharset.cpp index dc0efa30a07..2642632afbe 100644 --- a/dbms/src/Functions/FunctionsCharset.cpp +++ b/dbms/src/Functions/FunctionsCharset.cpp @@ -44,7 +44,7 @@ private: { UConverter * impl; - Converter(const String & charset) + explicit Converter(const String & charset) { UErrorCode status = U_ZERO_ERROR; impl = ucnv_open(charset.data(), &status); diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index c18bbe9dc93..7772c39394c 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1280,7 +1280,7 @@ public: return std::make_shared(context.getUptimeSeconds()); } - FunctionUptime(time_t uptime_) : uptime(uptime_) + explicit FunctionUptime(time_t uptime_) : uptime(uptime_) { } @@ -1598,7 +1598,7 @@ public: return std::make_shared(context.getGlobalContext()); } - FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_) + explicit FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_) { } diff --git a/dbms/src/Functions/tests/functions_arithmetic.cpp b/dbms/src/Functions/tests/functions_arithmetic.cpp index 189bef0a39b..1798ae7a47f 100644 --- a/dbms/src/Functions/tests/functions_arithmetic.cpp +++ b/dbms/src/Functions/tests/functions_arithmetic.cpp @@ -49,8 +49,6 @@ try arg_nums.push_back(0); arg_nums.push_back(1); - size_t res_num = 2; - DataTypePtr res_type = f.getReturnType(arg_types); ColumnWithTypeAndName descr_res; @@ -61,6 +59,7 @@ try Stopwatch stopwatch; stopwatch.start(); + size_t res_num = 2; f.execute(block, arg_nums, res_num); stopwatch.stop(); diff --git a/dbms/src/Functions/tests/logical_functions_performance.cpp b/dbms/src/Functions/tests/logical_functions_performance.cpp index 09cf1784a3c..a2d76416a5a 100644 --- a/dbms/src/Functions/tests/logical_functions_performance.cpp +++ b/dbms/src/Functions/tests/logical_functions_performance.cpp @@ -75,7 +75,7 @@ struct AssociativeOperationImpl AssociativeOperationImpl continuation; /// Remembers the last N columns from in. - AssociativeOperationImpl(UInt8ColumnPtrs & in) + explicit AssociativeOperationImpl(UInt8ColumnPtrs & in) : vec(in[in.size() - N]->getData()), continuation(in) {} /// Returns a combination of values in the i-th row of all columns stored in the constructor. @@ -97,7 +97,7 @@ struct AssociativeOperationImpl const UInt8 * vec; - AssociativeOperationImpl(UInt8ColumnPtrs & in) + explicit AssociativeOperationImpl(UInt8ColumnPtrs & in) : vec(&in[in.size() - 1]->getData()[0]) {} inline UInt8 apply(size_t i) const diff --git a/dbms/src/IO/BufferWithOwnMemory.h b/dbms/src/IO/BufferWithOwnMemory.h index 89ffcf053ac..43040efd042 100644 --- a/dbms/src/IO/BufferWithOwnMemory.h +++ b/dbms/src/IO/BufferWithOwnMemory.h @@ -86,6 +86,7 @@ struct Memory : boost::noncopyable, Allocator } } +private: static size_t align(const size_t value, const size_t alignment) { if (!alignment) @@ -94,7 +95,6 @@ struct Memory : boost::noncopyable, Allocator return (value + alignment - 1) / alignment * alignment; } -private: void alloc() { if (!m_capacity) diff --git a/dbms/src/IO/MemoryReadWriteBuffer.cpp b/dbms/src/IO/MemoryReadWriteBuffer.cpp index c007d611ce3..4f849f157ab 100644 --- a/dbms/src/IO/MemoryReadWriteBuffer.cpp +++ b/dbms/src/IO/MemoryReadWriteBuffer.cpp @@ -15,12 +15,10 @@ namespace ErrorCodes class ReadBufferFromMemoryWriteBuffer : public ReadBuffer, boost::noncopyable, private Allocator { public: - - ReadBufferFromMemoryWriteBuffer(MemoryWriteBuffer && origin) - : - ReadBuffer(nullptr, 0), - chunk_list(std::move(origin.chunk_list)), - end_pos(origin.position()) + explicit ReadBufferFromMemoryWriteBuffer(MemoryWriteBuffer && origin) + : ReadBuffer(nullptr, 0), + chunk_list(std::move(origin.chunk_list)), + end_pos(origin.position()) { chunk_head = chunk_list.begin(); setChunk(); @@ -76,11 +74,16 @@ private: MemoryWriteBuffer::MemoryWriteBuffer(size_t max_total_size_, size_t initial_chunk_size_, double growth_rate_, size_t max_chunk_size_) -: WriteBuffer(nullptr, 0), max_total_size(max_total_size_), initial_chunk_size(initial_chunk_size_), max_chunk_size(max_chunk_size_), growth_rate(growth_rate_) + : WriteBuffer(nullptr, 0), + max_total_size(max_total_size_), + initial_chunk_size(initial_chunk_size_), + max_chunk_size(max_chunk_size_), + growth_rate(growth_rate_) { addChunk(); } + void MemoryWriteBuffer::nextImpl() { if (unlikely(hasPendingData())) @@ -93,6 +96,7 @@ void MemoryWriteBuffer::nextImpl() addChunk(); } + void MemoryWriteBuffer::addChunk() { size_t next_chunk_size; @@ -126,6 +130,7 @@ void MemoryWriteBuffer::addChunk() set(chunk_tail->begin(), chunk_tail->size()); } + std::shared_ptr MemoryWriteBuffer::getReadBufferImpl() { auto res = std::make_shared(std::move(*this)); @@ -137,6 +142,7 @@ std::shared_ptr MemoryWriteBuffer::getReadBufferImpl() return res; } + MemoryWriteBuffer::~MemoryWriteBuffer() { for (const auto & range : chunk_list) diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 9853baef198..6d9f66307a6 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -49,6 +49,7 @@ void parseUUID(const UInt8 * src36, std::reverse_iterator dst16) { /// If string is not like UUID - implementation specific behaviour. + /// FIXME This code looks like trash. parseHex(&src36[0], dst16 + 8, 4); parseHex(&src36[9], dst16 + 12, 2); parseHex(&src36[14], dst16 + 14, 2); diff --git a/dbms/src/IO/WriteHelpers.cpp b/dbms/src/IO/WriteHelpers.cpp index 282ca23464a..39b39ae7284 100644 --- a/dbms/src/IO/WriteHelpers.cpp +++ b/dbms/src/IO/WriteHelpers.cpp @@ -1,6 +1,7 @@ #include #include -#include +#include + namespace DB { @@ -8,30 +9,11 @@ namespace DB template void formatHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes) { - /// More optimal than lookup table by nibbles. - constexpr auto hex = - "000102030405060708090a0b0c0d0e0f" - "101112131415161718191a1b1c1d1e1f" - "202122232425262728292a2b2c2d2e2f" - "303132333435363738393a3b3c3d3e3f" - "404142434445464748494a4b4c4d4e4f" - "505152535455565758595a5b5c5d5e5f" - "606162636465666768696a6b6c6d6e6f" - "707172737475767778797a7b7c7d7e7f" - "808182838485868788898a8b8c8d8e8f" - "909192939495969798999a9b9c9d9e9f" - "a0a1a2a3a4a5a6a7a8a9aaabacadaeaf" - "b0b1b2b3b4b5b6b7b8b9babbbcbdbebf" - "c0c1c2c3c4c5c6c7c8c9cacbcccdcecf" - "d0d1d2d3d4d5d6d7d8d9dadbdcdddedf" - "e0e1e2e3e4e5e6e7e8e9eaebecedeeef" - "f0f1f2f3f4f5f6f7f8f9fafbfcfdfeff"; - size_t src_pos = 0; size_t dst_pos = 0; for (; src_pos < num_bytes; ++src_pos) { - memcpy(&dst[dst_pos], &hex[src[src_pos] * 2], 2); + writeHexByteLowercase(src[src_pos], &dst[dst_pos]); dst_pos += 2; } } diff --git a/dbms/src/IO/tests/compressed_buffer.cpp b/dbms/src/IO/tests/compressed_buffer.cpp index f663474b035..2bdb6b17be6 100644 --- a/dbms/src/IO/tests/compressed_buffer.cpp +++ b/dbms/src/IO/tests/compressed_buffer.cpp @@ -43,7 +43,6 @@ int main(int argc, char ** argv) { DB::ReadBufferFromFile buf("test1"); DB::CompressedReadBuffer compressed_buf(buf); - std::string s; stopwatch.restart(); for (size_t i = 0; i < n; ++i) diff --git a/dbms/src/IO/tests/read_buffer_aio.cpp b/dbms/src/IO/tests/read_buffer_aio.cpp index 56fbeab5d8b..1b0a36c22f9 100644 --- a/dbms/src/IO/tests/read_buffer_aio.cpp +++ b/dbms/src/IO/tests/read_buffer_aio.cpp @@ -20,7 +20,7 @@ void prepare3(std::string & filename, std::string & buf); void prepare4(std::string & filename, std::string & buf); std::string createTmpFile(); void die(const std::string & msg); -void runTest(unsigned int num, const std::function func); +void runTest(unsigned int num, const std::function & func); bool test1(const std::string & filename); bool test2(const std::string & filename, const std::string & buf); @@ -184,7 +184,7 @@ void die(const std::string & msg) ::exit(EXIT_FAILURE); } -void runTest(unsigned int num, const std::function func) +void runTest(unsigned int num, const std::function & func) { bool ok; diff --git a/dbms/src/IO/tests/write_buffer_aio.cpp b/dbms/src/IO/tests/write_buffer_aio.cpp index 5394d67a36d..a325c834022 100644 --- a/dbms/src/IO/tests/write_buffer_aio.cpp +++ b/dbms/src/IO/tests/write_buffer_aio.cpp @@ -15,7 +15,7 @@ namespace fs = boost::filesystem; void run(); void die(const std::string & msg); -void runTest(unsigned int num, const std::function func); +void runTest(unsigned int num, const std::function & func); std::string createTmpFile(); std::string generateString(size_t n); @@ -60,7 +60,7 @@ void die(const std::string & msg) ::exit(EXIT_FAILURE); } -void runTest(unsigned int num, const std::function func) +void runTest(unsigned int num, const std::function & func) { bool ok; diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 4cfd5967a1d..8b038f6aa67 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1734,7 +1734,7 @@ private: std::mutex mutex; std::condition_variable condvar; - ParallelMergeData(size_t threads) : pool(threads) {} + explicit ParallelMergeData(size_t threads) : pool(threads) {} ~ParallelMergeData() { diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index cf379b83dd9..06900b87037 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -100,7 +100,7 @@ struct ContextShared mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. String interserver_io_host; /// The host name by which this server is available for other servers. - int interserver_io_port; /// and port, + UInt16 interserver_io_port = 0; /// and port. String path; /// Path to the data directory, with a slash at the end. String tmp_path; /// The path to the temporary files that occur when processing the request. diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 73dd4a9e686..eda33e983ee 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -62,7 +62,7 @@ struct HostID HostID() = default; - HostID(const Cluster::Address & address) + explicit HostID(const Cluster::Address & address) : host_name(address.host_name), port(address.port) {} static HostID fromString(const String & host_port_str) diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 2930c12b4a9..9fcb8510404 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -320,19 +320,16 @@ void ExternalDictionaries::reloadFromConfigFile(const std::string & config_path, { failed_dict_it->second = FailedDictionaryInfo{ std::move(dict_ptr), - std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec} - }; + std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}}; } else failed_dictionaries.emplace(name, FailedDictionaryInfo{ std::move(dict_ptr), - std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec} - }); + std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}}); std::rethrow_exception(exception_ptr); } - - if (!dict_ptr->isCached()) + else if (!dict_ptr->isCached()) { const auto & lifetime = dict_ptr->getLifetime(); if (lifetime.min_sec != 0 && lifetime.max_sec != 0) diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 99b256e77e9..afa61220a6d 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -245,10 +245,10 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr BlockInputStreamPtr input_stream; BlockOutputStreamPtr output_stream; - IProfilingBlockInputStream * input_stream_casted; if (elem->tryGetQueryStreams(input_stream, output_stream)) { + IProfilingBlockInputStream * input_stream_casted; if (input_stream && (input_stream_casted = dynamic_cast(input_stream.get()))) { input_stream_casted->cancel(); diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index ae98811b95c..4f2a21ba402 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -162,7 +162,7 @@ private: } public: - HostExactPattern(const String & host_) : host(host_) {} + explicit HostExactPattern(const String & host_) : host(host_) {} bool contains(const Poco::Net::IPAddress & addr) const override { @@ -192,7 +192,7 @@ private: } public: - HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {} + explicit HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {} bool contains(const Poco::Net::IPAddress & addr) const override { diff --git a/dbms/src/Interpreters/sortBlock.cpp b/dbms/src/Interpreters/sortBlock.cpp index 58cbb75a6ec..73015222e99 100644 --- a/dbms/src/Interpreters/sortBlock.cpp +++ b/dbms/src/Interpreters/sortBlock.cpp @@ -50,7 +50,7 @@ struct PartialSortingLess { const ColumnsWithSortDescriptions & columns; - PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} + explicit PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} bool operator() (size_t a, size_t b) const { @@ -70,7 +70,7 @@ struct PartialSortingLessWithCollation { const ColumnsWithSortDescriptions & columns; - PartialSortingLessWithCollation(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} + explicit PartialSortingLessWithCollation(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} bool operator() (size_t a, size_t b) const { diff --git a/dbms/src/Interpreters/tests/hash_map_string.cpp b/dbms/src/Interpreters/tests/hash_map_string.cpp index 817d03876e4..6fbe2f084de 100644 --- a/dbms/src/Interpreters/tests/hash_map_string.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string.cpp @@ -40,7 +40,7 @@ struct CompactStringRef } CompactStringRef(const unsigned char * data_, size_t size_) : CompactStringRef(reinterpret_cast(data_), size_) {} - CompactStringRef(const std::string & s) : CompactStringRef(s.data(), s.size()) {} + explicit CompactStringRef(const std::string & s) : CompactStringRef(s.data(), s.size()) {} CompactStringRef() {} const char * data() const { return reinterpret_cast(reinterpret_cast(data_mixed) & 0x0000FFFFFFFFFFFFULL); } diff --git a/dbms/src/Interpreters/tests/hash_map_string_small.cpp b/dbms/src/Interpreters/tests/hash_map_string_small.cpp index cbd98451dd4..588fdcc0116 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_small.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_small.cpp @@ -49,7 +49,7 @@ struct SmallStringRef } SmallStringRef(const unsigned char * data_, size_t size_) : SmallStringRef(reinterpret_cast(data_), size_) {} - SmallStringRef(const std::string & s) : SmallStringRef(s.data(), s.size()) {} + explicit SmallStringRef(const std::string & s) : SmallStringRef(s.data(), s.size()) {} SmallStringRef() {} std::string toString() const { return std::string(data(), size); } diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 9c47e16bdbc..b4feb245224 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -82,7 +82,7 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n struct Arrow { Arrow() = default; - Arrow(size_t to_position_) : + explicit Arrow(size_t to_position_) : to_position(to_position_), is_selected(true) { } diff --git a/dbms/src/Parsers/ParserCase.cpp b/dbms/src/Parsers/ParserCase.cpp index dfa30bf8355..7a03a8d0307 100644 --- a/dbms/src/Parsers/ParserCase.cpp +++ b/dbms/src/Parsers/ParserCase.cpp @@ -26,10 +26,8 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return ParserFunction{}.parse(pos, node, expected); } - bool has_case_expr = false; - auto old_pos = pos; - has_case_expr = !s_when.parse(pos, node, expected); + bool has_case_expr = !s_when.parse(pos, node, expected); pos = old_pos; ASTs args; diff --git a/dbms/src/Server/InterserverIOHTTPHandler.h b/dbms/src/Server/InterserverIOHTTPHandler.h index a2132a39840..bf9fef59982 100644 --- a/dbms/src/Server/InterserverIOHTTPHandler.h +++ b/dbms/src/Server/InterserverIOHTTPHandler.h @@ -19,7 +19,7 @@ namespace DB class InterserverIOHTTPHandler : public Poco::Net::HTTPRequestHandler { public: - InterserverIOHTTPHandler(IServer & server_) + explicit InterserverIOHTTPHandler(IServer & server_) : server(server_) , log(&Poco::Logger::get("InterserverIOHTTPHandler")) { diff --git a/dbms/src/Server/PerformanceTest.cpp b/dbms/src/Server/PerformanceTest.cpp index f33f1df9c04..26e779bfac4 100644 --- a/dbms/src/Server/PerformanceTest.cpp +++ b/dbms/src/Server/PerformanceTest.cpp @@ -58,6 +58,8 @@ static String pad(size_t padding) return String(padding * 4, ' '); } + +/// NOTE The code is totally wrong. class JSONString { private: @@ -65,7 +67,7 @@ private: size_t padding; public: - JSONString(size_t padding_ = 1) : padding(padding_){}; + explicit JSONString(size_t padding_ = 1) : padding(padding_){}; void set(const String key, String value, bool wrap = true) { @@ -1041,16 +1043,13 @@ private: TestStopConditions & stop_conditions = stop_conditions_by_run[run_index]; Stats & statistics = statistics_by_run[run_index]; - size_t iteration = 0; - statistics.clear(); execute(query, statistics, stop_conditions); if (exec_type == ExecutionType::Loop) { - while (!gotSIGINT) + for (size_t iteration = 1; !gotSIGINT; ++iteration) { - ++iteration; stop_conditions.reportIterations(iteration); if (stop_conditions.areFulfilled()) break; diff --git a/dbms/src/Server/ReplicasStatusHandler.h b/dbms/src/Server/ReplicasStatusHandler.h index feb6279a6d7..02759bbc8ac 100644 --- a/dbms/src/Server/ReplicasStatusHandler.h +++ b/dbms/src/Server/ReplicasStatusHandler.h @@ -15,7 +15,7 @@ private: Context & context; public: - ReplicasStatusHandler(Context & context_); + explicit ReplicasStatusHandler(Context & context_); void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; }; diff --git a/dbms/src/Server/RootRequestHandler.h b/dbms/src/Server/RootRequestHandler.h index eed65485577..73261471bb1 100644 --- a/dbms/src/Server/RootRequestHandler.h +++ b/dbms/src/Server/RootRequestHandler.h @@ -15,7 +15,7 @@ private: IServer & server; public: - RootRequestHandler(IServer & server_) : server(server_) + explicit RootRequestHandler(IServer & server_) : server(server_) { } diff --git a/dbms/src/Server/StatusFile.h b/dbms/src/Server/StatusFile.h index 906a84b1d61..0dde3e3d16f 100644 --- a/dbms/src/Server/StatusFile.h +++ b/dbms/src/Server/StatusFile.h @@ -13,7 +13,7 @@ namespace DB class StatusFile : private boost::noncopyable { public: - StatusFile(const std::string & path_); + explicit StatusFile(const std::string & path_); ~StatusFile(); private: diff --git a/dbms/src/Server/TCPHandlerFactory.h b/dbms/src/Server/TCPHandlerFactory.h index 3abde2ef393..1f454b35a8f 100644 --- a/dbms/src/Server/TCPHandlerFactory.h +++ b/dbms/src/Server/TCPHandlerFactory.h @@ -17,7 +17,7 @@ private: Poco::Logger * log; public: - TCPHandlerFactory(IServer & server_) + explicit TCPHandlerFactory(IServer & server_) : server(server_) , log(&Logger::get("TCPHandlerFactory")) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h index ccfac96ccb2..455c0d97435 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h @@ -72,7 +72,6 @@ protected: MergeTreeReaderPtr reader; MergeTreeReaderPtr pre_reader; - Logger * log; size_t max_block_size_marks; }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp index 8fe0dc0e6ea..29506d42e27 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp @@ -45,8 +45,6 @@ MergeTreeBlockInputStream::MergeTreeBlockInputStream( check_columns(check_columns), path(data_part->getFullPath()) { - log = &Logger::get("MergeTreeBlockInputStream"); - /// Let's estimate total number of rows for progress bar. size_t total_rows = 0; for (const auto & range : all_mark_ranges) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h index 0df9ccde54c..6eb9c2e8577 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h @@ -68,6 +68,8 @@ private: bool check_columns; String path; bool is_first_task = true; + + Logger * log = &Logger::get("MergeTreeBlockInputStream"); }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp index b4df68f684a..0914f3f59f1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp @@ -36,8 +36,6 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream( } else min_marks_to_read = min_marks_to_read_; - - log = &Logger::get("MergeTreeThreadBlockInputStream"); } diff --git a/dbms/src/Storages/MergeTree/PKCondition.cpp b/dbms/src/Storages/MergeTree/PKCondition.cpp index 621b322c31a..7eb95b44aab 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.cpp +++ b/dbms/src/Storages/MergeTree/PKCondition.cpp @@ -837,9 +837,9 @@ bool PKCondition::mayBeTrueInRangeImpl(const std::vector & key_ranges, co /// The case when the column is wrapped in a chain of possibly monotonic functions. Range key_range_transformed; - bool evaluation_is_not_possible = false; if (!element.monotonic_functions_chain.empty()) { + bool evaluation_is_not_possible = false; key_range_transformed = *key_range; DataTypePtr current_type = data_types[element.key_column]; for (auto & func : element.monotonic_functions_chain) diff --git a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp index bc1f866c8b8..e0d0086c3fe 100644 --- a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp +++ b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp @@ -115,7 +115,7 @@ public: { } - Status(const std::string & str) + explicit Status(const std::string & str) { size_t pos = str.find(','); code = static_cast(std::stoull(str.substr(0, pos))); @@ -2094,8 +2094,6 @@ ReshardingWorker::StatusCode ReshardingWorker::getStatusCommon(const std::string std::string ReshardingWorker::dumpCoordinatorState(const std::string & coordinator_id) { - std::string out; - auto current_host = getFQDNOrHostName(); WriteBufferFromOwnString buf; diff --git a/dbms/src/Storages/MergeTree/SimpleMergeSelector.cpp b/dbms/src/Storages/MergeTree/SimpleMergeSelector.cpp index 093120ff2f4..041b8431271 100644 --- a/dbms/src/Storages/MergeTree/SimpleMergeSelector.cpp +++ b/dbms/src/Storages/MergeTree/SimpleMergeSelector.cpp @@ -101,7 +101,7 @@ bool allow( // std::cerr << "sum_size: " << sum_size << "\n"; /// Map size to 0..1 using logarithmic scale - double size_normalized = mapPiecewiseLinearToUnit(log(1 + sum_size), log(1 + settings.min_size_to_lower_base), log(1 + settings.max_size_to_lower_base)); + double size_normalized = mapPiecewiseLinearToUnit(log1p(sum_size), log1p(settings.min_size_to_lower_base), log1p(settings.max_size_to_lower_base)); // std::cerr << "size_normalized: " << size_normalized << "\n"; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index e8241fec0ad..1ceb2d8ac1e 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -210,7 +210,7 @@ static void appendBlock(const Block & from, Block & to) class BufferBlockOutputStream : public IBlockOutputStream { public: - BufferBlockOutputStream(StorageBuffer & storage_) : storage(storage_) {} + explicit BufferBlockOutputStream(StorageBuffer & storage_) : storage(storage_) {} void write(const Block & block) override { @@ -440,67 +440,65 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds) size_t bytes = 0; time_t time_passed = 0; + std::lock_guard lock(buffer.mutex); + + block_to_write = buffer.data.cloneEmpty(); + + rows = buffer.data.rows(); + bytes = buffer.data.bytes(); + if (buffer.first_write_time) + time_passed = current_time - buffer.first_write_time; + + if (check_thresholds) { - std::lock_guard lock(buffer.mutex); + if (!checkThresholdsImpl(rows, bytes, time_passed)) + return; + } + else + { + if (rows == 0) + return; + } - block_to_write = buffer.data.cloneEmpty(); + buffer.data.swap(block_to_write); + buffer.first_write_time = 0; - rows = buffer.data.rows(); - bytes = buffer.data.bytes(); - if (buffer.first_write_time) - time_passed = current_time - buffer.first_write_time; + CurrentMetrics::sub(CurrentMetrics::StorageBufferRows, block_to_write.rows()); + CurrentMetrics::sub(CurrentMetrics::StorageBufferBytes, block_to_write.bytes()); - if (check_thresholds) - { - if (!checkThresholdsImpl(rows, bytes, time_passed)) - return; - } - else - { - if (rows == 0) - return; - } + ProfileEvents::increment(ProfileEvents::StorageBufferFlush); + + LOG_TRACE(log, "Flushing buffer with " << rows << " rows, " << bytes << " bytes, age " << time_passed << " seconds."); + + if (no_destination) + return; + + /** For simplicity, buffer is locked during write. + * We could unlock buffer temporary, but it would lead to too much difficulties: + * - data, that is written, will not be visible for SELECTs; + * - new data could be appended to buffer, and in case of exception, we must merge it with old data, that has not been written; + * - this could lead to infinite memory growth. + */ + try + { + writeBlockToDestination(block_to_write, context.tryGetTable(destination_database, destination_table)); + } + catch (...) + { + ProfileEvents::increment(ProfileEvents::StorageBufferErrorOnFlush); + + /// Return the block to its place in the buffer. + + CurrentMetrics::add(CurrentMetrics::StorageBufferRows, block_to_write.rows()); + CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, block_to_write.bytes()); buffer.data.swap(block_to_write); - buffer.first_write_time = 0; - CurrentMetrics::sub(CurrentMetrics::StorageBufferRows, block_to_write.rows()); - CurrentMetrics::sub(CurrentMetrics::StorageBufferBytes, block_to_write.bytes()); + if (!buffer.first_write_time) + buffer.first_write_time = current_time; - ProfileEvents::increment(ProfileEvents::StorageBufferFlush); - - LOG_TRACE(log, "Flushing buffer with " << rows << " rows, " << bytes << " bytes, age " << time_passed << " seconds."); - - if (no_destination) - return; - - /** For simplicity, buffer is locked during write. - * We could unlock buffer temporary, but it would lead to too much difficulties: - * - data, that is written, will not be visible for SELECTs; - * - new data could be appended to buffer, and in case of exception, we must merge it with old data, that has not been written; - * - this could lead to infinite memory growth. - */ - try - { - writeBlockToDestination(block_to_write, context.tryGetTable(destination_database, destination_table)); - } - catch (...) - { - ProfileEvents::increment(ProfileEvents::StorageBufferErrorOnFlush); - - /// Return the block to its place in the buffer. - - CurrentMetrics::add(CurrentMetrics::StorageBufferRows, block_to_write.rows()); - CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, block_to_write.bytes()); - - buffer.data.swap(block_to_write); - - if (!buffer.first_write_time) - buffer.first_write_time = current_time; - - /// After a while, the next write attempt will happen. - throw; - } + /// After a while, the next write attempt will happen. + throw; } } diff --git a/dbms/src/Storages/StorageCloud.cpp b/dbms/src/Storages/StorageCloud.cpp index ee926e81659..41daff34dda 100644 --- a/dbms/src/Storages/StorageCloud.cpp +++ b/dbms/src/Storages/StorageCloud.cpp @@ -26,6 +26,8 @@ StorageCloud::StorageCloud( throw Exception("DatabaseCloud is detached", ErrorCodes::LOGICAL_ERROR); DatabaseCloud & db = static_cast(*owned_db); + + /// This code is unfinished. } } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 5a8c1e94f1e..febd73f83e9 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -180,7 +180,7 @@ class StorageFileBlockOutputStream : public IBlockOutputStream { public: - StorageFileBlockOutputStream(StorageFile & storage_) + explicit StorageFileBlockOutputStream(StorageFile & storage_) : storage(storage_), lock(storage.rwlock) { if (storage.use_table_fd) diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 2bcdc6e1a57..9ea51f86f31 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -133,7 +133,7 @@ private: class LogBlockOutputStream : public IBlockOutputStream { public: - LogBlockOutputStream(StorageLog & storage_) + explicit LogBlockOutputStream(StorageLog & storage_) : storage(storage_), lock(storage.rwlock), marks_stream(storage.marks_file.path(), 4096, O_APPEND | O_CREAT | O_WRONLY), diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index abd45d365c9..e3b3d53d6f8 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -61,7 +61,7 @@ private: class MemoryBlockOutputStream : public IBlockOutputStream { public: - MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {} + explicit MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {} void write(const Block & block) override { diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index a35052db311..38100cd0bd5 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -380,7 +380,7 @@ namespace { const MergeTreeData & data; - TableMetadata(const MergeTreeData & data_) + explicit TableMetadata(const MergeTreeData & data_) : data(data_) {} void write(WriteBuffer & out) const diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index fc7fe894b71..d7e4cc7fff5 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -114,7 +114,7 @@ private: class StripeLogBlockOutputStream : public IBlockOutputStream { public: - StripeLogBlockOutputStream(StorageStripeLog & storage_) + explicit StripeLogBlockOutputStream(StorageStripeLog & storage_) : storage(storage_), lock(storage.rwlock), data_out_compressed(storage.full_path() + "data.bin", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), data_out(data_out_compressed, CompressionMethod::LZ4, storage.max_compress_block_size), diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 484fe417d30..9435d45094c 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -96,7 +96,7 @@ private: class TinyLogBlockOutputStream : public IBlockOutputStream { public: - TinyLogBlockOutputStream(StorageTinyLog & storage_) + explicit TinyLogBlockOutputStream(StorageTinyLog & storage_) : storage(storage_) { for (const auto & col : storage.getColumnsList()) diff --git a/dbms/src/Storages/StorageTrivialBuffer.cpp b/dbms/src/Storages/StorageTrivialBuffer.cpp index 9dafe302766..06e693b3494 100644 --- a/dbms/src/Storages/StorageTrivialBuffer.cpp +++ b/dbms/src/Storages/StorageTrivialBuffer.cpp @@ -279,7 +279,7 @@ void StorageTrivialBuffer::flush(bool check_thresholds, bool is_called_from_back class TrivialBufferBlockOutputStream : public IBlockOutputStream { public: - TrivialBufferBlockOutputStream(StorageTrivialBuffer & buffer_) : buffer(buffer_) {} + explicit TrivialBufferBlockOutputStream(StorageTrivialBuffer & buffer_) : buffer(buffer_) {} void write(const Block & block) override { if (!block) diff --git a/dbms/tests/external_dictionaries/dictionary_library/dictionary_library.cpp b/dbms/tests/external_dictionaries/dictionary_library/dictionary_library.cpp index d96e73b53f2..fa2c333766f 100644 --- a/dbms/tests/external_dictionaries/dictionary_library/dictionary_library.cpp +++ b/dbms/tests/external_dictionaries/dictionary_library/dictionary_library.cpp @@ -23,7 +23,10 @@ void * ClickHouseDictionary_v1_loadIds( void * data_ptr, ClickHouseLibrary::CStrings * settings, ClickHouseLibrary::CStrings * columns, const struct ClickHouseLibrary::VectorUInt64 * ids) { auto ptr = static_cast(data_ptr); - std::cerr << "loadIds lib call ptr=" << data_ptr << " => " << ptr << " size=" << ids->size << "\n"; + + if (ids) + std::cerr << "loadIds lib call ptr=" << data_ptr << " => " << ptr << " size=" << ids->size << "\n"; + if (settings) { std::cerr << "settings passed: " << settings->size << "\n"; diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 56bb0826479..375f8dfa9cb 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -223,9 +223,9 @@ public: StopThread = -2 }; - SignalListener(BaseDaemon & daemon_) - : log(&Logger::get("BaseDaemon")) - , daemon(daemon_) + explicit SignalListener(BaseDaemon & daemon_) + : log(&Logger::get("BaseDaemon")) + , daemon(daemon_) { } @@ -400,7 +400,7 @@ static void terminate_handler() if (terminating) { abort(); - return; + return; /// Just for convenience. } terminating = true; diff --git a/libs/libmysqlxx/src/Pool.cpp b/libs/libmysqlxx/src/Pool.cpp index a5683659a32..b2217caf6fa 100644 --- a/libs/libmysqlxx/src/Pool.cpp +++ b/libs/libmysqlxx/src/Pool.cpp @@ -85,8 +85,8 @@ Pool::~Pool() { std::lock_guard lock(mutex); - for (Connections::iterator it = connections.begin(); it != connections.end(); it++) - delete static_cast(*it); + for (auto & connection : connections) + delete static_cast(connection); } @@ -97,10 +97,10 @@ Pool::Entry Pool::Get() initialize(); for (;;) { - for (Connections::iterator it = connections.begin(); it != connections.end(); it++) + for (auto & connection : connections) { - if ((*it)->ref_count == 0) - return Entry(*it, this); + if (connection->ref_count == 0) + return Entry(connection, this); } if (connections.size() < static_cast(max_connections)) @@ -124,11 +124,11 @@ Pool::Entry Pool::tryGet() initialize(); /// Searching for connection which was established but wasn't used. - for (Connections::iterator it = connections.begin(); it != connections.end(); ++it) + for (auto & connection : connections) { - if ((*it)->ref_count == 0) + if (connection->ref_count == 0) { - Entry res(*it, this); + Entry res(connection, this); return res.tryForceConnected() ? res : Entry(); } } diff --git a/utils/iotest/iotest.cpp b/utils/iotest/iotest.cpp index bc6f2eae67a..eb5999d02a9 100644 --- a/utils/iotest/iotest.cpp +++ b/utils/iotest/iotest.cpp @@ -17,6 +17,8 @@ #include #include +#include + #include #ifdef __APPLE__ @@ -37,29 +39,9 @@ enum Mode }; -struct AlignedBuffer -{ - int size; - char * data; - - AlignedBuffer(int size_) - { - size_t page = sysconf(_SC_PAGESIZE); - size = size_; - int rc = posix_memalign(reinterpret_cast(&data), page, (size + page - 1) / page * page); - if (data == nullptr || rc != 0) - throwFromErrno("memalign failed"); - } - - ~AlignedBuffer() - { - free(data); - } -}; - void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t count) { - AlignedBuffer direct_buf(block_size); + DB::Memory direct_buf(block_size, sysconf(_SC_PAGESIZE)); std::vector simple_buf(block_size); char * buf; diff --git a/utils/iotest/iotest_aio.cpp b/utils/iotest/iotest_aio.cpp index ecc4b04b172..eba52895b16 100644 --- a/utils/iotest/iotest_aio.cpp +++ b/utils/iotest/iotest_aio.cpp @@ -16,6 +16,8 @@ #include #include +#include + #include #if !defined(__APPLE__) && !defined(__FreeBSD__) #include @@ -61,42 +63,6 @@ enum Mode }; -struct AlignedBuffer -{ - int size = 0; - char * data = nullptr; - - AlignedBuffer() {} - - void init(int size_) - { - uninit(); - size_t page = sysconf(_SC_PAGESIZE); - size = size_; - data = static_cast(memalign(page, (size + page - 1) / page * page)); - if (!data) - throwFromErrno("memalign failed"); - } - - void uninit() - { - if (data) - free(data); - data = nullptr; - size = 0; - } - - AlignedBuffer(int size_) : size(0), data(NULL) - { - init(size_); - } - - ~AlignedBuffer() - { - uninit(); - } -}; - struct AioContext { aio_context_t ctx; @@ -119,11 +85,9 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block { AioContext ctx; - std::vector buffers(buffers_count); + std::vector buffers(buffers_count); for (size_t i = 0; i < buffers_count; ++i) - { - buffers[i].init(block_size); - } + buffers[i] = DB::Memory(block_size, sysconf(_SC_PAGESIZE)); drand48_data rand_data; timespec times; diff --git a/utils/test-data-generator/MarkovModel.h b/utils/test-data-generator/MarkovModel.h index fcd2d9bd308..19bde2cb31d 100644 --- a/utils/test-data-generator/MarkovModel.h +++ b/utils/test-data-generator/MarkovModel.h @@ -74,7 +74,7 @@ private: } public: - MarkovModel(size_t n_) : n(n_) {} + explicit MarkovModel(size_t n_) : n(n_) {} MarkovModel() {} void consume(const char * data, size_t size) From 9befe805e237738b371e27959791e767a7753b67 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 00:09:58 +0300 Subject: [PATCH 046/108] Added instruction for cppcheck usage [#CLICKHOUSE-3300]. --- dbms/tests/instructions/cppcheck.txt | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 dbms/tests/instructions/cppcheck.txt diff --git a/dbms/tests/instructions/cppcheck.txt b/dbms/tests/instructions/cppcheck.txt new file mode 100644 index 00000000000..af985624b13 --- /dev/null +++ b/dbms/tests/instructions/cppcheck.txt @@ -0,0 +1,17 @@ +# Install cppcheck + +mkdir cppcheck && cd cppcheck +git clone git@github.com:danmar/cppcheck.git . +mkdir build && cd build +CC=gcc-7 CXX=g++-7 cmake -D CMAKE_BUILD_TYPE=Release .. +make -j24 +sudo make install + +# Perform analysis +# cd ClickHouse_clean + +cppcheck -j24 -i contrib -i build --enable=all . 2> cppcheck-errors.txt + +# Check is pretty fast. +# It doesn't understand atomics, lambdas, std::rethrow_exception, macros, it have poor logic inference. +# But the result is worth looking and at least few real errors found. From 3de701850c6b25d27827ccdc46bdde2c8ad3277b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 00:13:30 +0300 Subject: [PATCH 047/108] Added instruction for Coverity usage [#CLICKHOUSE-3285]. --- dbms/tests/instructions/coverity.txt | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 dbms/tests/instructions/coverity.txt diff --git a/dbms/tests/instructions/coverity.txt b/dbms/tests/instructions/coverity.txt new file mode 100644 index 00000000000..59d94a9e54d --- /dev/null +++ b/dbms/tests/instructions/coverity.txt @@ -0,0 +1,28 @@ +# Download tool at https://scan.coverity.com/download?tab=cxx + +tar xf cov-analysis-linux64-2017.07.tar.gz +export PATH=$PATH:/home/milovidov/cov-analysis-linux64-2017.07/bin + +mkdir ClickHouse_coverity +cd ClickHouse_coverity +git clone git@github.com:yandex/ClickHouse.git . + +mkdir build +cd build + +# "Debug" is for faster build +CC=gcc-7 CXX=g++-7 cmake -D CMAKE_BUILD_TYPE=Debug -D CCACHE_FOUND=0 .. + +# Build all targets that we don't want to analyze. +cd contrib && make -j24 && cd .. + +cov-configure --comptype gcc --compiler gcc-7 --template + +cov-build --dir cov-int make -j24 + +# Build is painful slow. Some targets compile in about one hour. Total time is about 4..5 hours. + +tar czvf clickhouse.tgz cov-int + +# tarball is 1.2 GB. +# Upload result at https://scan.coverity.com/projects/yandex-clickhouse/builds/new From 79f65f4fd53117ff2a1fef8ec88b4c68a2a74673 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 00:19:43 +0300 Subject: [PATCH 048/108] Addition to prev. revision [#CLICKHOUSE-3300]. --- utils/iotest/iotest.cpp | 2 +- utils/iotest/iotest_aio.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/iotest/iotest.cpp b/utils/iotest/iotest.cpp index eb5999d02a9..2c91e11cd41 100644 --- a/utils/iotest/iotest.cpp +++ b/utils/iotest/iotest.cpp @@ -46,7 +46,7 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block char * buf; if ((mode & MODE_DIRECT)) - buf = direct_buf.data; + buf = direct_buf.data(); else buf = &simple_buf[0]; diff --git a/utils/iotest/iotest_aio.cpp b/utils/iotest/iotest_aio.cpp index eba52895b16..f23ce99dcf6 100644 --- a/utils/iotest/iotest_aio.cpp +++ b/utils/iotest/iotest_aio.cpp @@ -117,7 +117,7 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block ++blocks_sent; ++in_progress; - char * buf = buffers[i].data; + char * buf = buffers[i].data(); long rand_result1 = 0; long rand_result2 = 0; From 6307661c35d3b722e72a0bde7b75004f303309c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 00:26:25 +0300 Subject: [PATCH 049/108] Addition to prev. revision [#CLICKHOUSE-3300]. --- dbms/src/Parsers/TokenIterator.h | 2 +- dbms/src/Storages/MergeTree/LevelMergeSelector.h | 2 +- dbms/src/Storages/MergeTree/SimpleMergeSelector.h | 2 +- dbms/tests/instructions/cppcheck.txt | 8 +++++--- libs/libcommon/include/common/ThreadPool.h | 2 +- 5 files changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/src/Parsers/TokenIterator.h b/dbms/src/Parsers/TokenIterator.h index 642721eb37a..71d2edfef42 100644 --- a/dbms/src/Parsers/TokenIterator.h +++ b/dbms/src/Parsers/TokenIterator.h @@ -57,7 +57,7 @@ private: size_t index = 0; public: - TokenIterator(Tokens & tokens) : tokens(&tokens) {} + explicit TokenIterator(Tokens & tokens) : tokens(&tokens) {} const Token & get() { return (*tokens)[index]; } const Token & operator*() { return get(); } diff --git a/dbms/src/Storages/MergeTree/LevelMergeSelector.h b/dbms/src/Storages/MergeTree/LevelMergeSelector.h index d524ac6d5f2..fa35500dcae 100644 --- a/dbms/src/Storages/MergeTree/LevelMergeSelector.h +++ b/dbms/src/Storages/MergeTree/LevelMergeSelector.h @@ -17,7 +17,7 @@ public: size_t parts_to_merge = 10; }; - LevelMergeSelector(const Settings & settings) : settings(settings) {} + explicit LevelMergeSelector(const Settings & settings) : settings(settings) {} PartsInPartition select( const Partitions & partitions, diff --git a/dbms/src/Storages/MergeTree/SimpleMergeSelector.h b/dbms/src/Storages/MergeTree/SimpleMergeSelector.h index 17eb0aa2839..28918f072ed 100644 --- a/dbms/src/Storages/MergeTree/SimpleMergeSelector.h +++ b/dbms/src/Storages/MergeTree/SimpleMergeSelector.h @@ -71,7 +71,7 @@ public: double heuristic_to_remove_small_parts_at_right_max_ratio = 0.01; }; - SimpleMergeSelector(const Settings & settings) : settings(settings) {} + explicit SimpleMergeSelector(const Settings & settings) : settings(settings) {} PartsInPartition select( const Partitions & partitions, diff --git a/dbms/tests/instructions/cppcheck.txt b/dbms/tests/instructions/cppcheck.txt index af985624b13..5cad3b245a8 100644 --- a/dbms/tests/instructions/cppcheck.txt +++ b/dbms/tests/instructions/cppcheck.txt @@ -8,10 +8,12 @@ make -j24 sudo make install # Perform analysis -# cd ClickHouse_clean +# cd ClickHouse_clean/build -cppcheck -j24 -i contrib -i build --enable=all . 2> cppcheck-errors.txt +cppcheck -j24 --project=compile_commands.json --enable=all 2> cppcheck-errors.txt # Check is pretty fast. -# It doesn't understand atomics, lambdas, std::rethrow_exception, macros, it have poor logic inference. +# It gives many false positives. # But the result is worth looking and at least few real errors found. + +grep -v -F 'contrib/' cppcheck-errors.txt diff --git a/libs/libcommon/include/common/ThreadPool.h b/libs/libcommon/include/common/ThreadPool.h index 7e20294ec08..15952042a04 100644 --- a/libs/libcommon/include/common/ThreadPool.h +++ b/libs/libcommon/include/common/ThreadPool.h @@ -20,7 +20,7 @@ public: using Job = std::function; /// Size is constant, all threads are created immediately. - ThreadPool(size_t m_size); + explicit ThreadPool(size_t m_size); /// Add new job. Locks until free thread in pool become available or exception in one of threads was thrown. /// If an exception in some thread was thrown, method silently returns, and exception will be rethrown only on call to 'wait' function. From 742f369ac8ef1e7a9dc7a8766adfc0a9b70985e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 00:35:21 +0300 Subject: [PATCH 050/108] Addition to prev. revision [#CLICKHOUSE-3300]. --- dbms/tests/instructions/cppcheck.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/instructions/cppcheck.txt b/dbms/tests/instructions/cppcheck.txt index 5cad3b245a8..6d370a57ec3 100644 --- a/dbms/tests/instructions/cppcheck.txt +++ b/dbms/tests/instructions/cppcheck.txt @@ -11,6 +11,8 @@ sudo make install # cd ClickHouse_clean/build cppcheck -j24 --project=compile_commands.json --enable=all 2> cppcheck-errors.txt +# or +# cppcheck -j24 -i contrib -i build --enable=all . 2> cppcheck-errors.txt # Check is pretty fast. # It gives many false positives. From 5a87ebf262f6157d193df5379145ecf16de67108 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 7 Sep 2017 22:19:32 +0300 Subject: [PATCH 051/108] Try fix macos build --- dbms/src/Dictionaries/LibraryDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index 3b8d35dfc93..b600715f4a0 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -155,7 +155,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector & { LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size()); - const ClickHouseLibrary::VectorUInt64 ids_data{ids.data(), ids.size()}; + const ClickHouseLibrary::VectorUInt64 ids_data{static_cast(ids.data()), ids.size()}; auto columns_holder = std::make_unique(dict_struct.attributes.size()); ClickHouseLibrary::CStrings columns_pass{ static_cast(columns_holder.get()), dict_struct.attributes.size()}; From 1cd09ac12faed4caee2a781a43edfd2a79d49112 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 03:25:25 +0300 Subject: [PATCH 052/108] Updated script to build latest clang [#CLICKHOUSE-2]. --- utils/prepare-environment/install-clang.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/utils/prepare-environment/install-clang.sh b/utils/prepare-environment/install-clang.sh index 0adc70c139b..d1ca427ad4e 100755 --- a/utils/prepare-environment/install-clang.sh +++ b/utils/prepare-environment/install-clang.sh @@ -14,9 +14,15 @@ cd llvm svn co "http://llvm.org/svn/llvm-project/llvm/${BRANCH}" llvm cd llvm/tools svn co "http://llvm.org/svn/llvm-project/cfe/${BRANCH}" clang -cd .. -cd projects/ +svn co "http://llvm.org/svn/llvm-project/lld/${BRANCH}" lld +svn co "http://llvm.org/svn/llvm-project/polly/${BRANCH}" polly +cd clang/tools +svn co "http://llvm.org/svn/llvm-project/clang-tools-extra/${BRANCH}" extra +cd ../../../.. +cd llvm/projects/ svn co "http://llvm.org/svn/llvm-project/compiler-rt/${BRANCH}" compiler-rt +svn co "http://llvm.org/svn/llvm-project/libcxx/${BRANCH}" libcxx +svn co "http://llvm.org/svn/llvm-project/libcxxabi/${BRANCH}" libcxxabi cd ../.. mkdir build cd build/ From e9ae1938706c8c6e4aab478c0356b1dcf4d300e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 05:29:47 +0300 Subject: [PATCH 053/108] Implemented some suggestions from clang-tidy (part 1) [#CLICKHOUSE-3301]. --- dbms/src/Common/Exception.h | 2 +- dbms/src/Core/ColumnWithTypeAndName.h | 4 ++-- dbms/src/Core/SortDescription.h | 14 ++++++------- .../DataStreams/AggregatingBlockInputStream.h | 4 ++-- .../AsynchronousBlockInputStream.h | 4 ++-- .../DataStreams/BlockExtraInfoInputStream.h | 4 ++-- .../DataStreams/DistinctBlockInputStream.cpp | 4 ++-- .../DataStreams/DistinctBlockInputStream.h | 2 +- .../DistinctSortedBlockInputStream.cpp | 6 +++--- .../DistinctSortedBlockInputStream.h | 4 ++-- .../ExpressionBlockInputStream.cpp | 4 ++-- .../DataStreams/ExpressionBlockInputStream.h | 2 +- .../DataStreams/FilterBlockInputStream.cpp | 8 ++++---- dbms/src/DataStreams/FilterBlockInputStream.h | 4 ++-- dbms/src/DataStreams/ForkBlockInputStreams.h | 2 +- .../src/DataStreams/LimitBlockInputStream.cpp | 4 ++-- dbms/src/DataStreams/LimitBlockInputStream.h | 2 +- .../DataStreams/LimitByBlockInputStream.cpp | 4 ++-- .../src/DataStreams/LimitByBlockInputStream.h | 2 +- .../MaterializingBlockInputStream.cpp | 4 ++-- .../MaterializingBlockInputStream.h | 2 +- .../MergeSortingBlockInputStream.h | 4 ++-- .../MergingAggregatedBlockInputStream.h | 4 ++-- .../NullAndDoCopyBlockInputStream.h | 2 +- .../NullableAdapterBlockInputStream.cpp | 4 ++-- .../NullableAdapterBlockInputStream.h | 2 +- .../ParallelAggregatingBlockInputStream.cpp | 2 +- .../ParallelAggregatingBlockInputStream.h | 2 +- .../src/DataStreams/ParallelInputsProcessor.h | 4 ++-- .../PartialSortingBlockInputStream.h | 2 +- dbms/src/DataStreams/tests/glue_streams.cpp | 2 +- dbms/src/IO/WriteHelpers.h | 8 ++++---- dbms/src/Interpreters/Aggregator.cpp | 4 ++-- dbms/src/Interpreters/Aggregator.h | 4 ++-- .../Interpreters/InterpreterSelectQuery.cpp | 20 +++++++++---------- .../src/Interpreters/InterpreterSelectQuery.h | 4 ++-- dbms/src/Interpreters/SettingsCommon.h | 4 ++-- libs/libmysqlxx/include/mysqlxx/Connection.h | 6 +++--- libs/libmysqlxx/include/mysqlxx/Manip.h | 8 ++++---- 39 files changed, 86 insertions(+), 86 deletions(-) diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index a978ba4a303..756434a73ef 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -116,7 +116,7 @@ typename std::enable_if::value, T>::type exception_cast(std:: { try { - std::rethrow_exception(e); + std::rethrow_exception(std::move(e)); } catch (typename std::remove_pointer::type & concrete) { diff --git a/dbms/src/Core/ColumnWithTypeAndName.h b/dbms/src/Core/ColumnWithTypeAndName.h index 09da42ccf30..edf61430abf 100644 --- a/dbms/src/Core/ColumnWithTypeAndName.h +++ b/dbms/src/Core/ColumnWithTypeAndName.h @@ -19,11 +19,11 @@ struct ColumnWithTypeAndName String name; ColumnWithTypeAndName() {} - ColumnWithTypeAndName(const ColumnPtr & column_, const DataTypePtr & type_, const String name_) + ColumnWithTypeAndName(const ColumnPtr & column_, const DataTypePtr & type_, const String & name_) : column(column_), type(type_), name(name_) {} /// Uses type->createColumn() to create column - ColumnWithTypeAndName(const DataTypePtr & type_, const String name_) + ColumnWithTypeAndName(const DataTypePtr & type_, const String & name_) : column(type_->createColumn()), type(type_), name(name_) {} ColumnWithTypeAndName cloneEmpty() const; diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index 70069b4beb9..3c757cc8c21 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -14,17 +14,17 @@ namespace DB /// Description of the sorting rule by one column. struct SortColumnDescription { - std::string column_name; /// The name of the column. - size_t column_number; /// Column number (used if no name is given). - int direction; /// 1 - ascending, -1 - descending. - int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less. - /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite. - std::shared_ptr collator; /// Collator for locale-specific comparison of strings + std::string column_name; /// The name of the column. + size_t column_number; /// Column number (used if no name is given). + int direction; /// 1 - ascending, -1 - descending. + int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less. + /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite. + std::shared_ptr collator; /// Collator for locale-specific comparison of strings SortColumnDescription(size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} - SortColumnDescription(std::string column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) + SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} /// For IBlockInputStream. diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.h b/dbms/src/DataStreams/AggregatingBlockInputStream.h index 8bc92c3c3ec..ea93188afba 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.h @@ -22,10 +22,10 @@ public: * Aggregate functions are searched everywhere in the expression. * Columns corresponding to keys and arguments of aggregate functions must already be computed. */ - AggregatingBlockInputStream(BlockInputStreamPtr input_, const Aggregator::Params & params_, bool final_) + AggregatingBlockInputStream(const BlockInputStreamPtr & input, const Aggregator::Params & params_, bool final_) : params(params_), aggregator(params), final(final_) { - children.push_back(input_); + children.push_back(input); } String getName() const override { return "Aggregating"; } diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index c14cb5bbd90..4ee17c2d30f 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -28,9 +28,9 @@ namespace DB class AsynchronousBlockInputStream : public IProfilingBlockInputStream { public: - AsynchronousBlockInputStream(BlockInputStreamPtr in_) + AsynchronousBlockInputStream(const BlockInputStreamPtr & in) { - children.push_back(in_); + children.push_back(in); } String getName() const override { return "Asynchronous"; } diff --git a/dbms/src/DataStreams/BlockExtraInfoInputStream.h b/dbms/src/DataStreams/BlockExtraInfoInputStream.h index fb3c7e03817..c1a6d77874b 100644 --- a/dbms/src/DataStreams/BlockExtraInfoInputStream.h +++ b/dbms/src/DataStreams/BlockExtraInfoInputStream.h @@ -11,10 +11,10 @@ namespace DB class BlockExtraInfoInputStream : public IProfilingBlockInputStream { public: - BlockExtraInfoInputStream(BlockInputStreamPtr input_, const BlockExtraInfo & block_extra_info_) + BlockExtraInfoInputStream(const BlockInputStreamPtr & input, const BlockExtraInfo & block_extra_info_) : block_extra_info(block_extra_info_) { - children.push_back(input_); + children.push_back(input); } BlockExtraInfo getBlockExtraInfo() const override diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.cpp b/dbms/src/DataStreams/DistinctBlockInputStream.cpp index f27c1ee0f62..d308eb9f17b 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctBlockInputStream.cpp @@ -8,14 +8,14 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; } -DistinctBlockInputStream::DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_) +DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_) : columns_names(columns_) , limit_hint(limit_hint_) , max_rows(limits.max_rows_in_distinct) , max_bytes(limits.max_bytes_in_distinct) , overflow_mode(limits.distinct_overflow_mode) { - children.push_back(input_); + children.push_back(input); } String DistinctBlockInputStream::getID() const diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.h b/dbms/src/DataStreams/DistinctBlockInputStream.h index 41a9e7a2087..7120d3e04a9 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctBlockInputStream.h @@ -18,7 +18,7 @@ class DistinctBlockInputStream : public IProfilingBlockInputStream { public: /// Empty columns_ means all collumns. - DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_); + DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_); String getName() const override { return "Distinct"; } diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp index 1dc33a78eab..2c8ab1e1d8e 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp @@ -8,15 +8,15 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; } -DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_) - : description( input_->getSortDescription() ) +DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_) + : description(input->getSortDescription()) , columns_names(columns_) , limit_hint(limit_hint_) , max_rows(limits.max_rows_in_distinct) , max_bytes(limits.max_bytes_in_distinct) , overflow_mode(limits.distinct_overflow_mode) { - children.push_back(input_); + children.push_back(input); } String DistinctSortedBlockInputStream::getID() const diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h index bf5bbb6c90c..28601f4bb04 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h @@ -21,7 +21,7 @@ class DistinctSortedBlockInputStream : public IProfilingBlockInputStream { public: /// Empty columns_ means all collumns. - DistinctSortedBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_); + DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_); String getName() const override { return "DistinctSorted"; } @@ -50,7 +50,7 @@ private: ClearableSetVariants & variants) const; const SortDescription & description; - + struct PreviousBlock { Block block; diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp index 916de1496b9..6130df31b51 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp @@ -5,10 +5,10 @@ namespace DB { -ExpressionBlockInputStream::ExpressionBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_) +ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_) : expression(expression_) { - children.push_back(input_); + children.push_back(input); } String ExpressionBlockInputStream::getName() const { return "Expression"; } diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.h b/dbms/src/DataStreams/ExpressionBlockInputStream.h index 6e5b166bad8..c57ea928d54 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.h +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.h @@ -19,7 +19,7 @@ private: using ExpressionActionsPtr = std::shared_ptr; public: - ExpressionBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_); + ExpressionBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/FilterBlockInputStream.cpp b/dbms/src/DataStreams/FilterBlockInputStream.cpp index dcf619bcfd7..be0e9182c08 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.cpp +++ b/dbms/src/DataStreams/FilterBlockInputStream.cpp @@ -17,16 +17,16 @@ namespace ErrorCodes } -FilterBlockInputStream::FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, ssize_t filter_column_) +FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, ssize_t filter_column_) : expression(expression_), filter_column(filter_column_) { - children.push_back(input_); + children.push_back(input); } -FilterBlockInputStream::FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, const String & filter_column_name_) +FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, const String & filter_column_name_) : expression(expression_), filter_column(-1), filter_column_name(filter_column_name_) { - children.push_back(input_); + children.push_back(input); } diff --git a/dbms/src/DataStreams/FilterBlockInputStream.h b/dbms/src/DataStreams/FilterBlockInputStream.h index 3f0cda420c6..95ade939256 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.h +++ b/dbms/src/DataStreams/FilterBlockInputStream.h @@ -20,8 +20,8 @@ private: public: /// filter_column_ - the number of the column with filter conditions. - FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, ssize_t filter_column_); - FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, const String & filter_column_name_); + FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, ssize_t filter_column_); + FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, const String & filter_column_name_); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/ForkBlockInputStreams.h b/dbms/src/DataStreams/ForkBlockInputStreams.h index e10e0d3f8fd..504cfc5bdbe 100644 --- a/dbms/src/DataStreams/ForkBlockInputStreams.h +++ b/dbms/src/DataStreams/ForkBlockInputStreams.h @@ -18,7 +18,7 @@ namespace DB class ForkBlockInputStreams : private boost::noncopyable { public: - ForkBlockInputStreams(BlockInputStreamPtr source_) : source(source_) {} + ForkBlockInputStreams(const BlockInputStreamPtr & source_) : source(source_) {} /// Create a source. Call the function as many times as many forked sources you need. BlockInputStreamPtr createInput() diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 56edf6d1e98..7939d1f73ca 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -6,10 +6,10 @@ namespace DB { -LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_) +LimitBlockInputStream::LimitBlockInputStream(const BlockInputStreamPtr & input, size_t limit_, size_t offset_, bool always_read_till_end_) : limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_) { - children.push_back(input_); + children.push_back(input); } diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index 1588441ee91..23bd5739aaf 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -17,7 +17,7 @@ public: * If always_read_till_end = true - reads all the data to the end, but ignores them. This is necessary in rare cases: * when otherwise, due to the cancellation of the request, we would not have received the data for GROUP BY WITH TOTALS from the remote server. */ - LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_ = false); + LimitBlockInputStream(const BlockInputStreamPtr & input, size_t limit_, size_t offset_, bool always_read_till_end_ = false); String getName() const override { return "Limit"; } diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.cpp b/dbms/src/DataStreams/LimitByBlockInputStream.cpp index 8e0125a891e..6182dd3413b 100644 --- a/dbms/src/DataStreams/LimitByBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitByBlockInputStream.cpp @@ -3,11 +3,11 @@ namespace DB { -LimitByBlockInputStream::LimitByBlockInputStream(BlockInputStreamPtr input_, size_t group_size_, Names columns_) +LimitByBlockInputStream::LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, Names columns_) : columns_names(columns_) , group_size(group_size_) { - children.push_back(input_); + children.push_back(input); } Block LimitByBlockInputStream::readImpl() diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.h b/dbms/src/DataStreams/LimitByBlockInputStream.h index 0460e799ad6..54874d37f1e 100644 --- a/dbms/src/DataStreams/LimitByBlockInputStream.h +++ b/dbms/src/DataStreams/LimitByBlockInputStream.h @@ -18,7 +18,7 @@ namespace DB class LimitByBlockInputStream : public IProfilingBlockInputStream { public: - LimitByBlockInputStream(BlockInputStreamPtr input_, size_t group_size_, Names columns_); + LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, Names columns_); String getName() const override { return "LimitBy"; } diff --git a/dbms/src/DataStreams/MaterializingBlockInputStream.cpp b/dbms/src/DataStreams/MaterializingBlockInputStream.cpp index fefdfb3be26..640eb87b103 100644 --- a/dbms/src/DataStreams/MaterializingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MaterializingBlockInputStream.cpp @@ -5,9 +5,9 @@ namespace DB { -MaterializingBlockInputStream::MaterializingBlockInputStream(BlockInputStreamPtr input_) +MaterializingBlockInputStream::MaterializingBlockInputStream(const BlockInputStreamPtr & input) { - children.push_back(input_); + children.push_back(input); } String MaterializingBlockInputStream::getName() const diff --git a/dbms/src/DataStreams/MaterializingBlockInputStream.h b/dbms/src/DataStreams/MaterializingBlockInputStream.h index 654249e309a..feeca05298e 100644 --- a/dbms/src/DataStreams/MaterializingBlockInputStream.h +++ b/dbms/src/DataStreams/MaterializingBlockInputStream.h @@ -10,7 +10,7 @@ namespace DB class MaterializingBlockInputStream : public IProfilingBlockInputStream { public: - MaterializingBlockInputStream(BlockInputStreamPtr input_); + MaterializingBlockInputStream(const BlockInputStreamPtr & input); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index ce59bb184b0..4a97d214a79 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -69,13 +69,13 @@ class MergeSortingBlockInputStream : public IProfilingBlockInputStream { public: /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. - MergeSortingBlockInputStream(BlockInputStreamPtr input_, SortDescription & description_, + MergeSortingBlockInputStream(const BlockInputStreamPtr & input, SortDescription & description_, size_t max_merged_block_size_, size_t limit_, size_t max_bytes_before_external_sort_, const std::string & tmp_path_) : description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_), max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_) { - children.push_back(input_); + children.push_back(input); } String getName() const override { return "MergeSorting"; } diff --git a/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h index 2346f7bd221..f133524d0f1 100644 --- a/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h @@ -14,10 +14,10 @@ namespace DB class MergingAggregatedBlockInputStream : public IProfilingBlockInputStream { public: - MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const Aggregator::Params & params, bool final_, size_t max_threads_) + MergingAggregatedBlockInputStream(const BlockInputStreamPtr & input, const Aggregator::Params & params, bool final_, size_t max_threads_) : aggregator(params), final(final_), max_threads(max_threads_) { - children.push_back(input_); + children.push_back(input); } String getName() const override { return "MergingAggregated"; } diff --git a/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h b/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h index 3aa39e2da23..cd7823f236d 100644 --- a/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h +++ b/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h @@ -20,7 +20,7 @@ using BlockOutputStreamPtr = std::shared_ptr; class NullAndDoCopyBlockInputStream : public IProfilingBlockInputStream { public: - NullAndDoCopyBlockInputStream(BlockInputStreamPtr input_, BlockOutputStreamPtr output_) + NullAndDoCopyBlockInputStream(const BlockInputStreamPtr & input_, BlockOutputStreamPtr output_) : input(input_), output(output_) { children.push_back(input_); diff --git a/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp b/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp index c7b87d4b86d..49a349ec980 100644 --- a/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp +++ b/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp @@ -17,11 +17,11 @@ extern const int TYPE_MISMATCH; } NullableAdapterBlockInputStream::NullableAdapterBlockInputStream( - BlockInputStreamPtr input_, + const BlockInputStreamPtr & input, const Block & in_sample_, const Block & out_sample_) { buildActions(in_sample_, out_sample_); - children.push_back(input_); + children.push_back(input); } String NullableAdapterBlockInputStream::getID() const diff --git a/dbms/src/DataStreams/NullableAdapterBlockInputStream.h b/dbms/src/DataStreams/NullableAdapterBlockInputStream.h index e58920cd147..917733cc6d4 100644 --- a/dbms/src/DataStreams/NullableAdapterBlockInputStream.h +++ b/dbms/src/DataStreams/NullableAdapterBlockInputStream.h @@ -18,7 +18,7 @@ namespace DB class NullableAdapterBlockInputStream : public IProfilingBlockInputStream { public: - NullableAdapterBlockInputStream(BlockInputStreamPtr input_, const Block & in_sample_, + NullableAdapterBlockInputStream(const BlockInputStreamPtr & input, const Block & in_sample_, const Block & out_sample_); String getName() const override { return "NullableAdapterBlockInputStream"; } diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index 40c71d244fc..92bfe5e1f9e 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -16,7 +16,7 @@ namespace DB ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream( - BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, + const BlockInputStreams & inputs, const BlockInputStreamPtr & additional_input_at_end, const Aggregator::Params & params_, bool final_, size_t max_threads_, size_t temporary_data_merge_threads_) : params(params_), aggregator(params), final(final_), max_threads(std::min(inputs.size(), max_threads_)), temporary_data_merge_threads(temporary_data_merge_threads_), diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index 3bc08e75ea3..f76ef4e13b8 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -22,7 +22,7 @@ public: /** Columns from key_names and arguments of aggregate functions must already be computed. */ ParallelAggregatingBlockInputStream( - BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, + const BlockInputStreams & inputs, const BlockInputStreamPtr & additional_input_at_end, const Aggregator::Params & params_, bool final_, size_t max_threads_, size_t temporary_data_merge_threads_); String getName() const override { return "ParallelAggregating"; } diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index cdf17e1ab4f..c00c9f26ffa 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -76,7 +76,7 @@ public: * - where you must first make JOIN in parallel, while noting which keys are not found, * and only after the completion of this work, create blocks of keys that are not found. */ - ParallelInputsProcessor(BlockInputStreams inputs_, BlockInputStreamPtr additional_input_at_end_, size_t max_threads_, Handler & handler_) + ParallelInputsProcessor(const BlockInputStreams & inputs_, const BlockInputStreamPtr & additional_input_at_end_, size_t max_threads_, Handler & handler_) : inputs(inputs_), additional_input_at_end(additional_input_at_end_), max_threads(std::min(inputs_.size(), max_threads_)), handler(handler_) { for (size_t i = 0; i < inputs_.size(); ++i) @@ -155,7 +155,7 @@ private: size_t i; /// The source number (for debugging). InputData() {} - InputData(BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {} + InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {} }; template diff --git a/dbms/src/DataStreams/PartialSortingBlockInputStream.h b/dbms/src/DataStreams/PartialSortingBlockInputStream.h index 924fdb0e2bf..f46fa8e5ca5 100644 --- a/dbms/src/DataStreams/PartialSortingBlockInputStream.h +++ b/dbms/src/DataStreams/PartialSortingBlockInputStream.h @@ -15,7 +15,7 @@ class PartialSortingBlockInputStream : public IProfilingBlockInputStream { public: /// limit - if not 0, then you can sort each block not completely, but only `limit` first rows by order. - PartialSortingBlockInputStream(BlockInputStreamPtr input_, SortDescription & description_, size_t limit_ = 0) + PartialSortingBlockInputStream(const BlockInputStreamPtr & input_, SortDescription & description_, size_t limit_ = 0) : description(description_), limit(limit_) { children.push_back(input_); diff --git a/dbms/src/DataStreams/tests/glue_streams.cpp b/dbms/src/DataStreams/tests/glue_streams.cpp index 78205b9c0ce..8982f2c0ef6 100644 --- a/dbms/src/DataStreams/tests/glue_streams.cpp +++ b/dbms/src/DataStreams/tests/glue_streams.cpp @@ -15,7 +15,7 @@ using namespace DB; -void inputThread(BlockInputStreamPtr in, BlockOutputStreamPtr out, WriteBuffer & wb, std::mutex & mutex) +void inputThread(const BlockInputStreamPtr & in, BlockOutputStreamPtr out, WriteBuffer & wb, std::mutex & mutex) { while (Block block = in->read()) { diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index 76def04e0a4..6c0cf219bac 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -197,7 +197,7 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & break; default: UInt8 c = *it; - if (0x00 <= c && c <= 0x1F) + if (c <= 0x1F) { /// Escaping of ASCII control characters. @@ -207,7 +207,7 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & writeCString("\\u00", buf); writeChar('0' + higher_half, buf); - if (0 <= lower_half && lower_half <= 9) + if (lower_half <= 9) writeChar('0' + lower_half, buf); else writeChar('A' + lower_half - 10, buf); @@ -522,7 +522,7 @@ inline void writeDateText(DayNum_t date, WriteBuffer & buf) buf.write(s, 10); } -inline void writeDateText(LocalDate date, WriteBuffer & buf) +inline void writeDateText(const LocalDate & date, WriteBuffer & buf) { char s[10] = {'0', '0', '0', '0', '-', '0', '0', '-', '0', '0'}; @@ -577,7 +577,7 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTI } template -inline void writeDateTimeText(LocalDateTime datetime, WriteBuffer & buf) +inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf) { char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'}; diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 8b038f6aa67..be4badb2af0 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1034,7 +1034,7 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const } -void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & result) +void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVariants & result) { if (isCancelled()) return; @@ -1989,7 +1989,7 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl( } -void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads) +void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataVariants & result, size_t max_threads) { if (isCancelled()) return; diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 2003759fbbf..ee0b0e6cd87 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -1039,7 +1039,7 @@ public: } /// Aggregate the source. Get the result in the form of one of the data structures. - void execute(BlockInputStreamPtr stream, AggregatedDataVariants & result); + void execute(const BlockInputStreamPtr & stream, AggregatedDataVariants & result); using AggregateColumns = std::vector; using AggregateColumnsData = std::vector; @@ -1067,7 +1067,7 @@ public: /** Merge the stream of partially aggregated blocks into one data structure. * (Pre-aggregate several blocks that represent the result of independent aggregations from remote servers.) */ - void mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads); + void mergeStream(const BlockInputStreamPtr & stream, AggregatedDataVariants & result, size_t max_threads); /// Merge several partially aggregated blocks into one. /// Precondition: for all blocks block.info.is_overflows flag must be the same. diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index c5518e7de35..da2a9e2e0d4 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -69,7 +69,7 @@ namespace ErrorCodes InterpreterSelectQuery::~InterpreterSelectQuery() = default; -void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & required_column_names) +void InterpreterSelectQuery::init(const BlockInputStreamPtr & input, const Names & required_column_names) { ProfileEvents::increment(ProfileEvents::SelectQuery); @@ -126,7 +126,7 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi } } -void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_) +void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input) { auto query_table = query.table(); @@ -171,8 +171,8 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_) if (!context.tryGetExternalTable(it.first)) context.addExternalTable(it.first, it.second); - if (input_) - streams.push_back(input_); + if (input) + streams.push_back(input); if (is_first_select_inside_union_all) { @@ -200,7 +200,7 @@ void InterpreterSelectQuery::initQueryAnalyzer() } InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, QueryProcessingStage::Enum to_stage_, - size_t subquery_depth_, BlockInputStreamPtr input_) + size_t subquery_depth_, BlockInputStreamPtr input) : query_ptr(query_ptr_) , query(typeid_cast(*query_ptr)) , context(context_) @@ -209,7 +209,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const , is_first_select_inside_union_all(query.isUnionAllHead()) , log(&Logger::get("InterpreterSelectQuery")) { - init(input_); + init(input); } InterpreterSelectQuery::InterpreterSelectQuery(OnlyAnalyzeTag, const ASTPtr & query_ptr_, const Context & context_) @@ -226,14 +226,14 @@ InterpreterSelectQuery::InterpreterSelectQuery(OnlyAnalyzeTag, const ASTPtr & qu InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, const Names & required_column_names_, - QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input_) - : InterpreterSelectQuery(query_ptr_, context_, required_column_names_, {}, to_stage_, subquery_depth_, input_) + QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input) + : InterpreterSelectQuery(query_ptr_, context_, required_column_names_, {}, to_stage_, subquery_depth_, input) { } InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, const Names & required_column_names_, - const NamesAndTypesList & table_column_names_, QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input_) + const NamesAndTypesList & table_column_names_, QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input) : query_ptr(query_ptr_) , query(typeid_cast(*query_ptr)) , context(context_) @@ -243,7 +243,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const , is_first_select_inside_union_all(query.isUnionAllHead()) , log(&Logger::get("InterpreterSelectQuery")) { - init(input_, required_column_names_); + init(input, required_column_names_); } bool InterpreterSelectQuery::hasAsterisk() const diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 83b1722b2cc..a7e8ea1445e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -93,8 +93,8 @@ private: const ASTPtr & query_ptr_, const Context & context_); - void init(BlockInputStreamPtr input, const Names & required_column_names = Names{}); - void basicInit(BlockInputStreamPtr input); + void init(const BlockInputStreamPtr & input, const Names & required_column_names = Names{}); + void basicInit(const BlockInputStreamPtr & input); void initQueryAnalyzer(); /// Execute one SELECT query from the UNION ALL chain. diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index e3cf90eee11..69fbee54d9e 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -219,7 +219,7 @@ struct SettingMilliseconds SettingMilliseconds(UInt64 milliseconds = 0) : value(milliseconds * 1000) {} operator Poco::Timespan() const { return value; } - SettingMilliseconds & operator= (Poco::Timespan x) { set(x); return *this; } + SettingMilliseconds & operator= (const Poco::Timespan & x) { set(x); return *this; } Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } @@ -228,7 +228,7 @@ struct SettingMilliseconds return DB::toString(totalMilliseconds()); } - void set(Poco::Timespan x) + void set(const Poco::Timespan & x) { value = x; changed = true; diff --git a/libs/libmysqlxx/include/mysqlxx/Connection.h b/libs/libmysqlxx/include/mysqlxx/Connection.h index 7ecb3c6fdc5..c704faa3212 100644 --- a/libs/libmysqlxx/include/mysqlxx/Connection.h +++ b/libs/libmysqlxx/include/mysqlxx/Connection.h @@ -47,7 +47,7 @@ private: * In order to use connection in other thread, you should call MySQL C API function mysql_thread_init() before and * mysql_thread_end() after working with it. */ -class Connection : private boost::noncopyable +class Connection final : private boost::noncopyable { public: /// For delayed initialisation @@ -69,10 +69,10 @@ public: /// All settings will be got from config_name section of configuration. Connection(const std::string & config_name); - virtual ~Connection(); + ~Connection(); /// Provides delayed initialization or reconnection with other settings. - virtual void connect(const char * db, + void connect(const char * db, const char * server, const char * user, const char * password, diff --git a/libs/libmysqlxx/include/mysqlxx/Manip.h b/libs/libmysqlxx/include/mysqlxx/Manip.h index 11fab8264dd..c02d9e10515 100644 --- a/libs/libmysqlxx/include/mysqlxx/Manip.h +++ b/libs/libmysqlxx/include/mysqlxx/Manip.h @@ -50,8 +50,8 @@ struct EscapeManipResult typename std::enable_if::value, std::ostream &>::type operator<< (T value) { return ostr << value; } - std::ostream & operator<< (LocalDate value) { return ostr << value; } - std::ostream & operator<< (LocalDateTime value) { return ostr << value; } + std::ostream & operator<< (const LocalDate & value) { return ostr << value; } + std::ostream & operator<< (const LocalDateTime & value) { return ostr << value; } std::ostream & operator<< (const std::string & value) { @@ -172,8 +172,8 @@ public: typename std::enable_if::value, std::ostream &>::type operator<< (T value) { return ostr << value; } - std::ostream & operator<< (LocalDate value) { return ostr << '\'' << value << '\''; } - std::ostream & operator<< (LocalDateTime value) { return ostr << '\'' << value << '\''; } + std::ostream & operator<< (const LocalDate & value) { return ostr << '\'' << value << '\''; } + std::ostream & operator<< (const LocalDateTime & value) { return ostr << '\'' << value << '\''; } std::ostream & operator<< (const std::string & value) { From abad3f432d4b97acf47a8db0598aa8965b1c49bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 05:31:58 +0300 Subject: [PATCH 054/108] Added instruction to run clang-tidy [#CLICKHOUSE-3301]. --- dbms/tests/instructions/clang-tidy.txt | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 dbms/tests/instructions/clang-tidy.txt diff --git a/dbms/tests/instructions/clang-tidy.txt b/dbms/tests/instructions/clang-tidy.txt new file mode 100644 index 00000000000..952e5f04a72 --- /dev/null +++ b/dbms/tests/instructions/clang-tidy.txt @@ -0,0 +1,11 @@ +# Install latest clang with extra tools. +# Look at utils/prepare-environment/install-clang.sh + +sudo apt-get install jq + +cd build +CC=clang CXX=clang++ cmake .. + +cd .. + +jq '.[] | .file' build/compile_commands.json | grep -v -F 'contrib' | head -n1 | xargs -I{} clang-tidy {} -p build -header-filter='dbms|libs' -checks='boost-*,bugprone-*,clang-analyzer-*,performance-*' From d19d9f8589d8e39d104a7ad8cd7d56c79324c250 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 05:37:02 +0300 Subject: [PATCH 055/108] Implemented some suggestions from clang-tidy (part 2) [#CLICKHOUSE-3301]. --- dbms/src/Columns/IColumn.h | 2 +- dbms/src/Interpreters/SettingsCommon.h | 4 ++-- utils/zookeeper-cli/zookeeper-cli.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 2b5925f889d..ff27021e584 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -284,7 +284,7 @@ protected: column = cloneEmpty(); { - size_t reserve_size = num_rows / num_columns * 1.1; /// 1.1 is just a guess. Better to use n-sigma rule. + size_t reserve_size = num_rows * 1.1 / num_columns; /// 1.1 is just a guess. Better to use n-sigma rule. if (reserve_size > 1) for (auto & column : columns) diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index 69fbee54d9e..55ffc48dff1 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -167,7 +167,7 @@ struct SettingSeconds SettingSeconds(UInt64 seconds = 0) : value(seconds, 0) {} operator Poco::Timespan() const { return value; } - SettingSeconds & operator= (Poco::Timespan x) { set(x); return *this; } + SettingSeconds & operator= (const Poco::Timespan & x) { set(x); return *this; } Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } @@ -176,7 +176,7 @@ struct SettingSeconds return DB::toString(totalSeconds()); } - void set(Poco::Timespan x) + void set(const Poco::Timespan & x) { value = x; changed = true; diff --git a/utils/zookeeper-cli/zookeeper-cli.cpp b/utils/zookeeper-cli/zookeeper-cli.cpp index d6d49116ff7..266bf5bb34d 100644 --- a/utils/zookeeper-cli/zookeeper-cli.cpp +++ b/utils/zookeeper-cli/zookeeper-cli.cpp @@ -25,7 +25,7 @@ void printStat(const zkutil::Stat & s) std::cout << " pzxid: " << s.pzxid << std::endl; } -void waitForWatch(zkutil::EventPtr event) +void waitForWatch(const zkutil::EventPtr & event) { std::cout << "waiting for watch" << std::endl; event->wait(); From f1c98dac3c4af9f8d4e011498bef16684a4d7fd0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 05:42:47 +0300 Subject: [PATCH 056/108] Updated instruction to run clang-tidy [#CLICKHOUSE-3301]. --- dbms/tests/instructions/clang-tidy.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/tests/instructions/clang-tidy.txt b/dbms/tests/instructions/clang-tidy.txt index 952e5f04a72..cc6cc2e9556 100644 --- a/dbms/tests/instructions/clang-tidy.txt +++ b/dbms/tests/instructions/clang-tidy.txt @@ -8,4 +8,7 @@ CC=clang CXX=clang++ cmake .. cd .. -jq '.[] | .file' build/compile_commands.json | grep -v -F 'contrib' | head -n1 | xargs -I{} clang-tidy {} -p build -header-filter='dbms|libs' -checks='boost-*,bugprone-*,clang-analyzer-*,performance-*' +for i in $(jq --raw-output '.[] | .file' build/compile_commands.json | grep -v -F 'contrib'); do + echo "$i"; + clang-tidy "$i" -p build -header-filter='dbms|libs' -checks='boost-*,bugprone-*,clang-analyzer-*,performance-*'; +done From d29c77adea08af7922d7b82e56ccee2f2d11783f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 06:47:27 +0300 Subject: [PATCH 057/108] Implemented some suggestions from clang-tidy (part 3) [#CLICKHOUSE-3301]. --- dbms/src/Analyzers/CollectAliases.h | 2 +- dbms/src/Columns/ColumnSet.h | 2 +- .../AddingDefaultBlockOutputStream.h | 2 +- .../BlockInputStreamFromRowInputStream.cpp | 2 +- .../BlockInputStreamFromRowInputStream.h | 2 +- dbms/src/DataStreams/CSVRowInputStream.cpp | 4 +- .../CollapsingFinalBlockInputStream.h | 12 ++- .../DataStreams/CountingBlockOutputStream.h | 6 +- .../CreatingSetsBlockInputStream.cpp | 27 +++--- .../CreatingSetsBlockInputStream.h | 2 +- .../DataStreams/DistinctBlockInputStream.cpp | 28 +++--- .../DataStreams/DistinctBlockInputStream.h | 2 +- .../DistinctSortedBlockInputStream.cpp | 24 +++-- .../ExpressionBlockInputStream.cpp | 2 +- .../DataStreams/ExpressionBlockInputStream.h | 2 +- .../DataStreams/FilterBlockInputStream.cpp | 4 +- dbms/src/DataStreams/FilterBlockInputStream.h | 4 +- .../FilterColumnsBlockInputStream.h | 4 +- dbms/src/DataStreams/IBlockInputStream.cpp | 4 +- dbms/src/DataStreams/IBlockInputStream.h | 2 +- .../IProfilingBlockInputStream.cpp | 96 +++++++++---------- .../DataStreams/IProfilingBlockInputStream.h | 2 +- .../DataStreams/LimitByBlockInputStream.cpp | 4 +- .../src/DataStreams/LimitByBlockInputStream.h | 2 +- ...ggregatedMemoryEfficientBlockInputStream.h | 4 +- .../TotalsHavingBlockInputStream.cpp | 4 +- .../TotalsHavingBlockInputStream.h | 4 +- dbms/src/DataTypes/DataTypeArray.cpp | 4 +- dbms/src/DataTypes/DataTypeArray.h | 4 +- dbms/src/DataTypes/DataTypeExpression.h | 2 +- dbms/src/DataTypes/DataTypeNested.cpp | 2 +- dbms/src/DataTypes/DataTypeNested.h | 2 +- dbms/src/DataTypes/DataTypeNullable.cpp | 2 +- dbms/src/DataTypes/DataTypeNullable.h | 2 +- dbms/src/DataTypes/DataTypeTuple.h | 2 +- .../Dictionaries/DictionaryBlockInputStream.h | 2 +- dbms/src/Dictionaries/DictionaryStructure.cpp | 2 +- .../ExecutableDictionarySource.cpp | 2 +- dbms/src/Dictionaries/FlatDictionary.cpp | 2 +- dbms/src/Interpreters/Aggregator.cpp | 24 +++-- dbms/src/Interpreters/Compiler.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 4 +- dbms/src/Interpreters/Context.h | 2 +- dbms/src/Interpreters/ExpressionActions.h | 2 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 12 +-- dbms/src/Interpreters/ExpressionAnalyzer.h | 12 +-- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Interpreters/Join.cpp | 24 +++-- dbms/src/Interpreters/ProcessList.cpp | 2 +- dbms/src/Interpreters/ProcessList.h | 2 +- dbms/src/Interpreters/Set.cpp | 24 +++-- dbms/src/Storages/MergeTree/PKCondition.cpp | 2 +- dbms/src/Storages/MergeTree/PKCondition.h | 2 +- 53 files changed, 211 insertions(+), 186 deletions(-) diff --git a/dbms/src/Analyzers/CollectAliases.h b/dbms/src/Analyzers/CollectAliases.h index 0c6046bc509..46ce74909f8 100644 --- a/dbms/src/Analyzers/CollectAliases.h +++ b/dbms/src/Analyzers/CollectAliases.h @@ -41,7 +41,7 @@ struct CollectAliases ASTPtr node; Kind kind; - AliasInfo(ASTPtr node, Kind kind) : node(node), kind(kind) {} + AliasInfo(const ASTPtr & node, Kind kind) : node(node), kind(kind) {} }; using Aliases = std::unordered_map; diff --git a/dbms/src/Columns/ColumnSet.h b/dbms/src/Columns/ColumnSet.h index 59e9c85a904..7e07d38f5c6 100644 --- a/dbms/src/Columns/ColumnSet.h +++ b/dbms/src/Columns/ColumnSet.h @@ -17,7 +17,7 @@ using ConstSetPtr = std::shared_ptr; class ColumnSet final : public IColumnDummy { public: - ColumnSet(size_t s_, ConstSetPtr data_) : IColumnDummy(s_), data(data_) {} + ColumnSet(size_t s_, const ConstSetPtr & data_) : IColumnDummy(s_), data(data_) {} /// The column is not a constant. Otherwise, the column will be used in calculations in ExpressionActions::prepare, when a set from subquery is not ready yet. bool isConst() const override { return false; } diff --git a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h index b9955792f41..a6e86ca16d0 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h +++ b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h @@ -18,7 +18,7 @@ class AddingDefaultBlockOutputStream : public IBlockOutputStream { public: AddingDefaultBlockOutputStream( - BlockOutputStreamPtr output_, + const BlockOutputStreamPtr & output_, NamesAndTypesListPtr required_columns_, const ColumnDefaults & column_defaults_, const Context & context_, diff --git a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp index 48ea7eb5f0a..dd0eb257331 100644 --- a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp +++ b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes BlockInputStreamFromRowInputStream::BlockInputStreamFromRowInputStream( - RowInputStreamPtr row_input_, + const RowInputStreamPtr & row_input_, const Block & sample_, size_t max_block_size_, UInt64 allow_errors_num_, diff --git a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.h b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.h index b589a767065..f5c2f2f35e1 100644 --- a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.h +++ b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.h @@ -18,7 +18,7 @@ class BlockInputStreamFromRowInputStream : public IProfilingBlockInputStream public: /** sample_ - block with zero rows, that structure describes how to interpret values */ BlockInputStreamFromRowInputStream( - RowInputStreamPtr row_input_, + const RowInputStreamPtr & row_input_, const Block & sample_, size_t max_block_size_, UInt64 allow_errors_num_, diff --git a/dbms/src/DataStreams/CSVRowInputStream.cpp b/dbms/src/DataStreams/CSVRowInputStream.cpp index edfc096b973..24c81cb3185 100644 --- a/dbms/src/DataStreams/CSVRowInputStream.cpp +++ b/dbms/src/DataStreams/CSVRowInputStream.cpp @@ -206,8 +206,8 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block, << "name: " << sample.safeGetByPosition(i).name << ", " << std::string(max_length_of_column_name - sample.safeGetByPosition(i).name.size(), ' ') << "type: " << data_types[i]->getName() << ", " << std::string(max_length_of_data_type_name - data_types[i]->getName().size(), ' '); - auto prev_position = istr.position(); - auto curr_position = istr.position(); + BufferBase::Position prev_position = istr.position(); + BufferBase::Position curr_position = istr.position(); std::exception_ptr exception; try diff --git a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h index 7896e96232d..5782742a425 100644 --- a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h +++ b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h @@ -15,11 +15,13 @@ namespace DB class CollapsingFinalBlockInputStream : public IProfilingBlockInputStream { public: - CollapsingFinalBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_, - const String & sign_column_name_) + CollapsingFinalBlockInputStream( + const BlockInputStreams & inputs, + const SortDescription & description_, + const String & sign_column_name_) : description(description_), sign_column_name(sign_column_name_) { - children.insert(children.end(), inputs_.begin(), inputs_.end()); + children.insert(children.end(), inputs.begin(), inputs.end()); } ~CollapsingFinalBlockInputStream(); @@ -55,10 +57,10 @@ private: struct MergingBlock : boost::noncopyable { - MergingBlock(Block block_, + MergingBlock(const Block & block_, size_t stream_index_, const SortDescription & desc, - String sign_column_name, + const String & sign_column_name, BlockPlainPtrs * output_blocks) : block(block_), stream_index(stream_index_), output_blocks(output_blocks) { diff --git a/dbms/src/DataStreams/CountingBlockOutputStream.h b/dbms/src/DataStreams/CountingBlockOutputStream.h index f0fac50f7f0..63ece36c2b0 100644 --- a/dbms/src/DataStreams/CountingBlockOutputStream.h +++ b/dbms/src/DataStreams/CountingBlockOutputStream.h @@ -16,7 +16,7 @@ public: CountingBlockOutputStream(const BlockOutputStreamPtr & stream_) : stream(stream_) {} - void setProgressCallback(ProgressCallback callback) + void setProgressCallback(const ProgressCallback & callback) { progress_callback = callback; } @@ -35,9 +35,9 @@ public: void writePrefix() override { stream->writePrefix(); } void writeSuffix() override { stream->writeSuffix(); } - void flush() override { stream->flush(); } + void flush() override { stream->flush(); } void onProgress(const Progress & progress) override { stream->onProgress(progress); } - String getContentType() const override { return stream->getContentType(); } + String getContentType() const override { return stream->getContentType(); } protected: diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 995b043983e..0d6ad4a097e 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -117,18 +117,21 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) if ((max_rows_to_transfer && rows_to_transfer > max_rows_to_transfer) || (max_bytes_to_transfer && bytes_to_transfer > max_bytes_to_transfer)) { - if (transfer_overflow_mode == OverflowMode::THROW) - throw Exception("IN/JOIN external table size limit exceeded." - " Rows: " + toString(rows_to_transfer) - + ", limit: " + toString(max_rows_to_transfer) - + ". Bytes: " + toString(bytes_to_transfer) - + ", limit: " + toString(max_bytes_to_transfer) + ".", - ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); - - if (transfer_overflow_mode == OverflowMode::BREAK) - done_with_table = true; - - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + switch (transfer_overflow_mode) + { + case OverflowMode::THROW: + throw Exception("IN/JOIN external table size limit exceeded." + " Rows: " + toString(rows_to_transfer) + + ", limit: " + toString(max_rows_to_transfer) + + ". Bytes: " + toString(bytes_to_transfer) + + ", limit: " + toString(max_bytes_to_transfer) + ".", + ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); + case OverflowMode::BREAK: + done_with_table = true; + break; + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } } } diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h index cae282e9970..605caac90bc 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h @@ -18,7 +18,7 @@ class CreatingSetsBlockInputStream : public IProfilingBlockInputStream { public: CreatingSetsBlockInputStream( - BlockInputStreamPtr input, + const BlockInputStreamPtr & input, const SubqueriesForSets & subqueries_for_sets_, const Limits & limits) : subqueries_for_sets(subqueries_for_sets_), diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.cpp b/dbms/src/DataStreams/DistinctBlockInputStream.cpp index d308eb9f17b..dbb6d5b63f8 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctBlockInputStream.cpp @@ -8,8 +8,8 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; } -DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_) - : columns_names(columns_) +DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns) + : columns_names(columns) , limit_hint(limit_hint_) , max_rows(limits.max_rows_in_distinct) , max_bytes(limits.max_bytes_in_distinct) @@ -68,18 +68,22 @@ Block DistinctBlockInputStream::readImpl() if (!checkLimits()) { - if (overflow_mode == OverflowMode::THROW) - throw Exception("DISTINCT-Set size limit exceeded." - " Rows: " + toString(data.getTotalRowCount()) + - ", limit: " + toString(max_rows) + - ". Bytes: " + toString(data.getTotalByteCount()) + - ", limit: " + toString(max_bytes) + ".", - ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); + switch (overflow_mode) + { + case OverflowMode::THROW: + throw Exception("DISTINCT-Set size limit exceeded." + " Rows: " + toString(data.getTotalRowCount()) + + ", limit: " + toString(max_rows) + + ". Bytes: " + toString(data.getTotalByteCount()) + + ", limit: " + toString(max_bytes) + ".", + ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); - if (overflow_mode == OverflowMode::BREAK) - return Block(); + case OverflowMode::BREAK: + return Block(); - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } } size_t all_columns = block.columns(); diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.h b/dbms/src/DataStreams/DistinctBlockInputStream.h index 7120d3e04a9..a422a05e8e3 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctBlockInputStream.h @@ -18,7 +18,7 @@ class DistinctBlockInputStream : public IProfilingBlockInputStream { public: /// Empty columns_ means all collumns. - DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_); + DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns); String getName() const override { return "Distinct"; } diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp index 2c8ab1e1d8e..7afa82fbcff 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp @@ -71,18 +71,22 @@ Block DistinctSortedBlockInputStream::readImpl() if (!checkLimits()) { - if (overflow_mode == OverflowMode::THROW) - throw Exception("DISTINCT-Set size limit exceeded." - " Rows: " + toString(data.getTotalRowCount()) + - ", limit: " + toString(max_rows) + - ". Bytes: " + toString(data.getTotalByteCount()) + - ", limit: " + toString(max_bytes) + ".", - ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); + switch (overflow_mode) + { + case OverflowMode::THROW: + throw Exception("DISTINCT-Set size limit exceeded." + " Rows: " + toString(data.getTotalRowCount()) + + ", limit: " + toString(max_rows) + + ". Bytes: " + toString(data.getTotalByteCount()) + + ", limit: " + toString(max_bytes) + ".", + ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); - if (overflow_mode == OverflowMode::BREAK) - return Block(); + case OverflowMode::BREAK: + return Block(); - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } } prev_block.block = block; diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp index 6130df31b51..369150796f7 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp @@ -5,7 +5,7 @@ namespace DB { -ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_) +ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_) : expression(expression_) { children.push_back(input); diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.h b/dbms/src/DataStreams/ExpressionBlockInputStream.h index c57ea928d54..11d498332a3 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.h +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.h @@ -19,7 +19,7 @@ private: using ExpressionActionsPtr = std::shared_ptr; public: - ExpressionBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_); + ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/FilterBlockInputStream.cpp b/dbms/src/DataStreams/FilterBlockInputStream.cpp index be0e9182c08..fa560a79419 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.cpp +++ b/dbms/src/DataStreams/FilterBlockInputStream.cpp @@ -17,13 +17,13 @@ namespace ErrorCodes } -FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, ssize_t filter_column_) +FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, ssize_t filter_column_) : expression(expression_), filter_column(filter_column_) { children.push_back(input); } -FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, const String & filter_column_name_) +FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_) : expression(expression_), filter_column(-1), filter_column_name(filter_column_name_) { children.push_back(input); diff --git a/dbms/src/DataStreams/FilterBlockInputStream.h b/dbms/src/DataStreams/FilterBlockInputStream.h index 95ade939256..651fab7f514 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.h +++ b/dbms/src/DataStreams/FilterBlockInputStream.h @@ -20,8 +20,8 @@ private: public: /// filter_column_ - the number of the column with filter conditions. - FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, ssize_t filter_column_); - FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, const String & filter_column_name_); + FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, ssize_t filter_column_); + FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/FilterColumnsBlockInputStream.h b/dbms/src/DataStreams/FilterColumnsBlockInputStream.h index dabf9ffb7bb..07a9f6bd093 100644 --- a/dbms/src/DataStreams/FilterColumnsBlockInputStream.h +++ b/dbms/src/DataStreams/FilterColumnsBlockInputStream.h @@ -13,10 +13,10 @@ class FilterColumnsBlockInputStream : public IProfilingBlockInputStream { public: FilterColumnsBlockInputStream( - BlockInputStreamPtr input_, const Names & columns_to_save_, bool throw_if_column_not_found_) + const BlockInputStreamPtr & input, const Names & columns_to_save_, bool throw_if_column_not_found_) : columns_to_save(columns_to_save_), throw_if_column_not_found(throw_if_column_not_found_) { - children.push_back(input_); + children.push_back(input); } String getName() const override diff --git a/dbms/src/DataStreams/IBlockInputStream.cpp b/dbms/src/DataStreams/IBlockInputStream.cpp index d3c6f06cab9..ac179115678 100644 --- a/dbms/src/DataStreams/IBlockInputStream.cpp +++ b/dbms/src/DataStreams/IBlockInputStream.cpp @@ -99,7 +99,7 @@ void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t mult BlockInputStreams IBlockInputStream::getLeaves() { BlockInputStreams res; - getLeavesImpl(res); + getLeavesImpl(res, nullptr); return res; } @@ -122,7 +122,7 @@ void IBlockInputStream::getLeafRowsBytes(size_t & rows, size_t & bytes) } -void IBlockInputStream::getLeavesImpl(BlockInputStreams & res, BlockInputStreamPtr this_shared_ptr) +void IBlockInputStream::getLeavesImpl(BlockInputStreams & res, const BlockInputStreamPtr & this_shared_ptr) { if (children.empty()) { diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index 68490b9be2b..fccccbd67e6 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -113,7 +113,7 @@ protected: BlockInputStreams children; private: - void getLeavesImpl(BlockInputStreams & res, BlockInputStreamPtr this_shared_ptr = nullptr); + void getLeavesImpl(BlockInputStreams & res, const BlockInputStreamPtr & this_shared_ptr); size_t checkDepthImpl(size_t max_depth, size_t level) const; diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 2c3e6603bf1..1b10f4899d7 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -142,53 +142,45 @@ void IProfilingBlockInputStream::updateExtremes(Block & block) bool IProfilingBlockInputStream::checkLimits() { + auto handle_overflow_mode = [this] (OverflowMode mode, const String & message, int code) + { + switch (mode) + { + case OverflowMode::THROW: + throw Exception(message, code); + case OverflowMode::BREAK: + return false; + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } + }; + if (limits.mode == LIMITS_CURRENT) { /// Check current stream limitations (i.e. max_result_{rows,bytes}) if (limits.max_rows_to_read && info.rows > limits.max_rows_to_read) - { - if (limits.read_overflow_mode == OverflowMode::THROW) - throw Exception(std::string("Limit for result rows") + return handle_overflow_mode(limits.read_overflow_mode, + std::string("Limit for result rows") + " exceeded: read " + toString(info.rows) + " rows, maximum: " + toString(limits.max_rows_to_read), - ErrorCodes::TOO_MUCH_ROWS); - - if (limits.read_overflow_mode == OverflowMode::BREAK) - return false; - - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); - } + ErrorCodes::TOO_MUCH_ROWS); if (limits.max_bytes_to_read && info.bytes > limits.max_bytes_to_read) - { - if (limits.read_overflow_mode == OverflowMode::THROW) - throw Exception(std::string("Limit for result bytes (uncompressed)") + return handle_overflow_mode(limits.read_overflow_mode, + std::string("Limit for result bytes (uncompressed)") + " exceeded: read " + toString(info.bytes) + " bytes, maximum: " + toString(limits.max_bytes_to_read), - ErrorCodes::TOO_MUCH_BYTES); - - if (limits.read_overflow_mode == OverflowMode::BREAK) - return false; - - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); - } + ErrorCodes::TOO_MUCH_BYTES); } if (limits.max_execution_time != 0 && info.total_stopwatch.elapsed() > static_cast(limits.max_execution_time.totalMicroseconds()) * 1000) - { - if (limits.timeout_overflow_mode == OverflowMode::THROW) - throw Exception("Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds()) + return handle_overflow_mode(limits.timeout_overflow_mode, + "Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds()) + " seconds, maximum: " + toString(limits.max_execution_time.totalMicroseconds() / 1000000.0), ErrorCodes::TIMEOUT_EXCEEDED); - if (limits.timeout_overflow_mode == OverflowMode::BREAK) - return false; - - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); - } - return true; } @@ -244,28 +236,36 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) && ((limits.max_rows_to_read && total_rows_estimate > limits.max_rows_to_read) || (limits.max_bytes_to_read && bytes_processed > limits.max_bytes_to_read))) { - if (limits.read_overflow_mode == OverflowMode::THROW) + switch (limits.read_overflow_mode) { - if (limits.max_rows_to_read && total_rows_estimate > limits.max_rows_to_read) - throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate) - + " rows read (or to read), maximum: " + toString(limits.max_rows_to_read), - ErrorCodes::TOO_MUCH_ROWS); - else - throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(bytes_processed) - + " bytes read, maximum: " + toString(limits.max_bytes_to_read), - ErrorCodes::TOO_MUCH_BYTES); - } - else if (limits.read_overflow_mode == OverflowMode::BREAK) - { - /// For `break`, we will stop only if so many lines were actually read, and not just supposed to be read. - if ((limits.max_rows_to_read && rows_processed > limits.max_rows_to_read) - || (limits.max_bytes_to_read && bytes_processed > limits.max_bytes_to_read)) + case OverflowMode::THROW: { - cancel(); + if (limits.max_rows_to_read && total_rows_estimate > limits.max_rows_to_read) + throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate) + + " rows read (or to read), maximum: " + toString(limits.max_rows_to_read), + ErrorCodes::TOO_MUCH_ROWS); + else + throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(bytes_processed) + + " bytes read, maximum: " + toString(limits.max_bytes_to_read), + ErrorCodes::TOO_MUCH_BYTES); + break; } + + case OverflowMode::BREAK: + { + /// For `break`, we will stop only if so many lines were actually read, and not just supposed to be read. + if ((limits.max_rows_to_read && rows_processed > limits.max_rows_to_read) + || (limits.max_bytes_to_read && bytes_processed > limits.max_bytes_to_read)) + { + cancel(); + } + + break; + } + + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); } - else - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); } size_t total_rows = process_list_elem->progress_in.total_rows; @@ -317,7 +317,7 @@ void IProfilingBlockInputStream::cancel() } -void IProfilingBlockInputStream::setProgressCallback(ProgressCallback callback) +void IProfilingBlockInputStream::setProgressCallback(const ProgressCallback & callback) { progress_callback = callback; diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h index a5a2b043a51..8fa9eb6731b 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.h @@ -69,7 +69,7 @@ public: * The function takes the number of rows in the last block, the number of bytes in the last block. * Note that the callback can be called from different threads. */ - void setProgressCallback(ProgressCallback callback); + void setProgressCallback(const ProgressCallback & callback); /** In this method: diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.cpp b/dbms/src/DataStreams/LimitByBlockInputStream.cpp index 6182dd3413b..ba25c1b93a6 100644 --- a/dbms/src/DataStreams/LimitByBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitByBlockInputStream.cpp @@ -3,8 +3,8 @@ namespace DB { -LimitByBlockInputStream::LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, Names columns_) - : columns_names(columns_) +LimitByBlockInputStream::LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, const Names & columns) + : columns_names(columns) , group_size(group_size_) { children.push_back(input); diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.h b/dbms/src/DataStreams/LimitByBlockInputStream.h index 54874d37f1e..96aea7092d1 100644 --- a/dbms/src/DataStreams/LimitByBlockInputStream.h +++ b/dbms/src/DataStreams/LimitByBlockInputStream.h @@ -18,7 +18,7 @@ namespace DB class LimitByBlockInputStream : public IProfilingBlockInputStream { public: - LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, Names columns_); + LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, const Names & columns); String getName() const override { return "LimitBy"; } diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index 1b1de6f90af..f1300ee12af 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -56,7 +56,7 @@ namespace DB * data from sources can also be read in several threads (reading_threads) * for optimal performance in the presence of a fast network or disks (from where these blocks are read). */ -class MergingAggregatedMemoryEfficientBlockInputStream : public IProfilingBlockInputStream +class MergingAggregatedMemoryEfficientBlockInputStream final : public IProfilingBlockInputStream { public: MergingAggregatedMemoryEfficientBlockInputStream( @@ -137,7 +137,7 @@ private: std::exception_ptr exception; /// It is necessary to give out blocks in the order of the key (bucket_num). /// If the value is an empty block, you need to wait for its merge. - /// (This means the promise that there will be data here, which is important because the data should be given out + /// (This means the promise that there will be data here, which is important because the data should be given out /// in the order of the key - bucket_num) std::map merged_blocks; std::mutex merged_blocks_mutex; diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp index c6e0dcb1a8e..fcc80071aba 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp @@ -16,8 +16,8 @@ namespace ErrorCodes TotalsHavingBlockInputStream::TotalsHavingBlockInputStream( - BlockInputStreamPtr input_, - bool overflow_row_, ExpressionActionsPtr expression_, + const BlockInputStreamPtr & input_, + bool overflow_row_, const ExpressionActionsPtr & expression_, const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_) : overflow_row(overflow_row_), expression(expression_), filter_column_name(filter_column_), totals_mode(totals_mode_), diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h index 42137947427..e2e79ee2b03 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h @@ -20,8 +20,8 @@ private: public: TotalsHavingBlockInputStream( - BlockInputStreamPtr input_, - bool overflow_row_, ExpressionActionsPtr expression_, + const BlockInputStreamPtr & input_, + bool overflow_row_, const ExpressionActionsPtr & expression_, const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_); String getName() const override { return "TotalsHaving"; } diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index fb13f6bab1e..f725bdad42b 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -29,13 +29,13 @@ namespace ErrorCodes } -DataTypeArray::DataTypeArray(DataTypePtr nested_) +DataTypeArray::DataTypeArray(const DataTypePtr & nested_) : enriched_nested(std::make_pair(nested_, std::make_shared())), nested{nested_} { offsets = std::make_shared>(); } -DataTypeArray::DataTypeArray(DataTypeTraits::EnrichedDataTypePtr enriched_nested_) +DataTypeArray::DataTypeArray(const DataTypeTraits::EnrichedDataTypePtr & enriched_nested_) : enriched_nested{enriched_nested_}, nested{enriched_nested.first} { offsets = std::make_shared>(); diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index 423d27e1889..012d73fb3f5 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -19,8 +19,8 @@ private: DataTypePtr offsets; public: - DataTypeArray(DataTypePtr nested_); - DataTypeArray(DataTypeTraits::EnrichedDataTypePtr enriched_nested_); + DataTypeArray(const DataTypePtr & nested_); + DataTypeArray(const DataTypeTraits::EnrichedDataTypePtr & enriched_nested_); std::string getName() const override { diff --git a/dbms/src/DataTypes/DataTypeExpression.h b/dbms/src/DataTypes/DataTypeExpression.h index d9f1bbe7598..5bf0752217f 100644 --- a/dbms/src/DataTypes/DataTypeExpression.h +++ b/dbms/src/DataTypes/DataTypeExpression.h @@ -16,7 +16,7 @@ private: public: /// Some types could be still unknown. - DataTypeExpression(DataTypes argument_types_ = DataTypes(), DataTypePtr return_type_ = nullptr) + DataTypeExpression(const DataTypes & argument_types_ = DataTypes(), const DataTypePtr & return_type_ = nullptr) : argument_types(argument_types_), return_type(return_type_) {} std::string getName() const override; diff --git a/dbms/src/DataTypes/DataTypeNested.cpp b/dbms/src/DataTypes/DataTypeNested.cpp index bf81432198b..4f51b302048 100644 --- a/dbms/src/DataTypes/DataTypeNested.cpp +++ b/dbms/src/DataTypes/DataTypeNested.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes } -DataTypeNested::DataTypeNested(NamesAndTypesListPtr nested_) +DataTypeNested::DataTypeNested(const NamesAndTypesListPtr & nested_) : nested(nested_) { } diff --git a/dbms/src/DataTypes/DataTypeNested.h b/dbms/src/DataTypes/DataTypeNested.h index 0af35a7912e..b086450b4ef 100644 --- a/dbms/src/DataTypes/DataTypeNested.h +++ b/dbms/src/DataTypes/DataTypeNested.h @@ -17,7 +17,7 @@ private: NamesAndTypesListPtr nested; public: - DataTypeNested(NamesAndTypesListPtr nested_); + DataTypeNested(const NamesAndTypesListPtr & nested_); std::string getName() const override; const char * getFamilyName() const override { return "Nested"; } diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index d26462c4721..40fb3ba77c7 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes } -DataTypeNullable::DataTypeNullable(DataTypePtr nested_data_type_) +DataTypeNullable::DataTypeNullable(const DataTypePtr & nested_data_type_) : nested_data_type{nested_data_type_} { if (!nested_data_type->canBeInsideNullable()) diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index 07e9dd8a9fa..4f1fdf902bc 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -11,7 +11,7 @@ namespace DB class DataTypeNullable final : public IDataType { public: - DataTypeNullable(DataTypePtr nested_data_type_); + DataTypeNullable(const DataTypePtr & nested_data_type_); std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } bool isNullable() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index 10813276057..498e246a6e6 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -16,7 +16,7 @@ class DataTypeTuple final : public IDataType private: DataTypes elems; public: - DataTypeTuple(DataTypes elems_) : elems(elems_) {} + DataTypeTuple(const DataTypes & elems_) : elems(elems_) {} std::string getName() const override; const char * getFamilyName() const override { return "Tuple"; } diff --git a/dbms/src/Dictionaries/DictionaryBlockInputStream.h b/dbms/src/Dictionaries/DictionaryBlockInputStream.h index 1d80b32c65a..a2f3fb31584 100644 --- a/dbms/src/Dictionaries/DictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/DictionaryBlockInputStream.h @@ -250,7 +250,7 @@ Block DictionaryBlockInputStream::fillBlock( data_types.reserve(keys.size()); const DictionaryStructure& dictionaty_structure = dictionary->getStructure(); if (data_types.empty() && dictionaty_structure.key) - for (const auto key : *dictionaty_structure.key) + for (const auto & key : *dictionaty_structure.key) data_types.push_back(key.type); for (const auto & column : view) diff --git a/dbms/src/Dictionaries/DictionaryStructure.cpp b/dbms/src/Dictionaries/DictionaryStructure.cpp index 8c8ce000b6e..f96c0f9262b 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.cpp +++ b/dbms/src/Dictionaries/DictionaryStructure.cpp @@ -220,7 +220,7 @@ bool DictionaryStructure::isKeySizeFixed() const if (!key) return true; - for (const auto key_i : * key) + for (const auto & key_i : *key) if (key_i.underlying_type == AttributeUnderlyingType::String) return false; diff --git a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp index c34a189a31e..a3a3c40ba65 100644 --- a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp @@ -54,7 +54,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll() /** A stream, that also runs and waits for background thread * (that will feed data into pipe to be read from the other side of the pipe). */ -class BlockInputStreamWithBackgroundThread : public IProfilingBlockInputStream +class BlockInputStreamWithBackgroundThread final : public IProfilingBlockInputStream { public: BlockInputStreamWithBackgroundThread( diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index d851d484757..973defc855f 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -366,7 +366,7 @@ void FlatDictionary::createAttributeImpl(Attribute & attribute, const Fi { attribute.string_arena = std::make_unique(); auto & null_value_ref = std::get(attribute.null_values); - const String string = null_value.get::Type>(); + const String & string = null_value.get::Type>(); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); null_value_ref = StringRef{string_in_arena, string.size()}; std::get>(attribute.arrays) = diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index be4badb2af0..e4f544a3b05 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1018,16 +1018,20 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const { if (!no_more_keys && params.max_rows_to_group_by && result_size > params.max_rows_to_group_by) { - if (params.group_by_overflow_mode == OverflowMode::THROW) - throw Exception("Limit for rows to GROUP BY exceeded: has " + toString(result_size) - + " rows, maximum: " + toString(params.max_rows_to_group_by), - ErrorCodes::TOO_MUCH_ROWS); - else if (params.group_by_overflow_mode == OverflowMode::BREAK) - return false; - else if (params.group_by_overflow_mode == OverflowMode::ANY) - no_more_keys = true; - else - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + switch (params.group_by_overflow_mode) + { + case OverflowMode::THROW: + throw Exception("Limit for rows to GROUP BY exceeded: has " + toString(result_size) + + " rows, maximum: " + toString(params.max_rows_to_group_by), + ErrorCodes::TOO_MUCH_ROWS); + + case OverflowMode::BREAK: + return false; + + case OverflowMode::ANY: + no_more_keys = true; + break; + } } return true; diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index e798c8add55..c6283ebf8d3 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -37,7 +37,7 @@ Compiler::Compiler(const std::string & path_, size_t threads) Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator dir_it(path); dir_end != dir_it; ++dir_it) { - std::string name = dir_it.name(); + const std::string & name = dir_it.name(); if (endsWith(name, ".so")) { files.insert(name.substr(0, name.size() - 3)); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 06900b87037..a2812495c69 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -770,7 +770,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta } -void Context::addExternalTable(const String & table_name, StoragePtr storage) +void Context::addExternalTable(const String & table_name, const StoragePtr & storage) { if (external_tables.end() != external_tables.find(table_name)) throw Exception("Temporary table " + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); @@ -1217,7 +1217,7 @@ void Context::setZooKeeper(zkutil::ZooKeeperPtr zookeeper) if (shared->zookeeper) throw Exception("ZooKeeper client has already been set.", ErrorCodes::LOGICAL_ERROR); - shared->zookeeper = zookeeper; + shared->zookeeper = std::move(zookeeper); } zkutil::ZooKeeperPtr Context::getZooKeeper() const diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 33087ff71b0..8795c5a9799 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -173,7 +173,7 @@ public: StoragePtr tryGetExternalTable(const String & table_name) const; StoragePtr getTable(const String & database_name, const String & table_name) const; StoragePtr tryGetTable(const String & database_name, const String & table_name) const; - void addExternalTable(const String & table_name, StoragePtr storage); + void addExternalTable(const String & table_name, const StoragePtr & storage); StoragePtr tryRemoveExternalTable(const String & table_name); void addDatabase(const String & database_name, const DatabasePtr & database); diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 84556ded0c2..cbeab9f8d5d 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -237,7 +237,7 @@ struct ExpressionActionsChain ExpressionActionsPtr actions; Names required_output; - Step(ExpressionActionsPtr actions_ = nullptr, Names required_output_ = Names()) + Step(const ExpressionActionsPtr & actions_ = nullptr, const Names & required_output_ = Names()) : actions(actions_), required_output(required_output_) {} }; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index e3cafeb0e3e..bdfaf4f6713 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -153,7 +153,7 @@ void removeDuplicateColumns(NamesAndTypesList & columns) ExpressionAnalyzer::ExpressionAnalyzer( const ASTPtr & ast_, const Context & context_, - StoragePtr storage_, + const StoragePtr & storage_, const NamesAndTypesList & columns_, size_t subquery_depth_, bool do_global_) @@ -1853,7 +1853,7 @@ struct ExpressionAnalyzer::ScopeStack }; -void ExpressionAnalyzer::getRootActions(ASTPtr ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions) +void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions) { ScopeStack scopes(actions, settings); getActionsImpl(ast, no_subqueries, only_consts, scopes); @@ -1919,7 +1919,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns() /// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that. -void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast) +void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast) { if (typeid_cast(ast.get())) return; @@ -1973,7 +1973,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast) } -void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack) +void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack) { /// If the result of the calculation already exists in the block. if ((typeid_cast(ast.get()) || typeid_cast(ast.get())) @@ -2615,7 +2615,7 @@ Block ExpressionAnalyzer::getSelectSampleBlock() return temp_actions->getSampleBlock(); } -void ExpressionAnalyzer::getActionsBeforeAggregation(ASTPtr ast, ExpressionActionsPtr & actions, bool no_subqueries) +void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries) { ASTFunction * node = typeid_cast(ast.get()); @@ -2848,7 +2848,7 @@ Names ExpressionAnalyzer::getRequiredColumns() } -void ExpressionAnalyzer::getRequiredColumnsImpl(ASTPtr ast, +void ExpressionAnalyzer::getRequiredColumnsImpl(const ASTPtr & ast, NameSet & required_columns, NameSet & ignored_names, const NameSet & available_joined_columns, NameSet & required_joined_columns) { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 44b5b6d78ee..52ebbb91d3a 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -69,7 +69,7 @@ public: ExpressionAnalyzer( const ASTPtr & ast_, const Context & context_, - StoragePtr storage_, + const StoragePtr & storage_, const NamesAndTypesList & columns_, size_t subquery_depth_ = 0, bool do_global_ = false); @@ -269,17 +269,17 @@ private: void addExternalStorage(ASTPtr & subquery_or_table_name); void getArrayJoinedColumns(); - void getArrayJoinedColumnsImpl(ASTPtr ast); + void getArrayJoinedColumnsImpl(const ASTPtr & ast); void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const; void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; struct ScopeStack; - void getActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack); + void getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack); - void getRootActions(ASTPtr ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions); + void getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions); - void getActionsBeforeAggregation(ASTPtr ast, ExpressionActionsPtr & actions, bool no_subqueries); + void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries); /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, @@ -295,7 +295,7 @@ private: * The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table. * Put in required_joined_columns the set of columns available from JOIN and needed. */ - void getRequiredColumnsImpl(ASTPtr ast, + void getRequiredColumnsImpl(const ASTPtr & ast, NameSet & required_columns, NameSet & ignored_names, const NameSet & available_joined_columns, NameSet & required_joined_columns); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index da2a9e2e0d4..dcbb35424db 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -167,7 +167,7 @@ void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input) query_analyzer = std::make_unique(query_ptr, context, storage, table_column_names, subquery_depth, !only_analyze); /// Save the new temporary tables in the query context - for (auto & it : query_analyzer->getExternalTables()) + for (const auto & it : query_analyzer->getExternalTables()) if (!context.tryGetExternalTable(it.first)) context.addExternalTable(it.first, it.second); diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index a7139fe68b1..28f43d6d213 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -525,18 +525,22 @@ bool Join::insertFromBlock(const Block & block) if (!checkSizeLimits()) { - if (overflow_mode == OverflowMode::THROW) - throw Exception("Join size limit exceeded." - " Rows: " + toString(getTotalRowCount()) + - ", limit: " + toString(max_rows) + - ". Bytes: " + toString(getTotalByteCount()) + - ", limit: " + toString(max_bytes) + ".", - ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); + switch (overflow_mode) + { + case OverflowMode::THROW: + throw Exception("Join size limit exceeded." + " Rows: " + toString(getTotalRowCount()) + + ", limit: " + toString(max_rows) + + ". Bytes: " + toString(getTotalByteCount()) + + ", limit: " + toString(max_bytes) + ".", + ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); - if (overflow_mode == OverflowMode::BREAK) - return false; + case OverflowMode::BREAK: + return false; - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } } return true; diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index afa61220a6d..0c7e0ef0ea0 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -206,7 +206,7 @@ bool ProcessListElement::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutpu } -void ProcessList::addTemporaryTable(ProcessListElement & elem, const String & table_name, StoragePtr storage) +void ProcessList::addTemporaryTable(ProcessListElement & elem, const String & table_name, const StoragePtr & storage) { std::lock_guard lock(mutex); diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index c853f4a06bc..6bc31a7ecf5 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -274,7 +274,7 @@ public: } /// Register temporary table. Then it is accessible by query_id and name. - void addTemporaryTable(ProcessListElement & elem, const String & table_name, StoragePtr storage); + void addTemporaryTable(ProcessListElement & elem, const String & table_name, const StoragePtr & storage); enum class CancellationCode { diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 4939e245f71..d93097c0fc9 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -181,18 +181,22 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set) if (!checkSetSizeLimits()) { - if (overflow_mode == OverflowMode::THROW) - throw Exception("IN-set size exceeded." - " Rows: " + toString(data.getTotalRowCount()) + - ", limit: " + toString(max_rows) + - ". Bytes: " + toString(data.getTotalByteCount()) + - ", limit: " + toString(max_bytes) + ".", - ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); + switch (overflow_mode) + { + case OverflowMode::THROW: + throw Exception("IN-set size exceeded." + " Rows: " + toString(data.getTotalRowCount()) + + ", limit: " + toString(max_rows) + + ". Bytes: " + toString(data.getTotalByteCount()) + + ", limit: " + toString(max_bytes) + ".", + ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); - if (overflow_mode == OverflowMode::BREAK) - return false; + case OverflowMode::BREAK: + return false; - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } } return true; diff --git a/dbms/src/Storages/MergeTree/PKCondition.cpp b/dbms/src/Storages/MergeTree/PKCondition.cpp index 7eb95b44aab..1e5cef69ef9 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.cpp +++ b/dbms/src/Storages/MergeTree/PKCondition.cpp @@ -223,7 +223,7 @@ PKCondition::PKCondition( const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_, - ExpressionActionsPtr pk_expr_) + const ExpressionActionsPtr & pk_expr_) : sort_descr(sort_descr_), pk_expr(pk_expr_), prepared_sets(query_info.sets) { for (size_t i = 0; i < sort_descr.size(); ++i) diff --git a/dbms/src/Storages/MergeTree/PKCondition.h b/dbms/src/Storages/MergeTree/PKCondition.h index 5c48df5becc..07deeb2ceaf 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.h +++ b/dbms/src/Storages/MergeTree/PKCondition.h @@ -207,7 +207,7 @@ public: const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr, - ExpressionActionsPtr pk_expr); + const ExpressionActionsPtr & pk_expr); /// Whether the condition is feasible in the key range. /// left_pk and right_pk must contain all fields in the sort_descr in the appropriate order. From 8c9ec83a33e3d3174df649be423c2886d170e707 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 07:10:43 +0300 Subject: [PATCH 058/108] Implemented some suggestions from clang-tidy (part 4) [#CLICKHOUSE-3301]. --- dbms/src/Common/tests/compact_array.cpp | 3 --- dbms/src/DataStreams/CollapsingFinalBlockInputStream.h | 2 +- dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp | 4 ++-- dbms/src/DataStreams/DistinctSortedBlockInputStream.h | 2 +- dbms/src/DataStreams/IProfilingBlockInputStream.cpp | 2 +- dbms/tests/instructions/clang-tidy.txt | 1 + 6 files changed, 6 insertions(+), 8 deletions(-) diff --git a/dbms/src/Common/tests/compact_array.cpp b/dbms/src/Common/tests/compact_array.cpp index e83496514a1..89ed1068abb 100644 --- a/dbms/src/Common/tests/compact_array.cpp +++ b/dbms/src/Common/tests/compact_array.cpp @@ -52,9 +52,6 @@ struct Test { DB::WriteBufferFromFile wb(filename); wb.write(reinterpret_cast(&store), sizeof(store)); - const unsigned char * p = reinterpret_cast(&store); - for (size_t i = 0; i < sizeof(store); ++i) - ++*p; } { diff --git a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h index 5782742a425..89a87fe41e3 100644 --- a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h +++ b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h @@ -181,7 +181,7 @@ private: size_t pos; Cursor() {} - explicit Cursor(MergingBlockPtr block_, size_t pos_ = 0) : block(block_), pos(pos_) {} + explicit Cursor(const MergingBlockPtr & block_, size_t pos_ = 0) : block(block_), pos(pos_) {} bool operator< (const Cursor & rhs) const { diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp index 7afa82fbcff..166f6c2dd6f 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp @@ -8,9 +8,9 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; } -DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_) +DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns) : description(input->getSortDescription()) - , columns_names(columns_) + , columns_names(columns) , limit_hint(limit_hint_) , max_rows(limits.max_rows_in_distinct) , max_bytes(limits.max_bytes_in_distinct) diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h index 28601f4bb04..dae1e489e18 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h @@ -21,7 +21,7 @@ class DistinctSortedBlockInputStream : public IProfilingBlockInputStream { public: /// Empty columns_ means all collumns. - DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_); + DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns); String getName() const override { return "DistinctSorted"; } diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 1b10f4899d7..beacaa7f79d 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -142,7 +142,7 @@ void IProfilingBlockInputStream::updateExtremes(Block & block) bool IProfilingBlockInputStream::checkLimits() { - auto handle_overflow_mode = [this] (OverflowMode mode, const String & message, int code) + auto handle_overflow_mode = [] (OverflowMode mode, const String & message, int code) { switch (mode) { diff --git a/dbms/tests/instructions/clang-tidy.txt b/dbms/tests/instructions/clang-tidy.txt index cc6cc2e9556..27234f7a101 100644 --- a/dbms/tests/instructions/clang-tidy.txt +++ b/dbms/tests/instructions/clang-tidy.txt @@ -1,3 +1,4 @@ +# http://clang.llvm.org/extra/clang-tidy/ # Install latest clang with extra tools. # Look at utils/prepare-environment/install-clang.sh From b282c8fcbbf05ddd3e5fc78c48628bdd3bafd0fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 07:11:39 +0300 Subject: [PATCH 059/108] Implemented some suggestions from clang-tidy (part 5) [#CLICKHOUSE-3301]. --- dbms/src/Common/tests/lru_cache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/tests/lru_cache.cpp b/dbms/src/Common/tests/lru_cache.cpp index 90de5903673..70c4eed2132 100644 --- a/dbms/src/Common/tests/lru_cache.cpp +++ b/dbms/src/Common/tests/lru_cache.cpp @@ -12,7 +12,7 @@ namespace { void run(); -void runTest(unsigned int num, const std::function func); +void runTest(unsigned int num, const std::function & func); bool test1(); bool test2(); bool test_concurrent(); From 5993e62f5bb5535817f49b2fdb7935a93622b57b Mon Sep 17 00:00:00 2001 From: robot-metrika-test Date: Fri, 8 Sep 2017 07:26:49 +0300 Subject: [PATCH 060/108] Auto version update to [54288] --- dbms/cmake/version.cmake | 4 ++-- debian/changelog | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 41e08de697d..7486c7f884c 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -1,6 +1,6 @@ # This strings autochanged from release_lib.sh: -set(VERSION_DESCRIBE v1.1.54287-testing) -set(VERSION_REVISION 54287) +set(VERSION_DESCRIBE v1.1.54288-testing) +set(VERSION_REVISION 54288) # end of autochange set (VERSION_MAJOR 1) diff --git a/debian/changelog b/debian/changelog index 2d309002fe3..5c978faf8f1 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (1.1.54287) unstable; urgency=low +clickhouse (1.1.54288) unstable; urgency=low * Modified source code - -- Wed, 06 Sep 2017 23:49:16 +0300 + -- Fri, 08 Sep 2017 07:26:49 +0300 From 4799f28dadc31f4c967f5304cf5465c307ee1573 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 07:39:03 +0300 Subject: [PATCH 061/108] Implemented some suggestions from clang-tidy (part 6) [#CLICKHOUSE-3301]. --- dbms/src/Common/Exception.cpp | 6 +++--- dbms/src/Interpreters/NullableUtils.cpp | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Common/Exception.cpp b/dbms/src/Common/Exception.cpp index 8a512f48f89..9a14ab72d19 100644 --- a/dbms/src/Common/Exception.cpp +++ b/dbms/src/Common/Exception.cpp @@ -166,7 +166,7 @@ void tryLogException(std::exception_ptr e, const char * log_name, const std::str { try { - std::rethrow_exception(e); + std::rethrow_exception(std::move(e)); } catch (...) { @@ -178,7 +178,7 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str { try { - std::rethrow_exception(e); + std::rethrow_exception(std::move(e)); } catch (...) { @@ -220,7 +220,7 @@ std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) { try { - std::rethrow_exception(e); + std::rethrow_exception(std::move(e)); } catch (...) { diff --git a/dbms/src/Interpreters/NullableUtils.cpp b/dbms/src/Interpreters/NullableUtils.cpp index da8747bcd40..f221eb1f9eb 100644 --- a/dbms/src/Interpreters/NullableUtils.cpp +++ b/dbms/src/Interpreters/NullableUtils.cpp @@ -34,6 +34,7 @@ void extractNestedColumnsAndNullMap(ConstColumnPlainPtrs & key_columns, ColumnPt } else { + mutable_null_map = &static_cast(*null_map_holder).getData(); const PaddedPODArray & other_null_map = column_nullable.getNullMap(); for (size_t i = 0, size = mutable_null_map->size(); i < size; ++i) (*mutable_null_map)[i] |= other_null_map[i]; From 71d29acdfb13431e253bfffa8fd0c769e8df5d7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 07:43:57 +0300 Subject: [PATCH 062/108] Implemented some suggestions from clang-tidy (part 7) [#CLICKHOUSE-3301]. --- dbms/src/Common/Exception.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index 756434a73ef..0cbe58b4caf 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -59,7 +59,7 @@ private: using Exceptions = std::vector; -void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno); +[[noreturn]] void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno); /** Try to write an exception to the log (and forget about it). From dc4976f5625c395d43f3fe40215d75f29c47db15 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 07:58:57 +0300 Subject: [PATCH 063/108] Implemented some suggestions from clang-tidy (part 8) [#CLICKHOUSE-3301]. --- dbms/src/Columns/ColumnFixedString.cpp | 2 +- dbms/src/Common/VirtualColumnUtils.cpp | 4 ++-- dbms/src/Common/localBackup.cpp | 4 ++-- dbms/src/Common/localBackup.h | 2 +- dbms/src/Common/tests/hash_table.cpp | 16 ---------------- dbms/src/DataStreams/UnionBlockInputStream.h | 2 +- dbms/src/Functions/IFunction.cpp | 4 ++-- dbms/src/IO/UncompressedCache.h | 2 +- dbms/src/Storages/StorageLog.cpp | 4 ++-- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageStripeLog.cpp | 4 ++-- dbms/src/Storages/StorageTinyLog.cpp | 4 ++-- 12 files changed, 17 insertions(+), 33 deletions(-) diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 946763f3f7a..ad846d45603 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -196,9 +196,9 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result } else { + size_t res_chars_size = res->chars.size(); for (size_t i = 0; i < SIMD_BYTES; ++i) { - size_t res_chars_size = res->chars.size(); if (filt_pos[i]) { res->chars.resize(res_chars_size + n); diff --git a/dbms/src/Common/VirtualColumnUtils.cpp b/dbms/src/Common/VirtualColumnUtils.cpp index 448281deeda..c403372c000 100644 --- a/dbms/src/Common/VirtualColumnUtils.cpp +++ b/dbms/src/Common/VirtualColumnUtils.cpp @@ -88,7 +88,7 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va } /// Verifying that the function depends only on the specified columns -static bool isValidFunction(ASTPtr expression, const NameSet & columns) +static bool isValidFunction(const ASTPtr & expression, const NameSet & columns) { for (size_t i = 0; i < expression->children.size(); ++i) if (!isValidFunction(expression->children[i], columns)) @@ -103,7 +103,7 @@ static bool isValidFunction(ASTPtr expression, const NameSet & columns) } /// Extract all subfunctions of the main conjunction, but depending only on the specified columns -static void extractFunctions(ASTPtr expression, const NameSet & columns, std::vector & result) +static void extractFunctions(const ASTPtr & expression, const NameSet & columns, std::vector & result) { const ASTFunction * function = typeid_cast(&*expression); if (function && function->name == "and") diff --git a/dbms/src/Common/localBackup.cpp b/dbms/src/Common/localBackup.cpp index da9cf9988c2..3b93e0ee452 100644 --- a/dbms/src/Common/localBackup.cpp +++ b/dbms/src/Common/localBackup.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes } -static void localBackupImpl(Poco::Path source_path, Poco::Path destination_path, size_t level) +static void localBackupImpl(const Poco::Path & source_path, const Poco::Path & destination_path, size_t level) { if (level >= 1000) throw DB::Exception("Too deep recursion", DB::ErrorCodes::TOO_DEEP_RECURSION); @@ -73,7 +73,7 @@ static void localBackupImpl(Poco::Path source_path, Poco::Path destination_path, } } -void localBackup(Poco::Path source_path, Poco::Path destination_path) +void localBackup(const Poco::Path & source_path, const Poco::Path & destination_path) { if (Poco::File(destination_path).exists() && Poco::DirectoryIterator(destination_path) != Poco::DirectoryIterator()) diff --git a/dbms/src/Common/localBackup.h b/dbms/src/Common/localBackup.h index 91107294e26..ccb6a24316e 100644 --- a/dbms/src/Common/localBackup.h +++ b/dbms/src/Common/localBackup.h @@ -13,4 +13,4 @@ * and is intended to be used as a simple means of protection against a human or program error, * but not from a hardware failure. */ -void localBackup(Poco::Path source_path, Poco::Path destination_path); +void localBackup(const Poco::Path & source_path, const Poco::Path & destination_path); diff --git a/dbms/src/Common/tests/hash_table.cpp b/dbms/src/Common/tests/hash_table.cpp index 86289d440a3..26c8e676546 100644 --- a/dbms/src/Common/tests/hash_table.cpp +++ b/dbms/src/Common/tests/hash_table.cpp @@ -35,22 +35,6 @@ int main(int argc, char ** argv) std::cerr << "dump: " << wb.str() << std::endl; } - { - using Cont = HashMap, HashTableGrower<1>>; - Cont cont; - - cont.insert(Cont::value_type(1, "Hello, world!")); - cont[1] = "Goodbye."; - - for (auto x : cont) - std::cerr << x.first << " -> " << x.second << std::endl; - - DB::WriteBufferFromOwnString wb; - cont.writeText(wb); - - std::cerr << "dump: " << wb.str() << std::endl; - } - { using Cont = HashSet< DB::UInt128, diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index 2267a8ae218..b72344cdede 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -63,7 +63,7 @@ struct OutputData */ template -class UnionBlockInputStream : public IProfilingBlockInputStream +class UnionBlockInputStream final : public IProfilingBlockInputStream { public: using ExceptionCallback = std::function; diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 6fe66910cfe..b2c00d62d77 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -107,7 +107,7 @@ ColumnsWithTypeAndName toNestedColumns(const ColumnsWithTypeAndName & args) auto nullable_col = static_cast(arg.column.get()); ColumnPtr nested_col = (nullable_col) ? nullable_col->getNestedColumn() : nullptr; auto nullable_type = static_cast(arg.type.get()); - DataTypePtr nested_type = nullable_type->getNestedType(); + const DataTypePtr & nested_type = nullable_type->getNestedType(); new_args.emplace_back(nested_col, nested_type, arg.name); } @@ -129,7 +129,7 @@ DataTypes toNestedDataTypes(const DataTypes & args) if (arg->isNullable()) { auto nullable_type = static_cast(arg.get()); - DataTypePtr nested_type = nullable_type->getNestedType(); + const DataTypePtr & nested_type = nullable_type->getNestedType(); new_args.push_back(nested_type); } else diff --git a/dbms/src/IO/UncompressedCache.h b/dbms/src/IO/UncompressedCache.h index 05fbb0f2b31..08875403594 100644 --- a/dbms/src/IO/UncompressedCache.h +++ b/dbms/src/IO/UncompressedCache.h @@ -69,7 +69,7 @@ public: return res; } - void set(const Key & key, MappedPtr mapped) + void set(const Key & key, const MappedPtr & mapped) { Base::set(key, mapped); ProfileEvents::increment(ProfileEvents::UncompressedCacheWeightLost, current_weight_lost); diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 9ea51f86f31..9361121844f 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -49,7 +49,7 @@ namespace ErrorCodes } -class LogBlockInputStream : public IProfilingBlockInputStream +class LogBlockInputStream final : public IProfilingBlockInputStream { public: LogBlockInputStream( @@ -130,7 +130,7 @@ private: }; -class LogBlockOutputStream : public IBlockOutputStream +class LogBlockOutputStream final : public IBlockOutputStream { public: explicit LogBlockOutputStream(StorageLog & storage_) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 3b9acc41af7..2ccbc8a59ca 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -500,7 +500,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & query, const Field & field ActiveDataPartSet active_parts; for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it) { - String name = it.name(); + const String & name = it.name(); MergeTreePartInfo part_info; if (!MergeTreePartInfo::tryParsePartName(name, &part_info) || part_info.partition_id != partition_id) diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index d7e4cc7fff5..351daa0e999 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -40,7 +40,7 @@ namespace ErrorCodes } -class StripeLogBlockInputStream : public IProfilingBlockInputStream +class StripeLogBlockInputStream final : public IProfilingBlockInputStream { public: StripeLogBlockInputStream(const NameSet & column_names_, StorageStripeLog & storage_, size_t max_read_buffer_size_, @@ -111,7 +111,7 @@ private: }; -class StripeLogBlockOutputStream : public IBlockOutputStream +class StripeLogBlockOutputStream final : public IBlockOutputStream { public: explicit StripeLogBlockOutputStream(StorageStripeLog & storage_) diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 9435d45094c..f0ce7d48281 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -52,7 +52,7 @@ namespace ErrorCodes } -class TinyLogBlockInputStream : public IProfilingBlockInputStream +class TinyLogBlockInputStream final : public IProfilingBlockInputStream { public: TinyLogBlockInputStream(size_t block_size_, const Names & column_names_, StorageTinyLog & storage_, size_t max_read_buffer_size_) @@ -93,7 +93,7 @@ private: }; -class TinyLogBlockOutputStream : public IBlockOutputStream +class TinyLogBlockOutputStream final : public IBlockOutputStream { public: explicit TinyLogBlockOutputStream(StorageTinyLog & storage_) From a81028377ebdfe4e8ca98cceed6bb1ddced00718 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 08:03:21 +0300 Subject: [PATCH 064/108] Implemented some suggestions from clang-tidy (part 9) [#CLICKHOUSE-3301]. --- dbms/src/IO/tests/valid_utf8.cpp | 8 ++++---- dbms/src/Storages/tests/seek_speed_test.cpp | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/IO/tests/valid_utf8.cpp b/dbms/src/IO/tests/valid_utf8.cpp index ccf176386f9..7033574b99d 100644 --- a/dbms/src/IO/tests/valid_utf8.cpp +++ b/dbms/src/IO/tests/valid_utf8.cpp @@ -13,10 +13,10 @@ int main(int argc, char ** argv) { std::string test1 = "kjhsgdfkjhg2378rtzgvxkz877%^&^*%&^*&*"; std::string test2 = "{\"asd\" = \"qw1\",\"qwe24\" = \"3asd\"}"; - test2[test2.find("1")] = char(127 + 64); - test2[test2.find("2")] = char(127 + 64 + 32); - test2[test2.find("3")] = char(127 + 64 + 32 + 16); - test2[test2.find("4")] = char(127 + 64 + 32 + 16 + 8); + test2[test2.find('1')] = char(127 + 64); + test2[test2.find('2')] = char(127 + 64 + 32); + test2[test2.find('3')] = char(127 + 64 + 32 + 16); + test2[test2.find('4')] = char(127 + 64 + 32 + 16 + 8); std::string str; { diff --git a/dbms/src/Storages/tests/seek_speed_test.cpp b/dbms/src/Storages/tests/seek_speed_test.cpp index bb06f7b95bf..3d5e8bcc6d2 100644 --- a/dbms/src/Storages/tests/seek_speed_test.cpp +++ b/dbms/src/Storages/tests/seek_speed_test.cpp @@ -33,7 +33,7 @@ int main(int argc, const char ** argv) UInt64 size = Poco::File(argv[1]).getSize(); UInt64 pos = 0; DB::ReadBufferFromFile in(argv[1], buf_size); - char * buf = new char[block]; + auto buf = std::make_unique(block); int checksum = 0; UInt64 bytes_read = 0; From 5d3cc618913c62a5e9b6756f967ae76076c86915 Mon Sep 17 00:00:00 2001 From: Vadim Skipin Date: Fri, 8 Sep 2017 14:57:43 +0300 Subject: [PATCH 065/108] CLICKHOUSE-3289: Do not use Application singleton #2 --- dbms/src/IO/HTTPCommon.cpp | 8 ++++---- dbms/src/IO/HTTPCommon.h | 2 +- dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp | 11 ++++++++--- dbms/src/IO/WriteBufferFromHTTPServerResponse.h | 2 ++ dbms/src/Server/HTTPHandler.cpp | 9 +++++++-- dbms/src/Server/HTTPHandlerFactory.h | 2 +- dbms/src/Server/InterserverIOHTTPHandler.cpp | 5 ++++- dbms/src/Server/PingRequestHandler.cpp | 4 +++- dbms/src/Server/PingRequestHandler.h | 9 +++++++++ dbms/src/Server/ReplicasStatusHandler.cpp | 3 ++- dbms/src/Server/RootRequestHandler.cpp | 7 +++++-- dbms/src/Server/RootRequestHandler.h | 4 ++-- 12 files changed, 48 insertions(+), 18 deletions(-) diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index 7e06377f002..6aea408eaa5 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -15,14 +15,14 @@ namespace DB { -void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response) +void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, int keep_alive_timeout) { if (!response.getKeepAlive()) return; - Poco::Timespan keep_alive_timeout(Poco::Util::Application::instance().config().getInt("keep_alive_timeout", 10), 0); - if (keep_alive_timeout.totalSeconds()) - response.set("Keep-Alive", "timeout=" + std::to_string(keep_alive_timeout.totalSeconds())); + Poco::Timespan timeout(keep_alive_timeout, 0); + if (timeout.totalSeconds()) + response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds())); } std::once_flag ssl_init_once; diff --git a/dbms/src/IO/HTTPCommon.h b/dbms/src/IO/HTTPCommon.h index 851400652b7..1278d6cd679 100644 --- a/dbms/src/IO/HTTPCommon.h +++ b/dbms/src/IO/HTTPCommon.h @@ -14,7 +14,7 @@ namespace Poco namespace DB { -void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response); +void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, int keep_alive_timeout); extern std::once_flag ssl_init_once; void SSLInit(); diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp b/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp index 26179b05873..70302d72aa6 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -27,7 +27,7 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() if (add_cors_header) response.set("Access-Control-Allow-Origin", "*"); - setResponseDefaultHeaders(response); + setResponseDefaultHeaders(response, keep_alive_timeout); #if POCO_CLICKHOUSE_PATCH if (request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) @@ -131,11 +131,16 @@ void WriteBufferFromHTTPServerResponse::nextImpl() WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerRequest & request_, Poco::Net::HTTPServerResponse & response_, + int keep_alive_timeout_, bool compress_, ZlibCompressionMethod compression_method_, size_t size) - : BufferWithOwnMemory(size), request(request_), response(response_), - compress(compress_), compression_method(compression_method_) + : BufferWithOwnMemory(size) + , request(request_) + , response(response_) + , keep_alive_timeout(keep_alive_timeout_) + , compress(compress_) + , compression_method(compression_method_) { } diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h index 54cfc15bb6f..d3dcb0de6e1 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h @@ -50,6 +50,7 @@ private: bool compress = false; ZlibCompressionMethod compression_method; int compression_level = Z_DEFAULT_COMPRESSION; + int keep_alive_timeout = 0; std::ostream * response_body_ostr = nullptr; @@ -86,6 +87,7 @@ public: WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerRequest & request_, Poco::Net::HTTPServerResponse & response_, + int keep_alive_timeout, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. ZlibCompressionMethod compression_method_ = ZlibCompressionMethod::Gzip, size_t size = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index 272b32da707..b3e9165ad06 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -229,6 +229,8 @@ void HTTPHandler::processQuery( std::string quota_key = request.get("X-ClickHouse-Quota", params.get("quota_key", "")); std::string query_id = params.get("query_id", ""); + const auto & config = server.config(); + Context context = server.context(); context.setGlobalContext(server.context()); @@ -246,7 +248,7 @@ void HTTPHandler::processQuery( if (session_is_set) { session_id = params.get("session_id"); - session_timeout = parseSessionTimeout(server.config(), params); + session_timeout = parseSessionTimeout(config, params); std::string session_check = params.get("session_check", ""); session = context.acquireSession(session_id, session_timeout, session_check == "1"); @@ -296,8 +298,11 @@ void HTTPHandler::processQuery( size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE; size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; + int keep_alive_timeout = config.getInt("keep_alive_timeout", 10); + used_output.out = std::make_shared( - request, response, client_supports_http_compression, http_response_compression_method, buffer_size_http); + request, response, keep_alive_timeout, + client_supports_http_compression, http_response_compression_method, buffer_size_http); if (internal_compression) used_output.out_maybe_compressed = std::make_shared(*used_output.out); else diff --git a/dbms/src/Server/HTTPHandlerFactory.h b/dbms/src/Server/HTTPHandlerFactory.h index 7ba25b0d194..34c49ce1671 100644 --- a/dbms/src/Server/HTTPHandlerFactory.h +++ b/dbms/src/Server/HTTPHandlerFactory.h @@ -47,7 +47,7 @@ public: if (uri == "/") return new RootRequestHandler(server); if (uri == "/ping") - return new PingRequestHandler; + return new PingRequestHandler(server); else if (startsWith(uri, "/replicas_status")) return new ReplicasStatusHandler(server.context()); } diff --git a/dbms/src/Server/InterserverIOHTTPHandler.cpp b/dbms/src/Server/InterserverIOHTTPHandler.cpp index c3813442897..41ce723b433 100644 --- a/dbms/src/Server/InterserverIOHTTPHandler.cpp +++ b/dbms/src/Server/InterserverIOHTTPHandler.cpp @@ -36,7 +36,10 @@ void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & reque ReadBufferFromIStream body(request.stream()); - WriteBufferFromHTTPServerResponse out(request, response); + const auto & config = server.config(); + int keep_alive_timeout = config.getInt("keep_alive_timeout", 10); + + WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout); auto endpoint = server.context().getInterserverIOHandler().getEndpoint(endpoint_name); diff --git a/dbms/src/Server/PingRequestHandler.cpp b/dbms/src/Server/PingRequestHandler.cpp index 492d2a5d9d1..4489145f35b 100644 --- a/dbms/src/Server/PingRequestHandler.cpp +++ b/dbms/src/Server/PingRequestHandler.cpp @@ -16,7 +16,9 @@ void PingRequestHandler::handleRequest( { try { - setResponseDefaultHeaders(response); + const auto & config = server.config(); + setResponseDefaultHeaders(response, config.getInt("keep_alive_timeout", 10)); + const char * data = "Ok.\n"; response.sendBuffer(data, strlen(data)); } diff --git a/dbms/src/Server/PingRequestHandler.h b/dbms/src/Server/PingRequestHandler.h index f81b45e376e..3728fb40adb 100644 --- a/dbms/src/Server/PingRequestHandler.h +++ b/dbms/src/Server/PingRequestHandler.h @@ -1,5 +1,7 @@ #pragma once +#include "IServer.h" + #include @@ -9,7 +11,14 @@ namespace DB /// Response with "Ok.\n". Used for availability checks. class PingRequestHandler : public Poco::Net::HTTPRequestHandler { +private: + IServer & server; + public: + explicit PingRequestHandler(IServer & server_) : server(server_) + { + } + void handleRequest( Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; diff --git a/dbms/src/Server/ReplicasStatusHandler.cpp b/dbms/src/Server/ReplicasStatusHandler.cpp index fc50ba7f48c..27323c593d4 100644 --- a/dbms/src/Server/ReplicasStatusHandler.cpp +++ b/dbms/src/Server/ReplicasStatusHandler.cpp @@ -62,7 +62,8 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request } } - setResponseDefaultHeaders(response); + const auto & config = context.getConfigRef(); + setResponseDefaultHeaders(response, config.getInt("keep_alive_timeout", 10)); if (ok && !verbose) { diff --git a/dbms/src/Server/RootRequestHandler.cpp b/dbms/src/Server/RootRequestHandler.cpp index 6020f9f5850..32eccd78cc4 100644 --- a/dbms/src/Server/RootRequestHandler.cpp +++ b/dbms/src/Server/RootRequestHandler.cpp @@ -16,9 +16,12 @@ void RootRequestHandler::handleRequest( { try { - setResponseDefaultHeaders(response); + const auto & config = server.config(); + setResponseDefaultHeaders(response, config.getInt("keep_alive_timeout", 10)); + response.setContentType("text/html; charset=UTF-8"); - const std::string data = server.config().getString("http_server_default_response", "Ok.\n"); + + const std::string data = config.getString("http_server_default_response", "Ok.\n"); response.sendBuffer(data.data(), data.size()); } catch (...) diff --git a/dbms/src/Server/RootRequestHandler.h b/dbms/src/Server/RootRequestHandler.h index 73261471bb1..1b6d53eeeda 100644 --- a/dbms/src/Server/RootRequestHandler.h +++ b/dbms/src/Server/RootRequestHandler.h @@ -1,9 +1,9 @@ #pragma once -#include - #include "IServer.h" +#include + namespace DB { From c8aa9f0f559baa49ab7c88d72778dcef0ca88c66 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Sep 2017 19:35:31 +0300 Subject: [PATCH 066/108] Update WriteBufferFromHTTPServerResponse.h --- dbms/src/IO/WriteBufferFromHTTPServerResponse.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h index d3dcb0de6e1..47ba573bbde 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h @@ -87,7 +87,7 @@ public: WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerRequest & request_, Poco::Net::HTTPServerResponse & response_, - int keep_alive_timeout, + int keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. ZlibCompressionMethod compression_method_ = ZlibCompressionMethod::Gzip, size_t size = DBMS_DEFAULT_BUFFER_SIZE); From db4c3f6e4866cfeb5111448dc74da9a10e50b702 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 19:41:35 +0300 Subject: [PATCH 067/108] Fixed build; clarifications [#CLICKHOUSE-3289]. --- dbms/src/IO/HTTPCommon.cpp | 2 +- dbms/src/IO/HTTPCommon.h | 2 +- dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp | 2 +- dbms/src/IO/WriteBufferFromHTTPServerResponse.h | 4 ++-- dbms/src/Server/HTTPHandler.cpp | 2 +- dbms/src/Server/InterserverIOHTTPHandler.cpp | 2 +- dbms/src/Server/PingRequestHandler.cpp | 2 +- dbms/src/Server/ReplicasStatusHandler.cpp | 2 +- dbms/src/Server/RootRequestHandler.cpp | 2 +- dbms/src/Server/Server.cpp | 4 ++-- 10 files changed, 12 insertions(+), 12 deletions(-) diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index 6aea408eaa5..7ca9058896f 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -15,7 +15,7 @@ namespace DB { -void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, int keep_alive_timeout) +void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, unsigned keep_alive_timeout) { if (!response.getKeepAlive()) return; diff --git a/dbms/src/IO/HTTPCommon.h b/dbms/src/IO/HTTPCommon.h index 1278d6cd679..c7fe7d5e7f8 100644 --- a/dbms/src/IO/HTTPCommon.h +++ b/dbms/src/IO/HTTPCommon.h @@ -14,7 +14,7 @@ namespace Poco namespace DB { -void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, int keep_alive_timeout); +void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, unsigned keep_alive_timeout); extern std::once_flag ssl_init_once; void SSLInit(); diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp b/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp index 70302d72aa6..51b5b20bba9 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -131,7 +131,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerRequest & request_, Poco::Net::HTTPServerResponse & response_, - int keep_alive_timeout_, + unsigned keep_alive_timeout_, bool compress_, ZlibCompressionMethod compression_method_, size_t size) diff --git a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h index 47ba573bbde..775cd9f30c6 100644 --- a/dbms/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/dbms/src/IO/WriteBufferFromHTTPServerResponse.h @@ -47,10 +47,10 @@ private: Poco::Net::HTTPServerResponse & response; bool add_cors_header = false; + unsigned keep_alive_timeout = 0; bool compress = false; ZlibCompressionMethod compression_method; int compression_level = Z_DEFAULT_COMPRESSION; - int keep_alive_timeout = 0; std::ostream * response_body_ostr = nullptr; @@ -87,7 +87,7 @@ public: WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerRequest & request_, Poco::Net::HTTPServerResponse & response_, - int keep_alive_timeout_, + unsigned keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. ZlibCompressionMethod compression_method_ = ZlibCompressionMethod::Gzip, size_t size = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index b3e9165ad06..85641950367 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -298,7 +298,7 @@ void HTTPHandler::processQuery( size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE; size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; - int keep_alive_timeout = config.getInt("keep_alive_timeout", 10); + unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); used_output.out = std::make_shared( request, response, keep_alive_timeout, diff --git a/dbms/src/Server/InterserverIOHTTPHandler.cpp b/dbms/src/Server/InterserverIOHTTPHandler.cpp index 41ce723b433..a70421ca986 100644 --- a/dbms/src/Server/InterserverIOHTTPHandler.cpp +++ b/dbms/src/Server/InterserverIOHTTPHandler.cpp @@ -37,7 +37,7 @@ void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & reque ReadBufferFromIStream body(request.stream()); const auto & config = server.config(); - int keep_alive_timeout = config.getInt("keep_alive_timeout", 10); + unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout); diff --git a/dbms/src/Server/PingRequestHandler.cpp b/dbms/src/Server/PingRequestHandler.cpp index 4489145f35b..21d45366044 100644 --- a/dbms/src/Server/PingRequestHandler.cpp +++ b/dbms/src/Server/PingRequestHandler.cpp @@ -17,7 +17,7 @@ void PingRequestHandler::handleRequest( try { const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getInt("keep_alive_timeout", 10)); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); const char * data = "Ok.\n"; response.sendBuffer(data, strlen(data)); diff --git a/dbms/src/Server/ReplicasStatusHandler.cpp b/dbms/src/Server/ReplicasStatusHandler.cpp index 27323c593d4..dfbd6cfc765 100644 --- a/dbms/src/Server/ReplicasStatusHandler.cpp +++ b/dbms/src/Server/ReplicasStatusHandler.cpp @@ -63,7 +63,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request } const auto & config = context.getConfigRef(); - setResponseDefaultHeaders(response, config.getInt("keep_alive_timeout", 10)); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); if (ok && !verbose) { diff --git a/dbms/src/Server/RootRequestHandler.cpp b/dbms/src/Server/RootRequestHandler.cpp index 32eccd78cc4..e01661be6b4 100644 --- a/dbms/src/Server/RootRequestHandler.cpp +++ b/dbms/src/Server/RootRequestHandler.cpp @@ -17,7 +17,7 @@ void RootRequestHandler::handleRequest( try { const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getInt("keep_alive_timeout", 10)); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); response.setContentType("text/html; charset=UTF-8"); diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 8167f85e2f9..53534463359 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -302,9 +302,9 @@ int Server::main(const std::vector & args) } { - Poco::Timespan keep_alive_timeout(config().getInt("keep_alive_timeout", 10), 0); + Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); - Poco::ThreadPool server_pool(3, config().getInt("max_connections", 1024)); + Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; http_params->setTimeout(settings.receive_timeout); http_params->setKeepAliveTimeout(keep_alive_timeout); From 9afe76e212f7bcb48f67706ad3a7ca129ab26864 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 19:46:56 +0300 Subject: [PATCH 068/108] Minor modification [#CLICKHOUSE-2]. --- dbms/src/Interpreters/Users.cpp | 6 +++--- dbms/src/Interpreters/Users.h | 3 --- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index 4f2a21ba402..887649232ac 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -284,8 +284,8 @@ User::User(const String & name_, const String & config_elem, Poco::Util::Abstrac throw Exception("password_sha256_hex for user " + name + " has length " + toString(password_sha256_hex.size()) + " but must be exactly 64 symbols.", ErrorCodes::BAD_ARGUMENTS); } - profile = config.getString(config_elem + ".profile"); - quota = config.getString(config_elem + ".quota"); + profile = config.getString(config_elem + ".profile"); + quota = config.getString(config_elem + ".quota"); addresses.addFromConfig(config_elem + ".networks", config); @@ -314,7 +314,7 @@ void Users::loadFromConfig(Poco::Util::AbstractConfiguration & config) config.keys("users", config_keys); for (const std::string & key : config_keys) - cont[key] = User(key, "users." + key, config); + cont.emplace(std::piecewise_construct, std::forward_as_tuple(key), std::forward_as_tuple(key, "users." + key, config)); } const User & Users::get(const String & user_name, const String & password, const Poco::Net::IPAddress & address) const diff --git a/dbms/src/Interpreters/Users.h b/dbms/src/Interpreters/Users.h index e28aced8d91..723b1f07324 100644 --- a/dbms/src/Interpreters/Users.h +++ b/dbms/src/Interpreters/Users.h @@ -67,9 +67,6 @@ struct User DatabaseSet databases; User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config); - - /// For insertion to containers. - User() {} }; From fce84167977285af923dae9210cc45f2dadb47a9 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 8 Sep 2017 21:24:15 +0300 Subject: [PATCH 069/108] Try fix macos build --- dbms/src/Dictionaries/LibraryDictionarySource.cpp | 5 +++-- dbms/src/Dictionaries/LibraryDictionarySourceExternal.h | 2 ++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index b600715f4a0..5785f176fe9 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -4,6 +4,7 @@ #include #include "LibraryDictionarySourceExternal.h" #include +#include namespace DB { @@ -155,7 +156,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector & { LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size()); - const ClickHouseLibrary::VectorUInt64 ids_data{static_cast(ids.data()), ids.size()}; + const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast(ids.data()), ids.size()}; auto columns_holder = std::make_unique(dict_struct.attributes.size()); ClickHouseLibrary::CStrings columns_pass{ static_cast(columns_holder.get()), dict_struct.attributes.size()}; @@ -201,7 +202,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column columns_pass.data[key_columns_n] = column->getName().c_str(); ++key_columns_n; } - const ClickHouseLibrary::VectorUInt64 requested_rows_c{reinterpret_cast(requested_rows.data()), requested_rows.size()}; + const ClickHouseLibrary::VectorUInt64 requested_rows_c{ext::bit_cast(requested_rows.data()), requested_rows.size()}; void * data_ptr = nullptr; /// Get function pointer before dataAllocate call because library->get may throw. diff --git a/dbms/src/Dictionaries/LibraryDictionarySourceExternal.h b/dbms/src/Dictionaries/LibraryDictionarySourceExternal.h index 3a3ecfff2c7..3a7af5acfce 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySourceExternal.h +++ b/dbms/src/Dictionaries/LibraryDictionarySourceExternal.h @@ -1,5 +1,7 @@ #pragma once +#include + namespace ClickHouseLibrary { using CString = const char *; From 57d244d739b5f06fd9f2e3ba050b238f39ac3227 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 23:24:11 +0300 Subject: [PATCH 070/108] Removed usage of deprecated C++ feature [#CLICKHOUSE-3174]. --- .../libpoco/Foundation/include/Poco/DirectoryIteratorStrategy.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libpoco/Foundation/include/Poco/DirectoryIteratorStrategy.h b/contrib/libpoco/Foundation/include/Poco/DirectoryIteratorStrategy.h index d4ce2bb8af1..b229ca41d73 100644 --- a/contrib/libpoco/Foundation/include/Poco/DirectoryIteratorStrategy.h +++ b/contrib/libpoco/Foundation/include/Poco/DirectoryIteratorStrategy.h @@ -34,7 +34,7 @@ class Foundation_API TraverseBase { public: typedef std::stack Stack; - typedef std::pointer_to_unary_function DepthFunPtr; + typedef std::function DepthFunPtr; enum { From ed1c0820f00c33b54eda7750b3b3c746a70038c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 02:25:42 +0300 Subject: [PATCH 071/108] Fixed minor issue found by Coverity [#CLICKHOUSE-3285]. --- dbms/src/Storages/System/StorageSystemGraphite.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index 109d6baafa7..2bb0d64a3f5 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NO_ELEMENTS_IN_CONFIG; +} + namespace { @@ -42,7 +47,11 @@ static Pattern readOnePattern( config.keys(path, keys); - for (const auto & key : keys) { + if (keys.empty()) + throw Exception("Empty pattern in Graphite rollup configuration", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + for (const auto & key : keys) + { const String key_path = path + "." + key; if (startsWith(key, "regexp")) @@ -74,7 +83,8 @@ static std::vector readPatterns( config.keys(section, keys); - for (const auto & key : keys) { + for (const auto & key : keys) + { if (startsWith(key, "pattern")) { Pattern pattern(readOnePattern(config, section + "." + key)); From 94b975e7f9e63724c0210627da04f5c509967691 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 02:31:18 +0300 Subject: [PATCH 072/108] Removed function deprecated in C++ [#CLICKHOUSE-3174]. --- dbms/src/DataStreams/narrowBlockInputStreams.cpp | 5 ++++- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- dbms/src/Storages/StorageReplicatedMergeTree.h | 4 ++++ 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/dbms/src/DataStreams/narrowBlockInputStreams.cpp b/dbms/src/DataStreams/narrowBlockInputStreams.cpp index 567ef97c20a..25927e451e4 100644 --- a/dbms/src/DataStreams/narrowBlockInputStreams.cpp +++ b/dbms/src/DataStreams/narrowBlockInputStreams.cpp @@ -1,3 +1,5 @@ +#include +#include #include @@ -18,7 +20,8 @@ BlockInputStreams narrowBlockInputStreams(BlockInputStreams & inputs, size_t wid for (size_t i = 0; i < size; ++i) distribution[i] = i % width; - std::random_shuffle(distribution.begin(), distribution.end()); + std::mt19937 generator(randomSeed()); + std::shuffle(distribution.begin(), distribution.end(), generator); for (size_t i = 0; i < size; ++i) partitions[distribution[i]].push_back(inputs[i]); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 38100cd0bd5..96fe887613f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -638,7 +638,7 @@ void StorageReplicatedMergeTree::createReplica() zookeeper->exists(replica_path, &stat); auto my_create_time = stat.czxid; - std::random_shuffle(replicas.begin(), replicas.end()); + std::shuffle(replicas.begin(), replicas.end(), rng); for (const String & replica : replicas) { if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica, &stat)) @@ -881,7 +881,7 @@ void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper( int expected_columns_version = columns_version; Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); - std::random_shuffle(replicas.begin(), replicas.end()); + std::shuffle(replicas.begin(), replicas.end(), rng); String expected_columns_str = part->columns.toString(); for (const String & replica : replicas) @@ -1931,7 +1931,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); /// Select replicas in uniformly random order. - std::random_shuffle(replicas.begin(), replicas.end()); + std::shuffle(replicas.begin(), replicas.end(), rng); for (const String & replica : replicas) { @@ -1956,7 +1956,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(const LogEntry Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); /// Select replicas in uniformly random order. - std::random_shuffle(replicas.begin(), replicas.end()); + std::shuffle(replicas.begin(), replicas.end(), rng); for (const String & replica : replicas) { diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 4379d38983d..0793231b16e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -21,6 +22,7 @@ #include #include #include +#include #include #include @@ -319,6 +321,8 @@ private: Logger * log; + std::mt19937 rng{randomSeed()}; + StorageReplicatedMergeTree( const String & zookeeper_path_, const String & replica_name_, From c0324aeb66881f0080bbba264ee5b1fea990cf72 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 02:40:48 +0300 Subject: [PATCH 073/108] Removed useless header [#CLICKHOUSE-3174]. --- dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h index a4cd4e158d1..003bc1d2d29 100644 --- a/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -12,7 +12,6 @@ #include #include #include -#include /// Implementation of Reservoir Sampling algorithm. Incrementally selects from the added objects a random subset of the `sample_count` size. From 7896e9d65fec342a814c8eed30c467239cb114e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 02:43:28 +0300 Subject: [PATCH 074/108] Imported pcg-random library (Reason: rng with small size of state, fast initialization, good performance and quality, with interface compatible with C++ standard library. Cause: boost::random::taus88 in latest Boost 1.65 is incompatible with C++17 standard) [#CLICKHOUSE-3174]. --- contrib/libpcg-random/LICENSE-APACHE.txt | 201 ++ contrib/libpcg-random/README.md | 52 + contrib/libpcg-random/include/pcg_extras.hpp | 639 ++++++ contrib/libpcg-random/include/pcg_random.hpp | 1771 +++++++++++++++++ contrib/libpcg-random/include/pcg_uint128.hpp | 748 +++++++ 5 files changed, 3411 insertions(+) create mode 100644 contrib/libpcg-random/LICENSE-APACHE.txt create mode 100644 contrib/libpcg-random/README.md create mode 100644 contrib/libpcg-random/include/pcg_extras.hpp create mode 100644 contrib/libpcg-random/include/pcg_random.hpp create mode 100644 contrib/libpcg-random/include/pcg_uint128.hpp diff --git a/contrib/libpcg-random/LICENSE-APACHE.txt b/contrib/libpcg-random/LICENSE-APACHE.txt new file mode 100644 index 00000000000..8dada3edaf5 --- /dev/null +++ b/contrib/libpcg-random/LICENSE-APACHE.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/contrib/libpcg-random/README.md b/contrib/libpcg-random/README.md new file mode 100644 index 00000000000..c6e579cc2cc --- /dev/null +++ b/contrib/libpcg-random/README.md @@ -0,0 +1,52 @@ +# PCG Random Number Generation, C++ Edition + +[PCG-Random website]: http://www.pcg-random.org + +This code provides an implementation of the PCG family of random number +generators, which are fast, statistically excellent, and offer a number of +useful features. + +Full details can be found at the [PCG-Random website]. This version +of the code provides many family members -- if you just want one +simple generator, you may prefer the minimal C version of the library. + +There are two kinds of generator, normal generators and extended generators. +Extended generators provide *k* dimensional equidistribution and can perform +party tricks, but generally speaking most people only need the normal +generators. + +There are two ways to access the generators, using a convenience typedef +or by using the underlying templates directly (similar to C++11's `std::mt19937` typedef vs its `std::mersenne_twister_engine` template). For most users, the convenience typedef is what you want, and probably you're fine with `pcg32` for 32-bit numbers. If you want 64-bit numbers, either use `pcg64` (or, if you're on a 32-bit system, making 64 bits from two calls to `pcg32_k2` may be faster). + +## Documentation and Examples + +Visit [PCG-Random website] for information on how to use this library, or look +at the sample code in the `sample` directory -- hopefully it should be fairly +self explanatory. + +## Building + +The code is written in C++11, as an include-only library (i.e., there is +nothing you need to build). There are some provided demo programs and tests +however. On a Unix-style system (e.g., Linux, Mac OS X) you should be able +to just type + + make + +To build the demo programs. + +## Testing + +Run + + make test + +## Directory Structure + +The directories are arranged as follows: + +* `include` -- contains `pcg_random.hpp` and supporting include files +* `test-high` -- test code for the high-level API where the functions have + shorter, less scary-looking names. +* `sample` -- sample code, some similar to the code in `test-high` but more + human readable, some other examples too diff --git a/contrib/libpcg-random/include/pcg_extras.hpp b/contrib/libpcg-random/include/pcg_extras.hpp new file mode 100644 index 00000000000..9bbe8edfe9a --- /dev/null +++ b/contrib/libpcg-random/include/pcg_extras.hpp @@ -0,0 +1,639 @@ +/* + * PCG Random Number Generation for C++ + * + * Copyright 2014-2017 Melissa O'Neill , + * and the PCG Project contributors. + * + * SPDX-License-Identifier: (Apache-2.0 OR MIT) + * + * Licensed under the Apache License, Version 2.0 (provided in + * LICENSE-APACHE.txt and at http://www.apache.org/licenses/LICENSE-2.0) + * or under the MIT license (provided in LICENSE-MIT.txt and at + * http://opensource.org/licenses/MIT), at your option. This file may not + * be copied, modified, or distributed except according to those terms. + * + * Distributed on an "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, either + * express or implied. See your chosen license for details. + * + * For additional information about the PCG random number generation scheme, + * visit http://www.pcg-random.org/. + */ + +/* + * This file provides support code that is useful for random-number generation + * but not specific to the PCG generation scheme, including: + * - 128-bit int support for platforms where it isn't available natively + * - bit twiddling operations + * - I/O of 128-bit and 8-bit integers + * - Handling the evilness of SeedSeq + * - Support for efficiently producing random numbers less than a given + * bound + */ + +#ifndef PCG_EXTRAS_HPP_INCLUDED +#define PCG_EXTRAS_HPP_INCLUDED 1 + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#ifdef __GNUC__ + #include +#endif + +/* + * Abstractions for compiler-specific directives + */ + +#ifdef __GNUC__ + #define PCG_NOINLINE __attribute__((noinline)) +#else + #define PCG_NOINLINE +#endif + +/* + * Some members of the PCG library use 128-bit math. When compiling on 64-bit + * platforms, both GCC and Clang provide 128-bit integer types that are ideal + * for the job. + * + * On 32-bit platforms (or with other compilers), we fall back to a C++ + * class that provides 128-bit unsigned integers instead. It may seem + * like we're reinventing the wheel here, because libraries already exist + * that support large integers, but most existing libraries provide a very + * generic multiprecision code, but here we're operating at a fixed size. + * Also, most other libraries are fairly heavyweight. So we use a direct + * implementation. Sadly, it's much slower than hand-coded assembly or + * direct CPU support. + * + */ +#if __SIZEOF_INT128__ + namespace pcg_extras { + typedef __uint128_t pcg128_t; + } + #define PCG_128BIT_CONSTANT(high,low) \ + ((pcg128_t(high) << 64) + low) +#else + #include "pcg_uint128.hpp" + namespace pcg_extras { + typedef pcg_extras::uint_x4 pcg128_t; + } + #define PCG_128BIT_CONSTANT(high,low) \ + pcg128_t(high,low) + #define PCG_EMULATED_128BIT_MATH 1 +#endif + + +namespace pcg_extras { + +/* + * We often need to represent a "number of bits". When used normally, these + * numbers are never greater than 128, so an unsigned char is plenty. + * If you're using a nonstandard generator of a larger size, you can set + * PCG_BITCOUNT_T to have it define it as a larger size. (Some compilers + * might produce faster code if you set it to an unsigned int.) + */ + +#ifndef PCG_BITCOUNT_T + typedef uint8_t bitcount_t; +#else + typedef PCG_BITCOUNT_T bitcount_t; +#endif + +/* + * C++ requires us to be able to serialize RNG state by printing or reading + * it from a stream. Because we use 128-bit ints, we also need to be able + * ot print them, so here is code to do so. + * + * This code provides enough functionality to print 128-bit ints in decimal + * and zero-padded in hex. It's not a full-featured implementation. + */ + +template +std::basic_ostream& +operator<<(std::basic_ostream& out, pcg128_t value) +{ + auto desired_base = out.flags() & out.basefield; + bool want_hex = desired_base == out.hex; + + if (want_hex) { + uint64_t highpart = uint64_t(value >> 64); + uint64_t lowpart = uint64_t(value); + auto desired_width = out.width(); + if (desired_width > 16) { + out.width(desired_width - 16); + } + if (highpart != 0 || desired_width > 16) + out << highpart; + CharT oldfill = '\0'; + if (highpart != 0) { + out.width(16); + oldfill = out.fill('0'); + } + auto oldflags = out.setf(decltype(desired_base){}, out.showbase); + out << lowpart; + out.setf(oldflags); + if (highpart != 0) { + out.fill(oldfill); + } + return out; + } + constexpr size_t MAX_CHARS_128BIT = 40; + + char buffer[MAX_CHARS_128BIT]; + char* pos = buffer+sizeof(buffer); + *(--pos) = '\0'; + constexpr auto BASE = pcg128_t(10ULL); + do { + auto div = value / BASE; + auto mod = uint32_t(value - (div * BASE)); + *(--pos) = '0' + char(mod); + value = div; + } while(value != pcg128_t(0ULL)); + return out << pos; +} + +template +std::basic_istream& +operator>>(std::basic_istream& in, pcg128_t& value) +{ + typename std::basic_istream::sentry s(in); + + if (!s) + return in; + + constexpr auto BASE = pcg128_t(10ULL); + pcg128_t current(0ULL); + bool did_nothing = true; + bool overflow = false; + for(;;) { + CharT wide_ch = in.get(); + if (!in.good()) + break; + auto ch = in.narrow(wide_ch, '\0'); + if (ch < '0' || ch > '9') { + in.unget(); + break; + } + did_nothing = false; + pcg128_t digit(uint32_t(ch - '0')); + pcg128_t timesbase = current*BASE; + overflow = overflow || timesbase < current; + current = timesbase + digit; + overflow = overflow || current < digit; + } + + if (did_nothing || overflow) { + in.setstate(std::ios::failbit); + if (overflow) + current = ~pcg128_t(0ULL); + } + + value = current; + + return in; +} + +/* + * Likewise, if people use tiny rngs, we'll be serializing uint8_t. + * If we just used the provided IO operators, they'd read/write chars, + * not ints, so we need to define our own. We *can* redefine this operator + * here because we're in our own namespace. + */ + +template +std::basic_ostream& +operator<<(std::basic_ostream&out, uint8_t value) +{ + return out << uint32_t(value); +} + +template +std::basic_istream& +operator>>(std::basic_istream& in, uint8_t& target) +{ + uint32_t value = 0xdecea5edU; + in >> value; + if (!in && value == 0xdecea5edU) + return in; + if (value > uint8_t(~0)) { + in.setstate(std::ios::failbit); + value = ~0U; + } + target = uint8_t(value); + return in; +} + +/* Unfortunately, the above functions don't get found in preference to the + * built in ones, so we create some more specific overloads that will. + * Ugh. + */ + +inline std::ostream& operator<<(std::ostream& out, uint8_t value) +{ + return pcg_extras::operator<< (out, value); +} + +inline std::istream& operator>>(std::istream& in, uint8_t& value) +{ + return pcg_extras::operator>> (in, value); +} + + + +/* + * Useful bitwise operations. + */ + +/* + * XorShifts are invertable, but they are someting of a pain to invert. + * This function backs them out. It's used by the whacky "inside out" + * generator defined later. + */ + +template +inline itype unxorshift(itype x, bitcount_t bits, bitcount_t shift) +{ + if (2*shift >= bits) { + return x ^ (x >> shift); + } + itype lowmask1 = (itype(1U) << (bits - shift*2)) - 1; + itype highmask1 = ~lowmask1; + itype top1 = x; + itype bottom1 = x & lowmask1; + top1 ^= top1 >> shift; + top1 &= highmask1; + x = top1 | bottom1; + itype lowmask2 = (itype(1U) << (bits - shift)) - 1; + itype bottom2 = x & lowmask2; + bottom2 = unxorshift(bottom2, bits - shift, shift); + bottom2 &= lowmask1; + return top1 | bottom2; +} + +/* + * Rotate left and right. + * + * In ideal world, compilers would spot idiomatic rotate code and convert it + * to a rotate instruction. Of course, opinions vary on what the correct + * idiom is and how to spot it. For clang, sometimes it generates better + * (but still crappy) code if you define PCG_USE_ZEROCHECK_ROTATE_IDIOM. + */ + +template +inline itype rotl(itype value, bitcount_t rot) +{ + constexpr bitcount_t bits = sizeof(itype) * 8; + constexpr bitcount_t mask = bits - 1; +#if PCG_USE_ZEROCHECK_ROTATE_IDIOM + return rot ? (value << rot) | (value >> (bits - rot)) : value; +#else + return (value << rot) | (value >> ((- rot) & mask)); +#endif +} + +template +inline itype rotr(itype value, bitcount_t rot) +{ + constexpr bitcount_t bits = sizeof(itype) * 8; + constexpr bitcount_t mask = bits - 1; +#if PCG_USE_ZEROCHECK_ROTATE_IDIOM + return rot ? (value >> rot) | (value << (bits - rot)) : value; +#else + return (value >> rot) | (value << ((- rot) & mask)); +#endif +} + +/* Unfortunately, both Clang and GCC sometimes perform poorly when it comes + * to properly recognizing idiomatic rotate code, so for we also provide + * assembler directives (enabled with PCG_USE_INLINE_ASM). Boo, hiss. + * (I hope that these compilers get better so that this code can die.) + * + * These overloads will be preferred over the general template code above. + */ +#if PCG_USE_INLINE_ASM && __GNUC__ && (__x86_64__ || __i386__) + +inline uint8_t rotr(uint8_t value, bitcount_t rot) +{ + asm ("rorb %%cl, %0" : "=r" (value) : "0" (value), "c" (rot)); + return value; +} + +inline uint16_t rotr(uint16_t value, bitcount_t rot) +{ + asm ("rorw %%cl, %0" : "=r" (value) : "0" (value), "c" (rot)); + return value; +} + +inline uint32_t rotr(uint32_t value, bitcount_t rot) +{ + asm ("rorl %%cl, %0" : "=r" (value) : "0" (value), "c" (rot)); + return value; +} + +#if __x86_64__ +inline uint64_t rotr(uint64_t value, bitcount_t rot) +{ + asm ("rorq %%cl, %0" : "=r" (value) : "0" (value), "c" (rot)); + return value; +} +#endif // __x86_64__ + +#endif // PCG_USE_INLINE_ASM + + +/* + * The C++ SeedSeq concept (modelled by seed_seq) can fill an array of + * 32-bit integers with seed data, but sometimes we want to produce + * larger or smaller integers. + * + * The following code handles this annoyance. + * + * uneven_copy will copy an array of 32-bit ints to an array of larger or + * smaller ints (actually, the code is general it only needing forward + * iterators). The copy is identical to the one that would be performed if + * we just did memcpy on a standard little-endian machine, but works + * regardless of the endian of the machine (or the weirdness of the ints + * involved). + * + * generate_to initializes an array of integers using a SeedSeq + * object. It is given the size as a static constant at compile time and + * tries to avoid memory allocation. If we're filling in 32-bit constants + * we just do it directly. If we need a separate buffer and it's small, + * we allocate it on the stack. Otherwise, we fall back to heap allocation. + * Ugh. + * + * generate_one produces a single value of some integral type using a + * SeedSeq object. + */ + + /* uneven_copy helper, case where destination ints are less than 32 bit. */ + +template +SrcIter uneven_copy_impl( + SrcIter src_first, DestIter dest_first, DestIter dest_last, + std::true_type) +{ + typedef typename std::iterator_traits::value_type src_t; + typedef typename std::iterator_traits::value_type dest_t; + + constexpr bitcount_t SRC_SIZE = sizeof(src_t); + constexpr bitcount_t DEST_SIZE = sizeof(dest_t); + constexpr bitcount_t DEST_BITS = DEST_SIZE * 8; + constexpr bitcount_t SCALE = SRC_SIZE / DEST_SIZE; + + size_t count = 0; + src_t value = 0; + + while (dest_first != dest_last) { + if ((count++ % SCALE) == 0) + value = *src_first++; // Get more bits + else + value >>= DEST_BITS; // Move down bits + + *dest_first++ = dest_t(value); // Truncates, ignores high bits. + } + return src_first; +} + + /* uneven_copy helper, case where destination ints are more than 32 bit. */ + +template +SrcIter uneven_copy_impl( + SrcIter src_first, DestIter dest_first, DestIter dest_last, + std::false_type) +{ + typedef typename std::iterator_traits::value_type src_t; + typedef typename std::iterator_traits::value_type dest_t; + + constexpr auto SRC_SIZE = sizeof(src_t); + constexpr auto SRC_BITS = SRC_SIZE * 8; + constexpr auto DEST_SIZE = sizeof(dest_t); + constexpr auto SCALE = (DEST_SIZE+SRC_SIZE-1) / SRC_SIZE; + + while (dest_first != dest_last) { + dest_t value(0UL); + unsigned int shift = 0; + + for (size_t i = 0; i < SCALE; ++i) { + value |= dest_t(*src_first++) << shift; + shift += SRC_BITS; + } + + *dest_first++ = value; + } + return src_first; +} + +/* uneven_copy, call the right code for larger vs. smaller */ + +template +inline SrcIter uneven_copy(SrcIter src_first, + DestIter dest_first, DestIter dest_last) +{ + typedef typename std::iterator_traits::value_type src_t; + typedef typename std::iterator_traits::value_type dest_t; + + constexpr bool DEST_IS_SMALLER = sizeof(dest_t) < sizeof(src_t); + + return uneven_copy_impl(src_first, dest_first, dest_last, + std::integral_constant{}); +} + +/* generate_to, fill in a fixed-size array of integral type using a SeedSeq + * (actually works for any random-access iterator) + */ + +template +inline void generate_to_impl(SeedSeq&& generator, DestIter dest, + std::true_type) +{ + generator.generate(dest, dest+size); +} + +template +void generate_to_impl(SeedSeq&& generator, DestIter dest, + std::false_type) +{ + typedef typename std::iterator_traits::value_type dest_t; + constexpr auto DEST_SIZE = sizeof(dest_t); + constexpr auto GEN_SIZE = sizeof(uint32_t); + + constexpr bool GEN_IS_SMALLER = GEN_SIZE < DEST_SIZE; + constexpr size_t FROM_ELEMS = + GEN_IS_SMALLER + ? size * ((DEST_SIZE+GEN_SIZE-1) / GEN_SIZE) + : (size + (GEN_SIZE / DEST_SIZE) - 1) + / ((GEN_SIZE / DEST_SIZE) + GEN_IS_SMALLER); + // this odd code ^^^^^^^^^^^^^^^^^ is work-around for + // a bug: http://llvm.org/bugs/show_bug.cgi?id=21287 + + if (FROM_ELEMS <= 1024) { + uint32_t buffer[FROM_ELEMS]; + generator.generate(buffer, buffer+FROM_ELEMS); + uneven_copy(buffer, dest, dest+size); + } else { + uint32_t* buffer = static_cast(malloc(GEN_SIZE * FROM_ELEMS)); + generator.generate(buffer, buffer+FROM_ELEMS); + uneven_copy(buffer, dest, dest+size); + free(static_cast(buffer)); + } +} + +template +inline void generate_to(SeedSeq&& generator, DestIter dest) +{ + typedef typename std::iterator_traits::value_type dest_t; + constexpr bool IS_32BIT = sizeof(dest_t) == sizeof(uint32_t); + + generate_to_impl(std::forward(generator), dest, + std::integral_constant{}); +} + +/* generate_one, produce a value of integral type using a SeedSeq + * (optionally, we can have it produce more than one and pick which one + * we want) + */ + +template +inline UInt generate_one(SeedSeq&& generator) +{ + UInt result[N]; + generate_to(std::forward(generator), result); + return result[i]; +} + +template +auto bounded_rand(RngType& rng, typename RngType::result_type upper_bound) + -> typename RngType::result_type +{ + typedef typename RngType::result_type rtype; + rtype threshold = (RngType::max() - RngType::min() + rtype(1) - upper_bound) + % upper_bound; + for (;;) { + rtype r = rng() - RngType::min(); + if (r >= threshold) + return r % upper_bound; + } +} + +template +void shuffle(Iter from, Iter to, RandType&& rng) +{ + typedef typename std::iterator_traits::difference_type delta_t; + typedef typename std::remove_reference::type::result_type result_t; + auto count = to - from; + while (count > 1) { + delta_t chosen = delta_t(bounded_rand(rng, result_t(count))); + --count; + --to; + using std::swap; + swap(*(from + chosen), *to); + } +} + +/* + * Although std::seed_seq is useful, it isn't everything. Often we want to + * initialize a random-number generator some other way, such as from a random + * device. + * + * Technically, it does not meet the requirements of a SeedSequence because + * it lacks some of the rarely-used member functions (some of which would + * be impossible to provide). However the C++ standard is quite specific + * that actual engines only called the generate method, so it ought not to be + * a problem in practice. + */ + +template +class seed_seq_from { +private: + RngType rng_; + + typedef uint_least32_t result_type; + +public: + template + seed_seq_from(Args&&... args) : + rng_(std::forward(args)...) + { + // Nothing (else) to do... + } + + template + void generate(Iter start, Iter finish) + { + for (auto i = start; i != finish; ++i) + *i = result_type(rng_()); + } + + constexpr size_t size() const + { + return (sizeof(typename RngType::result_type) > sizeof(result_type) + && RngType::max() > ~size_t(0UL)) + ? ~size_t(0UL) + : size_t(RngType::max()); + } +}; + +/* + * Sometimes you might want a distinct seed based on when the program + * was compiled. That way, a particular instance of the program will + * behave the same way, but when recompiled it'll produce a different + * value. + */ + +template +struct static_arbitrary_seed { +private: + static constexpr IntType fnv(IntType hash, const char* pos) { + return *pos == '\0' + ? hash + : fnv((hash * IntType(16777619U)) ^ *pos, (pos+1)); + } + +public: + static constexpr IntType value = fnv(IntType(2166136261U ^ sizeof(IntType)), + __DATE__ __TIME__ __FILE__); +}; + +// Sometimes, when debugging or testing, it's handy to be able print the name +// of a (in human-readable form). This code allows the idiom: +// +// cout << printable_typename() +// +// to print out my_foo_type_t (or its concrete type if it is a synonym) + +#if __cpp_rtti || __GXX_RTTI + +template +struct printable_typename {}; + +template +std::ostream& operator<<(std::ostream& out, printable_typename) { + const char *implementation_typename = typeid(T).name(); +#ifdef __GNUC__ + int status; + char* pretty_name = + abi::__cxa_demangle(implementation_typename, NULL, NULL, &status); + if (status == 0) + out << pretty_name; + free(static_cast(pretty_name)); + if (status == 0) + return out; +#endif + out << implementation_typename; + return out; +} + +#endif // __cpp_rtti || __GXX_RTTI + +} // namespace pcg_extras + +#endif // PCG_EXTRAS_HPP_INCLUDED diff --git a/contrib/libpcg-random/include/pcg_random.hpp b/contrib/libpcg-random/include/pcg_random.hpp new file mode 100644 index 00000000000..ea441013d18 --- /dev/null +++ b/contrib/libpcg-random/include/pcg_random.hpp @@ -0,0 +1,1771 @@ +/* + * PCG Random Number Generation for C++ + * + * Copyright 2014-2017 Melissa O'Neill , + * and the PCG Project contributors. + * + * SPDX-License-Identifier: (Apache-2.0 OR MIT) + * + * Licensed under the Apache License, Version 2.0 (provided in + * LICENSE-APACHE.txt and at http://www.apache.org/licenses/LICENSE-2.0) + * or under the MIT license (provided in LICENSE-MIT.txt and at + * http://opensource.org/licenses/MIT), at your option. This file may not + * be copied, modified, or distributed except according to those terms. + * + * Distributed on an "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, either + * express or implied. See your chosen license for details. + * + * For additional information about the PCG random number generation scheme, + * visit http://www.pcg-random.org/. + */ + +/* + * This code provides the reference implementation of the PCG family of + * random number generators. The code is complex because it implements + * + * - several members of the PCG family, specifically members corresponding + * to the output functions: + * - XSH RR (good for 64-bit state, 32-bit output) + * - XSH RS (good for 64-bit state, 32-bit output) + * - XSL RR (good for 128-bit state, 64-bit output) + * - RXS M XS (statistically most powerful generator) + * - XSL RR RR (good for 128-bit state, 128-bit output) + * - and RXS, RXS M, XSH, XSL (mostly for testing) + * - at potentially *arbitrary* bit sizes + * - with four different techniques for random streams (MCG, one-stream + * LCG, settable-stream LCG, unique-stream LCG) + * - and the extended generation schemes allowing arbitrary periods + * - with all features of C++11 random number generation (and more), + * some of which are somewhat painful, including + * - initializing with a SeedSequence which writes 32-bit values + * to memory, even though the state of the generator may not + * use 32-bit values (it might use smaller or larger integers) + * - I/O for RNGs and a prescribed format, which needs to handle + * the issue that 8-bit and 128-bit integers don't have working + * I/O routines (e.g., normally 8-bit = char, not integer) + * - equality and inequality for RNGs + * - and a number of convenience typedefs to mask all the complexity + * + * The code employes a fairly heavy level of abstraction, and has to deal + * with various C++ minutia. If you're looking to learn about how the PCG + * scheme works, you're probably best of starting with one of the other + * codebases (see www.pcg-random.org). But if you're curious about the + * constants for the various output functions used in those other, simpler, + * codebases, this code shows how they are calculated. + * + * On the positive side, at least there are convenience typedefs so that you + * can say + * + * pcg32 myRNG; + * + * rather than: + * + * pcg_detail::engine< + * uint32_t, // Output Type + * uint64_t, // State Type + * pcg_detail::xsh_rr_mixin, true, // Output Func + * pcg_detail::specific_stream, // Stream Kind + * pcg_detail::default_multiplier // LCG Mult + * > myRNG; + * + */ + +#ifndef PCG_RAND_HPP_INCLUDED +#define PCG_RAND_HPP_INCLUDED 1 + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#ifdef _MSC_VER + #pragma warning(disable:4146) +#endif + +#ifdef _MSC_VER + #define PCG_ALWAYS_INLINE _forceinline +#elif __GNUC__ + #define PCG_ALWAYS_INLINE __attribute__((always_inline)) +#else + #define PCG_ALWAYS_INLINE inline +#endif + +/* + * The pcg_extras namespace contains some support code that is likley to + * be useful for a variety of RNGs, including: + * - 128-bit int support for platforms where it isn't available natively + * - bit twiddling operations + * - I/O of 128-bit and 8-bit integers + * - Handling the evilness of SeedSeq + * - Support for efficiently producing random numbers less than a given + * bound + */ + +#include "pcg_extras.hpp" + +namespace pcg_detail { + +using namespace pcg_extras; + +/* + * The LCG generators need some constants to function. This code lets you + * look up the constant by *type*. For example + * + * default_multiplier::multiplier() + * + * gives you the default multipler for 32-bit integers. We use the name + * of the constant and not a generic word like value to allow these classes + * to be used as mixins. + */ + +template +struct default_multiplier { + // Not defined for an arbitrary type +}; + +template +struct default_increment { + // Not defined for an arbitrary type +}; + +#define PCG_DEFINE_CONSTANT(type, what, kind, constant) \ + template <> \ + struct what ## _ ## kind { \ + static constexpr type kind() { \ + return constant; \ + } \ + }; + +PCG_DEFINE_CONSTANT(uint8_t, default, multiplier, 141U) +PCG_DEFINE_CONSTANT(uint8_t, default, increment, 77U) + +PCG_DEFINE_CONSTANT(uint16_t, default, multiplier, 12829U) +PCG_DEFINE_CONSTANT(uint16_t, default, increment, 47989U) + +PCG_DEFINE_CONSTANT(uint32_t, default, multiplier, 747796405U) +PCG_DEFINE_CONSTANT(uint32_t, default, increment, 2891336453U) + +PCG_DEFINE_CONSTANT(uint64_t, default, multiplier, 6364136223846793005ULL) +PCG_DEFINE_CONSTANT(uint64_t, default, increment, 1442695040888963407ULL) + +PCG_DEFINE_CONSTANT(pcg128_t, default, multiplier, + PCG_128BIT_CONSTANT(2549297995355413924ULL,4865540595714422341ULL)) +PCG_DEFINE_CONSTANT(pcg128_t, default, increment, + PCG_128BIT_CONSTANT(6364136223846793005ULL,1442695040888963407ULL)) + + +/* + * Each PCG generator is available in four variants, based on how it applies + * the additive constant for its underlying LCG; the variations are: + * + * single stream - all instances use the same fixed constant, thus + * the RNG always somewhere in same sequence + * mcg - adds zero, resulting in a single stream and reduced + * period + * specific stream - the constant can be changed at any time, selecting + * a different random sequence + * unique stream - the constant is based on the memory addresss of the + * object, thus every RNG has its own unique sequence + * + * This variation is provided though mixin classes which define a function + * value called increment() that returns the nesessary additive constant. + */ + + + +/* + * unique stream + */ + + +template +class unique_stream { +protected: + static constexpr bool is_mcg = false; + + // Is never called, but is provided for symmetry with specific_stream + void set_stream(...) + { + abort(); + } + +public: + typedef itype state_type; + + constexpr itype increment() const { + return itype(reinterpret_cast(this) | 1); + } + + constexpr itype stream() const + { + return increment() >> 1; + } + + static constexpr bool can_specify_stream = false; + + static constexpr size_t streams_pow2() + { + return (sizeof(itype) < sizeof(size_t) ? sizeof(itype) + : sizeof(size_t))*8 - 1u; + } + +protected: + constexpr unique_stream() = default; +}; + + +/* + * no stream (mcg) + */ + +template +class no_stream { +protected: + static constexpr bool is_mcg = true; + + // Is never called, but is provided for symmetry with specific_stream + void set_stream(...) + { + abort(); + } + +public: + typedef itype state_type; + + static constexpr itype increment() { + return 0; + } + + static constexpr bool can_specify_stream = false; + + static constexpr size_t streams_pow2() + { + return 0u; + } + +protected: + constexpr no_stream() = default; +}; + + +/* + * single stream/sequence (oneseq) + */ + +template +class oneseq_stream : public default_increment { +protected: + static constexpr bool is_mcg = false; + + // Is never called, but is provided for symmetry with specific_stream + void set_stream(...) + { + abort(); + } + +public: + typedef itype state_type; + + static constexpr itype stream() + { + return default_increment::increment() >> 1; + } + + static constexpr bool can_specify_stream = false; + + static constexpr size_t streams_pow2() + { + return 0u; + } + +protected: + constexpr oneseq_stream() = default; +}; + + +/* + * specific stream + */ + +template +class specific_stream { +protected: + static constexpr bool is_mcg = false; + + itype inc_ = default_increment::increment(); + +public: + typedef itype state_type; + typedef itype stream_state; + + constexpr itype increment() const { + return inc_; + } + + itype stream() + { + return inc_ >> 1; + } + + void set_stream(itype specific_seq) + { + inc_ = (specific_seq << 1) | 1; + } + + static constexpr bool can_specify_stream = true; + + static constexpr size_t streams_pow2() + { + return (sizeof(itype)*8) - 1u; + } + +protected: + specific_stream() = default; + + specific_stream(itype specific_seq) + : inc_(itype(specific_seq << 1) | itype(1U)) + { + // Nothing (else) to do. + } +}; + + +/* + * This is where it all comes together. This function joins together three + * mixin classes which define + * - the LCG additive constant (the stream) + * - the LCG multiplier + * - the output function + * in addition, we specify the type of the LCG state, and the result type, + * and whether to use the pre-advance version of the state for the output + * (increasing instruction-level parallelism) or the post-advance version + * (reducing register pressure). + * + * Given the high level of parameterization, the code has to use some + * template-metaprogramming tricks to handle some of the suble variations + * involved. + */ + +template , + typename multiplier_mixin = default_multiplier > +class engine : protected output_mixin, + public stream_mixin, + protected multiplier_mixin { +protected: + itype state_; + + struct can_specify_stream_tag {}; + struct no_specifiable_stream_tag {}; + + using stream_mixin::increment; + using multiplier_mixin::multiplier; + +public: + typedef xtype result_type; + typedef itype state_type; + + static constexpr size_t period_pow2() + { + return sizeof(state_type)*8 - 2*stream_mixin::is_mcg; + } + + // It would be nice to use std::numeric_limits for these, but + // we can't be sure that it'd be defined for the 128-bit types. + + static constexpr result_type min() + { + return result_type(0UL); + } + + static constexpr result_type max() + { + return result_type(~result_type(0UL)); + } + +protected: + itype bump(itype state) + { + return state * multiplier() + increment(); + } + + itype base_generate() + { + return state_ = bump(state_); + } + + itype base_generate0() + { + itype old_state = state_; + state_ = bump(state_); + return old_state; + } + +public: + result_type operator()() + { + if (output_previous) + return this->output(base_generate0()); + else + return this->output(base_generate()); + } + + result_type operator()(result_type upper_bound) + { + return bounded_rand(*this, upper_bound); + } + +protected: + static itype advance(itype state, itype delta, + itype cur_mult, itype cur_plus); + + static itype distance(itype cur_state, itype newstate, itype cur_mult, + itype cur_plus, itype mask = ~itype(0U)); + + itype distance(itype newstate, itype mask = itype(~itype(0U))) const + { + return distance(state_, newstate, multiplier(), increment(), mask); + } + +public: + void advance(itype delta) + { + state_ = advance(state_, delta, this->multiplier(), this->increment()); + } + + void backstep(itype delta) + { + advance(-delta); + } + + void discard(itype delta) + { + advance(delta); + } + + bool wrapped() + { + if (stream_mixin::is_mcg) { + // For MCGs, the low order two bits never change. In this + // implementation, we keep them fixed at 3 to make this test + // easier. + return state_ == 3; + } else { + return state_ == 0; + } + } + + engine(itype state = itype(0xcafef00dd15ea5e5ULL)) + : state_(this->is_mcg ? state|state_type(3U) + : bump(state + this->increment())) + { + // Nothing else to do. + } + + // This function may or may not exist. It thus has to be a template + // to use SFINAE; users don't have to worry about its template-ness. + + template + engine(itype state, typename sm::stream_state stream_seed) + : stream_mixin(stream_seed), + state_(this->is_mcg ? state|state_type(3U) + : bump(state + this->increment())) + { + // Nothing else to do. + } + + template + engine(SeedSeq&& seedSeq, typename std::enable_if< + !stream_mixin::can_specify_stream + && !std::is_convertible::value + && !std::is_convertible::value, + no_specifiable_stream_tag>::type = {}) + : engine(generate_one(std::forward(seedSeq))) + { + // Nothing else to do. + } + + template + engine(SeedSeq&& seedSeq, typename std::enable_if< + stream_mixin::can_specify_stream + && !std::is_convertible::value + && !std::is_convertible::value, + can_specify_stream_tag>::type = {}) + : engine(generate_one(seedSeq), + generate_one(seedSeq)) + { + // Nothing else to do. + } + + + template + void seed(Args&&... args) + { + new (this) engine(std::forward(args)...); + } + + template + friend bool operator==(const engine&, + const engine&); + + template + friend itype1 operator-(const engine&, + const engine&); + + template + friend std::basic_ostream& + operator<<(std::basic_ostream& out, + const engine&); + + template + friend std::basic_istream& + operator>>(std::basic_istream& in, + engine& rng); +}; + +template +std::basic_ostream& +operator<<(std::basic_ostream& out, + const engine& rng) +{ + auto orig_flags = out.flags(std::ios_base::dec | std::ios_base::left); + auto space = out.widen(' '); + auto orig_fill = out.fill(); + + out << rng.multiplier() << space + << rng.increment() << space + << rng.state_; + + out.flags(orig_flags); + out.fill(orig_fill); + return out; +} + + +template +std::basic_istream& +operator>>(std::basic_istream& in, + engine& rng) +{ + auto orig_flags = in.flags(std::ios_base::dec | std::ios_base::skipws); + + itype multiplier, increment, state; + in >> multiplier >> increment >> state; + + if (!in.fail()) { + bool good = true; + if (multiplier != rng.multiplier()) { + good = false; + } else if (rng.can_specify_stream) { + rng.set_stream(increment >> 1); + } else if (increment != rng.increment()) { + good = false; + } + if (good) { + rng.state_ = state; + } else { + in.clear(std::ios::failbit); + } + } + + in.flags(orig_flags); + return in; +} + + +template +itype engine::advance( + itype state, itype delta, itype cur_mult, itype cur_plus) +{ + // The method used here is based on Brown, "Random Number Generation + // with Arbitrary Stride,", Transactions of the American Nuclear + // Society (Nov. 1994). The algorithm is very similar to fast + // exponentiation. + // + // Even though delta is an unsigned integer, we can pass a + // signed integer to go backwards, it just goes "the long way round". + + constexpr itype ZERO = 0u; // itype may be a non-trivial types, so + constexpr itype ONE = 1u; // we define some ugly constants. + itype acc_mult = 1; + itype acc_plus = 0; + while (delta > ZERO) { + if (delta & ONE) { + acc_mult *= cur_mult; + acc_plus = acc_plus*cur_mult + cur_plus; + } + cur_plus = (cur_mult+ONE)*cur_plus; + cur_mult *= cur_mult; + delta >>= 1; + } + return acc_mult * state + acc_plus; +} + +template +itype engine::distance( + itype cur_state, itype newstate, itype cur_mult, itype cur_plus, itype mask) +{ + constexpr itype ONE = 1u; // itype could be weird, so use constant + itype the_bit = stream_mixin::is_mcg ? itype(4u) : itype(1u); + itype distance = 0u; + while ((cur_state & mask) != (newstate & mask)) { + if ((cur_state & the_bit) != (newstate & the_bit)) { + cur_state = cur_state * cur_mult + cur_plus; + distance |= the_bit; + } + assert((cur_state & the_bit) == (newstate & the_bit)); + the_bit <<= 1; + cur_plus = (cur_mult+ONE)*cur_plus; + cur_mult *= cur_mult; + } + return stream_mixin::is_mcg ? distance >> 2 : distance; +} + +template +itype operator-(const engine& lhs, + const engine& rhs) +{ + static_assert( + std::is_same::value && + std::is_same::value, + "Incomparable generators"); + return rhs.distance(lhs.state_); +} + + +template +bool operator==(const engine& lhs, + const engine& rhs) +{ + return (lhs.multiplier() == rhs.multiplier()) + && (lhs.increment() == rhs.increment()) + && (lhs.state_ == rhs.state_); +} + +template +inline bool operator!=(const engine& lhs, + const engine& rhs) +{ + return !operator==(lhs,rhs); +} + + +template class output_mixin, + bool output_previous = (sizeof(itype) <= 8)> +using oneseq_base = engine, output_previous, + oneseq_stream >; + +template class output_mixin, + bool output_previous = (sizeof(itype) <= 8)> +using unique_base = engine, output_previous, + unique_stream >; + +template class output_mixin, + bool output_previous = (sizeof(itype) <= 8)> +using setseq_base = engine, output_previous, + specific_stream >; + +template class output_mixin, + bool output_previous = (sizeof(itype) <= 8)> +using mcg_base = engine, output_previous, + no_stream >; + +/* + * OUTPUT FUNCTIONS. + * + * These are the core of the PCG generation scheme. They specify how to + * turn the base LCG's internal state into the output value of the final + * generator. + * + * They're implemented as mixin classes. + * + * All of the classes have code that is written to allow it to be applied + * at *arbitrary* bit sizes, although in practice they'll only be used at + * standard sizes supported by C++. + */ + +/* + * XSH RS -- high xorshift, followed by a random shift + * + * Fast. A good performer. + */ + +template +struct xsh_rs_mixin { + static xtype output(itype internal) + { + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype) * 8); + constexpr bitcount_t sparebits = bits - xtypebits; + constexpr bitcount_t opbits = + sparebits-5 >= 64 ? 5 + : sparebits-4 >= 32 ? 4 + : sparebits-3 >= 16 ? 3 + : sparebits-2 >= 4 ? 2 + : sparebits-1 >= 1 ? 1 + : 0; + constexpr bitcount_t mask = (1 << opbits) - 1; + constexpr bitcount_t maxrandshift = mask; + constexpr bitcount_t topspare = opbits; + constexpr bitcount_t bottomspare = sparebits - topspare; + constexpr bitcount_t xshift = topspare + (xtypebits+maxrandshift)/2; + bitcount_t rshift = + opbits ? bitcount_t(internal >> (bits - opbits)) & mask : 0; + internal ^= internal >> xshift; + xtype result = xtype(internal >> (bottomspare - maxrandshift + rshift)); + return result; + } +}; + +/* + * XSH RR -- high xorshift, followed by a random rotate + * + * Fast. A good performer. Slightly better statistically than XSH RS. + */ + +template +struct xsh_rr_mixin { + static xtype output(itype internal) + { + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype)*8); + constexpr bitcount_t sparebits = bits - xtypebits; + constexpr bitcount_t wantedopbits = + xtypebits >= 128 ? 7 + : xtypebits >= 64 ? 6 + : xtypebits >= 32 ? 5 + : xtypebits >= 16 ? 4 + : 3; + constexpr bitcount_t opbits = + sparebits >= wantedopbits ? wantedopbits + : sparebits; + constexpr bitcount_t amplifier = wantedopbits - opbits; + constexpr bitcount_t mask = (1 << opbits) - 1; + constexpr bitcount_t topspare = opbits; + constexpr bitcount_t bottomspare = sparebits - topspare; + constexpr bitcount_t xshift = (topspare + xtypebits)/2; + bitcount_t rot = opbits ? bitcount_t(internal >> (bits - opbits)) & mask + : 0; + bitcount_t amprot = (rot << amplifier) & mask; + internal ^= internal >> xshift; + xtype result = xtype(internal >> bottomspare); + result = rotr(result, amprot); + return result; + } +}; + +/* + * RXS -- random xorshift + */ + +template +struct rxs_mixin { +static xtype output_rxs(itype internal) + { + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype)*8); + constexpr bitcount_t shift = bits - xtypebits; + constexpr bitcount_t extrashift = (xtypebits - shift)/2; + bitcount_t rshift = shift > 64+8 ? (internal >> (bits - 6)) & 63 + : shift > 32+4 ? (internal >> (bits - 5)) & 31 + : shift > 16+2 ? (internal >> (bits - 4)) & 15 + : shift > 8+1 ? (internal >> (bits - 3)) & 7 + : shift > 4+1 ? (internal >> (bits - 2)) & 3 + : shift > 2+1 ? (internal >> (bits - 1)) & 1 + : 0; + internal ^= internal >> (shift + extrashift - rshift); + xtype result = internal >> rshift; + return result; + } +}; + +/* + * RXS M XS -- random xorshift, mcg multiply, fixed xorshift + * + * The most statistically powerful generator, but all those steps + * make it slower than some of the others. We give it the rottenest jobs. + * + * Because it's usually used in contexts where the state type and the + * result type are the same, it is a permutation and is thus invertable. + * We thus provide a function to invert it. This function is used to + * for the "inside out" generator used by the extended generator. + */ + +/* Defined type-based concepts for the multiplication step. They're actually + * all derived by truncating the 128-bit, which was computed to be a good + * "universal" constant. + */ + +template +struct mcg_multiplier { + // Not defined for an arbitrary type +}; + +template +struct mcg_unmultiplier { + // Not defined for an arbitrary type +}; + +PCG_DEFINE_CONSTANT(uint8_t, mcg, multiplier, 217U) +PCG_DEFINE_CONSTANT(uint8_t, mcg, unmultiplier, 105U) + +PCG_DEFINE_CONSTANT(uint16_t, mcg, multiplier, 62169U) +PCG_DEFINE_CONSTANT(uint16_t, mcg, unmultiplier, 28009U) + +PCG_DEFINE_CONSTANT(uint32_t, mcg, multiplier, 277803737U) +PCG_DEFINE_CONSTANT(uint32_t, mcg, unmultiplier, 2897767785U) + +PCG_DEFINE_CONSTANT(uint64_t, mcg, multiplier, 12605985483714917081ULL) +PCG_DEFINE_CONSTANT(uint64_t, mcg, unmultiplier, 15009553638781119849ULL) + +PCG_DEFINE_CONSTANT(pcg128_t, mcg, multiplier, + PCG_128BIT_CONSTANT(17766728186571221404ULL, 12605985483714917081ULL)) +PCG_DEFINE_CONSTANT(pcg128_t, mcg, unmultiplier, + PCG_128BIT_CONSTANT(14422606686972528997ULL, 15009553638781119849ULL)) + + +template +struct rxs_m_xs_mixin { + static xtype output(itype internal) + { + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype) * 8); + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t opbits = xtypebits >= 128 ? 6 + : xtypebits >= 64 ? 5 + : xtypebits >= 32 ? 4 + : xtypebits >= 16 ? 3 + : 2; + constexpr bitcount_t shift = bits - xtypebits; + constexpr bitcount_t mask = (1 << opbits) - 1; + bitcount_t rshift = + opbits ? bitcount_t(internal >> (bits - opbits)) & mask : 0; + internal ^= internal >> (opbits + rshift); + internal *= mcg_multiplier::multiplier(); + xtype result = internal >> shift; + result ^= result >> ((2U*xtypebits+2U)/3U); + return result; + } + + static itype unoutput(itype internal) + { + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t opbits = bits >= 128 ? 6 + : bits >= 64 ? 5 + : bits >= 32 ? 4 + : bits >= 16 ? 3 + : 2; + constexpr bitcount_t mask = (1 << opbits) - 1; + + internal = unxorshift(internal, bits, (2U*bits+2U)/3U); + + internal *= mcg_unmultiplier::unmultiplier(); + + bitcount_t rshift = opbits ? (internal >> (bits - opbits)) & mask : 0; + internal = unxorshift(internal, bits, opbits + rshift); + + return internal; + } +}; + + +/* + * RXS M -- random xorshift, mcg multiply + */ + +template +struct rxs_m_mixin { + static xtype output(itype internal) + { + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype) * 8); + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t opbits = xtypebits >= 128 ? 6 + : xtypebits >= 64 ? 5 + : xtypebits >= 32 ? 4 + : xtypebits >= 16 ? 3 + : 2; + constexpr bitcount_t shift = bits - xtypebits; + constexpr bitcount_t mask = (1 << opbits) - 1; + bitcount_t rshift = opbits ? (internal >> (bits - opbits)) & mask : 0; + internal ^= internal >> (opbits + rshift); + internal *= mcg_multiplier::multiplier(); + xtype result = internal >> shift; + return result; + } +}; + +/* + * XSL RR -- fixed xorshift (to low bits), random rotate + * + * Useful for 128-bit types that are split across two CPU registers. + */ + +template +struct xsl_rr_mixin { + static xtype output(itype internal) + { + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype) * 8); + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t sparebits = bits - xtypebits; + constexpr bitcount_t wantedopbits = xtypebits >= 128 ? 7 + : xtypebits >= 64 ? 6 + : xtypebits >= 32 ? 5 + : xtypebits >= 16 ? 4 + : 3; + constexpr bitcount_t opbits = sparebits >= wantedopbits ? wantedopbits + : sparebits; + constexpr bitcount_t amplifier = wantedopbits - opbits; + constexpr bitcount_t mask = (1 << opbits) - 1; + constexpr bitcount_t topspare = sparebits; + constexpr bitcount_t bottomspare = sparebits - topspare; + constexpr bitcount_t xshift = (topspare + xtypebits) / 2; + + bitcount_t rot = + opbits ? bitcount_t(internal >> (bits - opbits)) & mask : 0; + bitcount_t amprot = (rot << amplifier) & mask; + internal ^= internal >> xshift; + xtype result = xtype(internal >> bottomspare); + result = rotr(result, amprot); + return result; + } +}; + + +/* + * XSL RR RR -- fixed xorshift (to low bits), random rotate (both parts) + * + * Useful for 128-bit types that are split across two CPU registers. + * If you really want an invertable 128-bit RNG, I guess this is the one. + */ + +template struct halfsize_trait {}; +template <> struct halfsize_trait { typedef uint64_t type; }; +template <> struct halfsize_trait { typedef uint32_t type; }; +template <> struct halfsize_trait { typedef uint16_t type; }; +template <> struct halfsize_trait { typedef uint8_t type; }; + +template +struct xsl_rr_rr_mixin { + typedef typename halfsize_trait::type htype; + + static itype output(itype internal) + { + constexpr bitcount_t htypebits = bitcount_t(sizeof(htype) * 8); + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t sparebits = bits - htypebits; + constexpr bitcount_t wantedopbits = htypebits >= 128 ? 7 + : htypebits >= 64 ? 6 + : htypebits >= 32 ? 5 + : htypebits >= 16 ? 4 + : 3; + constexpr bitcount_t opbits = sparebits >= wantedopbits ? wantedopbits + : sparebits; + constexpr bitcount_t amplifier = wantedopbits - opbits; + constexpr bitcount_t mask = (1 << opbits) - 1; + constexpr bitcount_t topspare = sparebits; + constexpr bitcount_t xshift = (topspare + htypebits) / 2; + + bitcount_t rot = + opbits ? bitcount_t(internal >> (bits - opbits)) & mask : 0; + bitcount_t amprot = (rot << amplifier) & mask; + internal ^= internal >> xshift; + htype lowbits = htype(internal); + lowbits = rotr(lowbits, amprot); + htype highbits = htype(internal >> topspare); + bitcount_t rot2 = lowbits & mask; + bitcount_t amprot2 = (rot2 << amplifier) & mask; + highbits = rotr(highbits, amprot2); + return (itype(highbits) << topspare) ^ itype(lowbits); + } +}; + + +/* + * XSH -- fixed xorshift (to high bits) + * + * You shouldn't use this at 64-bits or less. + */ + +template +struct xsh_mixin { + static xtype output(itype internal) + { + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype) * 8); + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t sparebits = bits - xtypebits; + constexpr bitcount_t topspare = 0; + constexpr bitcount_t bottomspare = sparebits - topspare; + constexpr bitcount_t xshift = (topspare + xtypebits) / 2; + + internal ^= internal >> xshift; + xtype result = internal >> bottomspare; + return result; + } +}; + +/* + * XSL -- fixed xorshift (to low bits) + * + * You shouldn't use this at 64-bits or less. + */ + +template +struct xsl_mixin { + inline xtype output(itype internal) + { + constexpr bitcount_t xtypebits = bitcount_t(sizeof(xtype) * 8); + constexpr bitcount_t bits = bitcount_t(sizeof(itype) * 8); + constexpr bitcount_t sparebits = bits - xtypebits; + constexpr bitcount_t topspare = sparebits; + constexpr bitcount_t bottomspare = sparebits - topspare; + constexpr bitcount_t xshift = (topspare + xtypebits) / 2; + + internal ^= internal >> xshift; + xtype result = internal >> bottomspare; + return result; + } +}; + +/* ---- End of Output Functions ---- */ + + +template +struct inside_out : private baseclass { + inside_out() = delete; + + typedef typename baseclass::result_type result_type; + typedef typename baseclass::state_type state_type; + static_assert(sizeof(result_type) == sizeof(state_type), + "Require a RNG whose output function is a permutation"); + + static bool external_step(result_type& randval, size_t i) + { + state_type state = baseclass::unoutput(randval); + state = state * baseclass::multiplier() + baseclass::increment() + + state_type(i*2); + result_type result = baseclass::output(state); + randval = result; + state_type zero = + baseclass::is_mcg ? state & state_type(3U) : state_type(0U); + return result == zero; + } + + static bool external_advance(result_type& randval, size_t i, + result_type delta, bool forwards = true) + { + state_type state = baseclass::unoutput(randval); + state_type mult = baseclass::multiplier(); + state_type inc = baseclass::increment() + state_type(i*2); + state_type zero = + baseclass::is_mcg ? state & state_type(3U) : state_type(0U); + state_type dist_to_zero = baseclass::distance(state, zero, mult, inc); + bool crosses_zero = + forwards ? dist_to_zero <= delta + : (-dist_to_zero) <= delta; + if (!forwards) + delta = -delta; + state = baseclass::advance(state, delta, mult, inc); + randval = baseclass::output(state); + return crosses_zero; + } +}; + + +template +class extended : public baseclass { +public: + typedef typename baseclass::state_type state_type; + typedef typename baseclass::result_type result_type; + typedef inside_out insideout; + +private: + static constexpr bitcount_t rtypebits = sizeof(result_type)*8; + static constexpr bitcount_t stypebits = sizeof(state_type)*8; + + static constexpr bitcount_t tick_limit_pow2 = 64U; + + static constexpr size_t table_size = 1UL << table_pow2; + static constexpr size_t table_shift = stypebits - table_pow2; + static constexpr state_type table_mask = + (state_type(1U) << table_pow2) - state_type(1U); + + static constexpr bool may_tick = + (advance_pow2 < stypebits) && (advance_pow2 < tick_limit_pow2); + static constexpr size_t tick_shift = stypebits - advance_pow2; + static constexpr state_type tick_mask = + may_tick ? state_type( + (uint64_t(1) << (advance_pow2*may_tick)) - 1) + // ^-- stupidity to appease GCC warnings + : ~state_type(0U); + + static constexpr bool may_tock = stypebits < tick_limit_pow2; + + result_type data_[table_size]; + + PCG_NOINLINE void advance_table(); + + PCG_NOINLINE void advance_table(state_type delta, bool isForwards = true); + + result_type& get_extended_value() + { + state_type state = this->state_; + if (kdd && baseclass::is_mcg) { + // The low order bits of an MCG are constant, so drop them. + state >>= 2; + } + size_t index = kdd ? state & table_mask + : state >> table_shift; + + if (may_tick) { + bool tick = kdd ? (state & tick_mask) == state_type(0u) + : (state >> tick_shift) == state_type(0u); + if (tick) + advance_table(); + } + if (may_tock) { + bool tock = state == state_type(0u); + if (tock) + advance_table(); + } + return data_[index]; + } + +public: + static constexpr size_t period_pow2() + { + return baseclass::period_pow2() + table_size*extvalclass::period_pow2(); + } + + PCG_ALWAYS_INLINE result_type operator()() + { + result_type rhs = get_extended_value(); + result_type lhs = this->baseclass::operator()(); + return lhs ^ rhs; + } + + result_type operator()(result_type upper_bound) + { + return bounded_rand(*this, upper_bound); + } + + void set(result_type wanted) + { + result_type& rhs = get_extended_value(); + result_type lhs = this->baseclass::operator()(); + rhs = lhs ^ wanted; + } + + void advance(state_type distance, bool forwards = true); + + void backstep(state_type distance) + { + advance(distance, false); + } + + extended(const result_type* data) + : baseclass() + { + datainit(data); + } + + extended(const result_type* data, state_type seed) + : baseclass(seed) + { + datainit(data); + } + + // This function may or may not exist. It thus has to be a template + // to use SFINAE; users don't have to worry about its template-ness. + + template + extended(const result_type* data, state_type seed, + typename bc::stream_state stream_seed) + : baseclass(seed, stream_seed) + { + datainit(data); + } + + extended() + : baseclass() + { + selfinit(); + } + + extended(state_type seed) + : baseclass(seed) + { + selfinit(); + } + + // This function may or may not exist. It thus has to be a template + // to use SFINAE; users don't have to worry about its template-ness. + + template + extended(state_type seed, typename bc::stream_state stream_seed) + : baseclass(seed, stream_seed) + { + selfinit(); + } + +private: + void selfinit(); + void datainit(const result_type* data); + +public: + + template::value + && !std::is_convertible::value>::type> + extended(SeedSeq&& seedSeq) + : baseclass(seedSeq) + { + generate_to(seedSeq, data_); + } + + template + void seed(Args&&... args) + { + new (this) extended(std::forward(args)...); + } + + template + friend bool operator==(const extended&, + const extended&); + + template + friend std::basic_ostream& + operator<<(std::basic_ostream& out, + const extended&); + + template + friend std::basic_istream& + operator>>(std::basic_istream& in, + extended&); + +}; + + +template +void extended::datainit( + const result_type* data) +{ + for (size_t i = 0; i < table_size; ++i) + data_[i] = data[i]; +} + +template +void extended::selfinit() +{ + // We need to fill the extended table with something, and we have + // very little provided data, so we use the base generator to + // produce values. Although not ideal (use a seed sequence, folks!), + // unexpected correlations are mitigated by + // - using XOR differences rather than the number directly + // - the way the table is accessed, its values *won't* be accessed + // in the same order the were written. + // - any strange correlations would only be apparent if we + // were to backstep the generator so that the base generator + // was generating the same values again + result_type xdiff = baseclass::operator()() - baseclass::operator()(); + for (size_t i = 0; i < table_size; ++i) { + data_[i] = baseclass::operator()() ^ xdiff; + } +} + +template +bool operator==(const extended& lhs, + const extended& rhs) +{ + auto& base_lhs = static_cast(lhs); + auto& base_rhs = static_cast(rhs); + return base_lhs == base_rhs + && std::equal( + std::begin(lhs.data_), std::end(lhs.data_), + std::begin(rhs.data_) + ); +} + +template +inline bool operator!=(const extended& lhs, + const extended& rhs) +{ + return !operator==(lhs, rhs); +} + +template +std::basic_ostream& +operator<<(std::basic_ostream& out, + const extended& rng) +{ + auto orig_flags = out.flags(std::ios_base::dec | std::ios_base::left); + auto space = out.widen(' '); + auto orig_fill = out.fill(); + + out << rng.multiplier() << space + << rng.increment() << space + << rng.state_; + + for (const auto& datum : rng.data_) + out << space << datum; + + out.flags(orig_flags); + out.fill(orig_fill); + return out; +} + +template +std::basic_istream& +operator>>(std::basic_istream& in, + extended& rng) +{ + extended new_rng; + auto& base_rng = static_cast(new_rng); + in >> base_rng; + + if (in.fail()) + return in; + + auto orig_flags = in.flags(std::ios_base::dec | std::ios_base::skipws); + + for (auto& datum : new_rng.data_) { + in >> datum; + if (in.fail()) + goto bail; + } + + rng = new_rng; + +bail: + in.flags(orig_flags); + return in; +} + + + +template +void +extended::advance_table() +{ + bool carry = false; + for (size_t i = 0; i < table_size; ++i) { + if (carry) { + carry = insideout::external_step(data_[i],i+1); + } + bool carry2 = insideout::external_step(data_[i],i+1); + carry = carry || carry2; + } +} + +template +void +extended::advance_table( + state_type delta, bool isForwards) +{ + typedef typename baseclass::state_type base_state_t; + typedef typename extvalclass::state_type ext_state_t; + constexpr bitcount_t basebits = sizeof(base_state_t)*8; + constexpr bitcount_t extbits = sizeof(ext_state_t)*8; + static_assert(basebits <= extbits || advance_pow2 > 0, + "Current implementation might overflow its carry"); + + base_state_t carry = 0; + for (size_t i = 0; i < table_size; ++i) { + base_state_t total_delta = carry + delta; + ext_state_t trunc_delta = ext_state_t(total_delta); + if (basebits > extbits) { + carry = total_delta >> extbits; + } else { + carry = 0; + } + carry += + insideout::external_advance(data_[i],i+1, trunc_delta, isForwards); + } +} + +template +void extended::advance( + state_type distance, bool forwards) +{ + static_assert(kdd, + "Efficient advance is too hard for non-kdd extension. " + "For a weak advance, cast to base class"); + state_type zero = + baseclass::is_mcg ? this->state_ & state_type(3U) : state_type(0U); + if (may_tick) { + state_type ticks = distance >> (advance_pow2*may_tick); + // ^-- stupidity to appease GCC + // warnings + state_type adv_mask = + baseclass::is_mcg ? tick_mask << 2 : tick_mask; + state_type next_advance_distance = this->distance(zero, adv_mask); + if (!forwards) + next_advance_distance = (-next_advance_distance) & tick_mask; + if (next_advance_distance < (distance & tick_mask)) { + ++ticks; + } + if (ticks) + advance_table(ticks, forwards); + } + if (forwards) { + if (may_tock && this->distance(zero) <= distance) + advance_table(); + baseclass::advance(distance); + } else { + if (may_tock && -(this->distance(zero)) <= distance) + advance_table(state_type(1U), false); + baseclass::advance(-distance); + } +} + +} // namespace pcg_detail + +namespace pcg_engines { + +using namespace pcg_detail; + +/* Predefined types for XSH RS */ + +typedef oneseq_base oneseq_xsh_rs_16_8; +typedef oneseq_base oneseq_xsh_rs_32_16; +typedef oneseq_base oneseq_xsh_rs_64_32; +typedef oneseq_base oneseq_xsh_rs_128_64; + +typedef unique_base unique_xsh_rs_16_8; +typedef unique_base unique_xsh_rs_32_16; +typedef unique_base unique_xsh_rs_64_32; +typedef unique_base unique_xsh_rs_128_64; + +typedef setseq_base setseq_xsh_rs_16_8; +typedef setseq_base setseq_xsh_rs_32_16; +typedef setseq_base setseq_xsh_rs_64_32; +typedef setseq_base setseq_xsh_rs_128_64; + +typedef mcg_base mcg_xsh_rs_16_8; +typedef mcg_base mcg_xsh_rs_32_16; +typedef mcg_base mcg_xsh_rs_64_32; +typedef mcg_base mcg_xsh_rs_128_64; + +/* Predefined types for XSH RR */ + +typedef oneseq_base oneseq_xsh_rr_16_8; +typedef oneseq_base oneseq_xsh_rr_32_16; +typedef oneseq_base oneseq_xsh_rr_64_32; +typedef oneseq_base oneseq_xsh_rr_128_64; + +typedef unique_base unique_xsh_rr_16_8; +typedef unique_base unique_xsh_rr_32_16; +typedef unique_base unique_xsh_rr_64_32; +typedef unique_base unique_xsh_rr_128_64; + +typedef setseq_base setseq_xsh_rr_16_8; +typedef setseq_base setseq_xsh_rr_32_16; +typedef setseq_base setseq_xsh_rr_64_32; +typedef setseq_base setseq_xsh_rr_128_64; + +typedef mcg_base mcg_xsh_rr_16_8; +typedef mcg_base mcg_xsh_rr_32_16; +typedef mcg_base mcg_xsh_rr_64_32; +typedef mcg_base mcg_xsh_rr_128_64; + + +/* Predefined types for RXS M XS */ + +typedef oneseq_base oneseq_rxs_m_xs_8_8; +typedef oneseq_base oneseq_rxs_m_xs_16_16; +typedef oneseq_base oneseq_rxs_m_xs_32_32; +typedef oneseq_base oneseq_rxs_m_xs_64_64; +typedef oneseq_base oneseq_rxs_m_xs_128_128; + +typedef unique_base unique_rxs_m_xs_8_8; +typedef unique_base unique_rxs_m_xs_16_16; +typedef unique_base unique_rxs_m_xs_32_32; +typedef unique_base unique_rxs_m_xs_64_64; +typedef unique_base unique_rxs_m_xs_128_128; + +typedef setseq_base setseq_rxs_m_xs_8_8; +typedef setseq_base setseq_rxs_m_xs_16_16; +typedef setseq_base setseq_rxs_m_xs_32_32; +typedef setseq_base setseq_rxs_m_xs_64_64; +typedef setseq_base setseq_rxs_m_xs_128_128; + + // MCG versions don't make sense here, so aren't defined. + +/* Predefined types for XSL RR (only defined for "large" types) */ + +typedef oneseq_base oneseq_xsl_rr_64_32; +typedef oneseq_base oneseq_xsl_rr_128_64; + +typedef unique_base unique_xsl_rr_64_32; +typedef unique_base unique_xsl_rr_128_64; + +typedef setseq_base setseq_xsl_rr_64_32; +typedef setseq_base setseq_xsl_rr_128_64; + +typedef mcg_base mcg_xsl_rr_64_32; +typedef mcg_base mcg_xsl_rr_128_64; + + +/* Predefined types for XSL RR RR (only defined for "large" types) */ + +typedef oneseq_base + oneseq_xsl_rr_rr_64_64; +typedef oneseq_base + oneseq_xsl_rr_rr_128_128; + +typedef unique_base + unique_xsl_rr_rr_64_64; +typedef unique_base + unique_xsl_rr_rr_128_128; + +typedef setseq_base + setseq_xsl_rr_rr_64_64; +typedef setseq_base + setseq_xsl_rr_rr_128_128; + + // MCG versions don't make sense here, so aren't defined. + +/* Extended generators */ + +template +using ext_std8 = extended; + +template +using ext_std16 = extended; + +template +using ext_std32 = extended; + +template +using ext_std64 = extended; + + +template +using ext_oneseq_rxs_m_xs_32_32 = + ext_std32; + +template +using ext_mcg_xsh_rs_64_32 = + ext_std32; + +template +using ext_oneseq_xsh_rs_64_32 = + ext_std32; + +template +using ext_setseq_xsh_rr_64_32 = + ext_std32; + +template +using ext_mcg_xsl_rr_128_64 = + ext_std64; + +template +using ext_oneseq_xsl_rr_128_64 = + ext_std64; + +template +using ext_setseq_xsl_rr_128_64 = + ext_std64; + +} // namespace pcg_engines + +typedef pcg_engines::setseq_xsh_rr_64_32 pcg32; +typedef pcg_engines::oneseq_xsh_rr_64_32 pcg32_oneseq; +typedef pcg_engines::unique_xsh_rr_64_32 pcg32_unique; +typedef pcg_engines::mcg_xsh_rs_64_32 pcg32_fast; + +typedef pcg_engines::setseq_xsl_rr_128_64 pcg64; +typedef pcg_engines::oneseq_xsl_rr_128_64 pcg64_oneseq; +typedef pcg_engines::unique_xsl_rr_128_64 pcg64_unique; +typedef pcg_engines::mcg_xsl_rr_128_64 pcg64_fast; + +typedef pcg_engines::setseq_rxs_m_xs_8_8 pcg8_once_insecure; +typedef pcg_engines::setseq_rxs_m_xs_16_16 pcg16_once_insecure; +typedef pcg_engines::setseq_rxs_m_xs_32_32 pcg32_once_insecure; +typedef pcg_engines::setseq_rxs_m_xs_64_64 pcg64_once_insecure; +typedef pcg_engines::setseq_xsl_rr_rr_128_128 pcg128_once_insecure; + +typedef pcg_engines::oneseq_rxs_m_xs_8_8 pcg8_oneseq_once_insecure; +typedef pcg_engines::oneseq_rxs_m_xs_16_16 pcg16_oneseq_once_insecure; +typedef pcg_engines::oneseq_rxs_m_xs_32_32 pcg32_oneseq_once_insecure; +typedef pcg_engines::oneseq_rxs_m_xs_64_64 pcg64_oneseq_once_insecure; +typedef pcg_engines::oneseq_xsl_rr_rr_128_128 pcg128_oneseq_once_insecure; + + +// These two extended RNGs provide two-dimensionally equidistributed +// 32-bit generators. pcg32_k2_fast occupies the same space as pcg64, +// and can be called twice to generate 64 bits, but does not required +// 128-bit math; on 32-bit systems, it's faster than pcg64 as well. + +typedef pcg_engines::ext_setseq_xsh_rr_64_32<1,16,true> pcg32_k2; +typedef pcg_engines::ext_oneseq_xsh_rs_64_32<1,32,true> pcg32_k2_fast; + +// These eight extended RNGs have about as much state as arc4random +// +// - the k variants are k-dimensionally equidistributed +// - the c variants offer better crypographic security +// +// (just how good the cryptographic security is is an open question) + +typedef pcg_engines::ext_setseq_xsh_rr_64_32<6,16,true> pcg32_k64; +typedef pcg_engines::ext_mcg_xsh_rs_64_32<6,32,true> pcg32_k64_oneseq; +typedef pcg_engines::ext_oneseq_xsh_rs_64_32<6,32,true> pcg32_k64_fast; + +typedef pcg_engines::ext_setseq_xsh_rr_64_32<6,16,false> pcg32_c64; +typedef pcg_engines::ext_oneseq_xsh_rs_64_32<6,32,false> pcg32_c64_oneseq; +typedef pcg_engines::ext_mcg_xsh_rs_64_32<6,32,false> pcg32_c64_fast; + +typedef pcg_engines::ext_setseq_xsl_rr_128_64<5,16,true> pcg64_k32; +typedef pcg_engines::ext_oneseq_xsl_rr_128_64<5,128,true> pcg64_k32_oneseq; +typedef pcg_engines::ext_mcg_xsl_rr_128_64<5,128,true> pcg64_k32_fast; + +typedef pcg_engines::ext_setseq_xsl_rr_128_64<5,16,false> pcg64_c32; +typedef pcg_engines::ext_oneseq_xsl_rr_128_64<5,128,false> pcg64_c32_oneseq; +typedef pcg_engines::ext_mcg_xsl_rr_128_64<5,128,false> pcg64_c32_fast; + +// These eight extended RNGs have more state than the Mersenne twister +// +// - the k variants are k-dimensionally equidistributed +// - the c variants offer better crypographic security +// +// (just how good the cryptographic security is is an open question) + +typedef pcg_engines::ext_setseq_xsh_rr_64_32<10,16,true> pcg32_k1024; +typedef pcg_engines::ext_oneseq_xsh_rs_64_32<10,32,true> pcg32_k1024_fast; + +typedef pcg_engines::ext_setseq_xsh_rr_64_32<10,16,false> pcg32_c1024; +typedef pcg_engines::ext_oneseq_xsh_rs_64_32<10,32,false> pcg32_c1024_fast; + +typedef pcg_engines::ext_setseq_xsl_rr_128_64<10,16,true> pcg64_k1024; +typedef pcg_engines::ext_oneseq_xsl_rr_128_64<10,128,true> pcg64_k1024_fast; + +typedef pcg_engines::ext_setseq_xsl_rr_128_64<10,16,false> pcg64_c1024; +typedef pcg_engines::ext_oneseq_xsl_rr_128_64<10,128,false> pcg64_c1024_fast; + +// These generators have an insanely huge period (2^524352), and is suitable +// for silly party tricks, such as dumping out 64 KB ZIP files at an arbitrary +// point in the future. [Actually, over the full period of the generator, it +// will produce every 64 KB ZIP file 2^64 times!] + +typedef pcg_engines::ext_setseq_xsh_rr_64_32<14,16,true> pcg32_k16384; +typedef pcg_engines::ext_oneseq_xsh_rs_64_32<14,32,true> pcg32_k16384_fast; + +#ifdef _MSC_VER + #pragma warning(default:4146) +#endif + +#endif // PCG_RAND_HPP_INCLUDED diff --git a/contrib/libpcg-random/include/pcg_uint128.hpp b/contrib/libpcg-random/include/pcg_uint128.hpp new file mode 100644 index 00000000000..1a1f61b9366 --- /dev/null +++ b/contrib/libpcg-random/include/pcg_uint128.hpp @@ -0,0 +1,748 @@ +/* + * PCG Random Number Generation for C++ + * + * Copyright 2014-2017 Melissa O'Neill , + * and the PCG Project contributors. + * + * SPDX-License-Identifier: (Apache-2.0 OR MIT) + * + * Licensed under the Apache License, Version 2.0 (provided in + * LICENSE-APACHE.txt and at http://www.apache.org/licenses/LICENSE-2.0) + * or under the MIT license (provided in LICENSE-MIT.txt and at + * http://opensource.org/licenses/MIT), at your option. This file may not + * be copied, modified, or distributed except according to those terms. + * + * Distributed on an "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, either + * express or implied. See your chosen license for details. + * + * For additional information about the PCG random number generation scheme, + * visit http://www.pcg-random.org/. + */ + +/* + * This code provides a a C++ class that can provide 128-bit (or higher) + * integers. To produce 2K-bit integers, it uses two K-bit integers, + * placed in a union that allowes the code to also see them as four K/2 bit + * integers (and access them either directly name, or by index). + * + * It may seem like we're reinventing the wheel here, because several + * libraries already exist that support large integers, but most existing + * libraries provide a very generic multiprecision code, but here we're + * operating at a fixed size. Also, most other libraries are fairly + * heavyweight. So we use a direct implementation. Sadly, it's much slower + * than hand-coded assembly or direct CPU support. + */ + +#ifndef PCG_UINT128_HPP_INCLUDED +#define PCG_UINT128_HPP_INCLUDED 1 + +#include +#include +#include +#include +#include +#include +#include + +/* + * We want to lay the type out the same way that a native type would be laid + * out, which means we must know the machine's endian, at compile time. + * This ugliness attempts to do so. + */ + +#ifndef PCG_LITTLE_ENDIAN + #if defined(__BYTE_ORDER__) + #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ + #define PCG_LITTLE_ENDIAN 1 + #elif __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + #define PCG_LITTLE_ENDIAN 0 + #else + #error __BYTE_ORDER__ does not match a standard endian, pick a side + #endif + #elif __LITTLE_ENDIAN__ || _LITTLE_ENDIAN + #define PCG_LITTLE_ENDIAN 1 + #elif __BIG_ENDIAN__ || _BIG_ENDIAN + #define PCG_LITTLE_ENDIAN 0 + #elif __x86_64 || __x86_64__ || _M_X64 || __i386 || __i386__ || _M_IX86 + #define PCG_LITTLE_ENDIAN 1 + #elif __powerpc__ || __POWERPC__ || __ppc__ || __PPC__ \ + || __m68k__ || __mc68000__ + #define PCG_LITTLE_ENDIAN 0 + #else + #error Unable to determine target endianness + #endif +#endif + +namespace pcg_extras { + +// Recent versions of GCC have intrinsics we can use to quickly calculate +// the number of leading and trailing zeros in a number. If possible, we +// use them, otherwise we fall back to old-fashioned bit twiddling to figure +// them out. + +#ifndef PCG_BITCOUNT_T + typedef uint8_t bitcount_t; +#else + typedef PCG_BITCOUNT_T bitcount_t; +#endif + +/* + * Provide some useful helper functions + * * flog2 floor(log2(x)) + * * trailingzeros number of trailing zero bits + */ + +#ifdef __GNUC__ // Any GNU-compatible compiler supporting C++11 has + // some useful intrinsics we can use. + +inline bitcount_t flog2(uint32_t v) +{ + return 31 - __builtin_clz(v); +} + +inline bitcount_t trailingzeros(uint32_t v) +{ + return __builtin_ctz(v); +} + +inline bitcount_t flog2(uint64_t v) +{ +#if UINT64_MAX == ULONG_MAX + return 63 - __builtin_clzl(v); +#elif UINT64_MAX == ULLONG_MAX + return 63 - __builtin_clzll(v); +#else + #error Cannot find a function for uint64_t +#endif +} + +inline bitcount_t trailingzeros(uint64_t v) +{ +#if UINT64_MAX == ULONG_MAX + return __builtin_ctzl(v); +#elif UINT64_MAX == ULLONG_MAX + return __builtin_ctzll(v); +#else + #error Cannot find a function for uint64_t +#endif +} + +#else // Otherwise, we fall back to bit twiddling + // implementations + +inline bitcount_t flog2(uint32_t v) +{ + // Based on code by Eric Cole and Mark Dickinson, which appears at + // https://graphics.stanford.edu/~seander/bithacks.html#IntegerLogDeBruijn + + static const uint8_t multiplyDeBruijnBitPos[32] = { + 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, + 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 + }; + + v |= v >> 1; // first round down to one less than a power of 2 + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + + return multiplyDeBruijnBitPos[(uint32_t)(v * 0x07C4ACDDU) >> 27]; +} + +inline bitcount_t trailingzeros(uint32_t v) +{ + static const uint8_t multiplyDeBruijnBitPos[32] = { + 0, 1, 28, 2, 29, 14, 24, 3, 30, 22, 20, 15, 25, 17, 4, 8, + 31, 27, 13, 23, 21, 19, 16, 7, 26, 12, 18, 6, 11, 5, 10, 9 + }; + + return multiplyDeBruijnBitPos[((uint32_t)((v & -v) * 0x077CB531U)) >> 27]; +} + +inline bitcount_t flog2(uint64_t v) +{ + uint32_t high = v >> 32; + uint32_t low = uint32_t(v); + + return high ? 32+flog2(high) : flog2(low); +} + +inline bitcount_t trailingzeros(uint64_t v) +{ + uint32_t high = v >> 32; + uint32_t low = uint32_t(v); + + return low ? trailingzeros(low) : trailingzeros(high)+32; +} + +#endif + +template +inline bitcount_t clog2(UInt v) +{ + return flog2(v) + ((v & (-v)) != v); +} + +template +inline UInt addwithcarry(UInt x, UInt y, bool carryin, bool* carryout) +{ + UInt half_result = y + carryin; + UInt result = x + half_result; + *carryout = (half_result < y) || (result < x); + return result; +} + +template +inline UInt subwithcarry(UInt x, UInt y, bool carryin, bool* carryout) +{ + UInt half_result = y + carryin; + UInt result = x - half_result; + *carryout = (half_result < y) || (result > x); + return result; +} + + +template +class uint_x4 { +// private: +public: + union { +#if PCG_LITTLE_ENDIAN + struct { + UInt v0, v1, v2, v3; + } w; + struct { + UIntX2 v01, v23; + } d; +#else + struct { + UInt v3, v2, v1, v0; + } w; + struct { + UIntX2 v23, v01; + } d; +#endif + // For the array access versions, the code that uses the array + // must handle endian itself. Yuck. + UInt wa[4]; + UIntX2 da[2]; + }; + +public: + uint_x4() = default; + + constexpr uint_x4(UInt v3, UInt v2, UInt v1, UInt v0) +#if PCG_LITTLE_ENDIAN + : w{v0, v1, v2, v3} +#else + : w{v3, v2, v1, v0} +#endif + { + // Nothing (else) to do + } + + constexpr uint_x4(UIntX2 v23, UIntX2 v01) +#if PCG_LITTLE_ENDIAN + : d{v01,v23} +#else + : d{v23,v01} +#endif + { + // Nothing (else) to do + } + + template::value + && sizeof(Integral) <= sizeof(UIntX2)) + >::type* = nullptr> + constexpr uint_x4(Integral v01) +#if PCG_LITTLE_ENDIAN + : d{UIntX2(v01),0UL} +#else + : d{0UL,UIntX2(v01)} +#endif + { + // Nothing (else) to do + } + + explicit constexpr operator uint64_t() const + { + return d.v01; + } + + explicit constexpr operator uint32_t() const + { + return w.v0; + } + + explicit constexpr operator int() const + { + return w.v0; + } + + explicit constexpr operator uint16_t() const + { + return w.v0; + } + + explicit constexpr operator uint8_t() const + { + return w.v0; + } + + typedef typename std::conditional::value, + unsigned long long, + unsigned long>::type + uint_missing_t; + + explicit constexpr operator uint_missing_t() const + { + return d.v01; + } + + explicit constexpr operator bool() const + { + return d.v01 || d.v23; + } + + template + friend uint_x4 operator*(const uint_x4&, const uint_x4&); + + template + friend std::pair< uint_x4,uint_x4 > + divmod(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator+(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator-(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator<<(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator>>(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator&(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator|(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator^(const uint_x4&, const uint_x4&); + + template + friend bool operator==(const uint_x4&, const uint_x4&); + + template + friend bool operator!=(const uint_x4&, const uint_x4&); + + template + friend bool operator<(const uint_x4&, const uint_x4&); + + template + friend bool operator<=(const uint_x4&, const uint_x4&); + + template + friend bool operator>(const uint_x4&, const uint_x4&); + + template + friend bool operator>=(const uint_x4&, const uint_x4&); + + template + friend uint_x4 operator~(const uint_x4&); + + template + friend uint_x4 operator-(const uint_x4&); + + template + friend bitcount_t flog2(const uint_x4&); + + template + friend bitcount_t trailingzeros(const uint_x4&); + + uint_x4& operator*=(const uint_x4& rhs) + { + uint_x4 result = *this * rhs; + return *this = result; + } + + uint_x4& operator/=(const uint_x4& rhs) + { + uint_x4 result = *this / rhs; + return *this = result; + } + + uint_x4& operator%=(const uint_x4& rhs) + { + uint_x4 result = *this % rhs; + return *this = result; + } + + uint_x4& operator+=(const uint_x4& rhs) + { + uint_x4 result = *this + rhs; + return *this = result; + } + + uint_x4& operator-=(const uint_x4& rhs) + { + uint_x4 result = *this - rhs; + return *this = result; + } + + uint_x4& operator&=(const uint_x4& rhs) + { + uint_x4 result = *this & rhs; + return *this = result; + } + + uint_x4& operator|=(const uint_x4& rhs) + { + uint_x4 result = *this | rhs; + return *this = result; + } + + uint_x4& operator^=(const uint_x4& rhs) + { + uint_x4 result = *this ^ rhs; + return *this = result; + } + + uint_x4& operator>>=(bitcount_t shift) + { + uint_x4 result = *this >> shift; + return *this = result; + } + + uint_x4& operator<<=(bitcount_t shift) + { + uint_x4 result = *this << shift; + return *this = result; + } + +}; + +template +bitcount_t flog2(const uint_x4& v) +{ +#if PCG_LITTLE_ENDIAN + for (uint8_t i = 4; i !=0; /* dec in loop */) { + --i; +#else + for (uint8_t i = 0; i < 4; ++i) { +#endif + if (v.wa[i] == 0) + continue; + return flog2(v.wa[i]) + (sizeof(U)*CHAR_BIT)*i; + } + abort(); +} + +template +bitcount_t trailingzeros(const uint_x4& v) +{ +#if PCG_LITTLE_ENDIAN + for (uint8_t i = 0; i < 4; ++i) { +#else + for (uint8_t i = 4; i !=0; /* dec in loop */) { + --i; +#endif + if (v.wa[i] != 0) + return trailingzeros(v.wa[i]) + (sizeof(U)*CHAR_BIT)*i; + } + return (sizeof(U)*CHAR_BIT)*4; +} + +template +std::pair< uint_x4, uint_x4 > + divmod(const uint_x4& orig_dividend, + const uint_x4& divisor) +{ + // If the dividend is less than the divisor, the answer is always zero. + // This takes care of boundary cases like 0/x (which would otherwise be + // problematic because we can't take the log of zero. (The boundary case + // of division by zero is undefined.) + if (orig_dividend < divisor) + return { uint_x4(0UL), orig_dividend }; + + auto dividend = orig_dividend; + + auto log2_divisor = flog2(divisor); + auto log2_dividend = flog2(dividend); + // assert(log2_dividend >= log2_divisor); + bitcount_t logdiff = log2_dividend - log2_divisor; + + constexpr uint_x4 ONE(1UL); + if (logdiff == 0) + return { ONE, dividend - divisor }; + + // Now we change the log difference to + // floor(log2(divisor)) - ceil(log2(dividend)) + // to ensure that we *underestimate* the result. + logdiff -= 1; + + uint_x4 quotient(0UL); + + auto qfactor = ONE << logdiff; + auto factor = divisor << logdiff; + + do { + dividend -= factor; + quotient += qfactor; + while (dividend < factor) { + factor >>= 1; + qfactor >>= 1; + } + } while (dividend >= divisor); + + return { quotient, dividend }; +} + +template +uint_x4 operator/(const uint_x4& dividend, + const uint_x4& divisor) +{ + return divmod(dividend, divisor).first; +} + +template +uint_x4 operator%(const uint_x4& dividend, + const uint_x4& divisor) +{ + return divmod(dividend, divisor).second; +} + + +template +uint_x4 operator*(const uint_x4& a, + const uint_x4& b) +{ + uint_x4 r = {0U, 0U, 0U, 0U}; + bool carryin = false; + bool carryout; + UIntX2 a0b0 = UIntX2(a.w.v0) * UIntX2(b.w.v0); + r.w.v0 = UInt(a0b0); + r.w.v1 = UInt(a0b0 >> 32); + + UIntX2 a1b0 = UIntX2(a.w.v1) * UIntX2(b.w.v0); + r.w.v2 = UInt(a1b0 >> 32); + r.w.v1 = addwithcarry(r.w.v1, UInt(a1b0), carryin, &carryout); + carryin = carryout; + r.w.v2 = addwithcarry(r.w.v2, UInt(0U), carryin, &carryout); + carryin = carryout; + r.w.v3 = addwithcarry(r.w.v3, UInt(0U), carryin, &carryout); + + UIntX2 a0b1 = UIntX2(a.w.v0) * UIntX2(b.w.v1); + carryin = false; + r.w.v2 = addwithcarry(r.w.v2, UInt(a0b1 >> 32), carryin, &carryout); + carryin = carryout; + r.w.v3 = addwithcarry(r.w.v3, UInt(0U), carryin, &carryout); + + carryin = false; + r.w.v1 = addwithcarry(r.w.v1, UInt(a0b1), carryin, &carryout); + carryin = carryout; + r.w.v2 = addwithcarry(r.w.v2, UInt(0U), carryin, &carryout); + carryin = carryout; + r.w.v3 = addwithcarry(r.w.v3, UInt(0U), carryin, &carryout); + + UIntX2 a1b1 = UIntX2(a.w.v1) * UIntX2(b.w.v1); + carryin = false; + r.w.v2 = addwithcarry(r.w.v2, UInt(a1b1), carryin, &carryout); + carryin = carryout; + r.w.v3 = addwithcarry(r.w.v3, UInt(a1b1 >> 32), carryin, &carryout); + + r.d.v23 += a.d.v01 * b.d.v23 + a.d.v23 * b.d.v01; + + return r; +} + + +template +uint_x4 operator+(const uint_x4& a, + const uint_x4& b) +{ + uint_x4 r = {0U, 0U, 0U, 0U}; + + bool carryin = false; + bool carryout; + r.w.v0 = addwithcarry(a.w.v0, b.w.v0, carryin, &carryout); + carryin = carryout; + r.w.v1 = addwithcarry(a.w.v1, b.w.v1, carryin, &carryout); + carryin = carryout; + r.w.v2 = addwithcarry(a.w.v2, b.w.v2, carryin, &carryout); + carryin = carryout; + r.w.v3 = addwithcarry(a.w.v3, b.w.v3, carryin, &carryout); + + return r; +} + +template +uint_x4 operator-(const uint_x4& a, + const uint_x4& b) +{ + uint_x4 r = {0U, 0U, 0U, 0U}; + + bool carryin = false; + bool carryout; + r.w.v0 = subwithcarry(a.w.v0, b.w.v0, carryin, &carryout); + carryin = carryout; + r.w.v1 = subwithcarry(a.w.v1, b.w.v1, carryin, &carryout); + carryin = carryout; + r.w.v2 = subwithcarry(a.w.v2, b.w.v2, carryin, &carryout); + carryin = carryout; + r.w.v3 = subwithcarry(a.w.v3, b.w.v3, carryin, &carryout); + + return r; +} + + +template +uint_x4 operator&(const uint_x4& a, + const uint_x4& b) +{ + return uint_x4(a.d.v23 & b.d.v23, a.d.v01 & b.d.v01); +} + +template +uint_x4 operator|(const uint_x4& a, + const uint_x4& b) +{ + return uint_x4(a.d.v23 | b.d.v23, a.d.v01 | b.d.v01); +} + +template +uint_x4 operator^(const uint_x4& a, + const uint_x4& b) +{ + return uint_x4(a.d.v23 ^ b.d.v23, a.d.v01 ^ b.d.v01); +} + +template +uint_x4 operator~(const uint_x4& v) +{ + return uint_x4(~v.d.v23, ~v.d.v01); +} + +template +uint_x4 operator-(const uint_x4& v) +{ + return uint_x4(0UL,0UL) - v; +} + +template +bool operator==(const uint_x4& a, const uint_x4& b) +{ + return (a.d.v01 == b.d.v01) && (a.d.v23 == b.d.v23); +} + +template +bool operator!=(const uint_x4& a, const uint_x4& b) +{ + return !operator==(a,b); +} + + +template +bool operator<(const uint_x4& a, const uint_x4& b) +{ + return (a.d.v23 < b.d.v23) + || ((a.d.v23 == b.d.v23) && (a.d.v01 < b.d.v01)); +} + +template +bool operator>(const uint_x4& a, const uint_x4& b) +{ + return operator<(b,a); +} + +template +bool operator<=(const uint_x4& a, const uint_x4& b) +{ + return !(operator<(b,a)); +} + +template +bool operator>=(const uint_x4& a, const uint_x4& b) +{ + return !(operator<(a,b)); +} + + + +template +uint_x4 operator<<(const uint_x4& v, + const bitcount_t shift) +{ + uint_x4 r = {0U, 0U, 0U, 0U}; + const bitcount_t bits = sizeof(UInt) * CHAR_BIT; + const bitcount_t bitmask = bits - 1; + const bitcount_t shiftdiv = shift / bits; + const bitcount_t shiftmod = shift & bitmask; + + if (shiftmod) { + UInt carryover = 0; +#if PCG_LITTLE_ENDIAN + for (uint8_t out = shiftdiv, in = 0; out < 4; ++out, ++in) { +#else + for (uint8_t out = 4-shiftdiv, in = 4; out != 0; /* dec in loop */) { + --out, --in; +#endif + r.wa[out] = (v.wa[in] << shiftmod) | carryover; + carryover = (v.wa[in] >> (bits - shiftmod)); + } + } else { +#if PCG_LITTLE_ENDIAN + for (uint8_t out = shiftdiv, in = 0; out < 4; ++out, ++in) { +#else + for (uint8_t out = 4-shiftdiv, in = 4; out != 0; /* dec in loop */) { + --out, --in; +#endif + r.wa[out] = v.wa[in]; + } + } + + return r; +} + +template +uint_x4 operator>>(const uint_x4& v, + const bitcount_t shift) +{ + uint_x4 r = {0U, 0U, 0U, 0U}; + const bitcount_t bits = sizeof(UInt) * CHAR_BIT; + const bitcount_t bitmask = bits - 1; + const bitcount_t shiftdiv = shift / bits; + const bitcount_t shiftmod = shift & bitmask; + + if (shiftmod) { + UInt carryover = 0; +#if PCG_LITTLE_ENDIAN + for (uint8_t out = 4-shiftdiv, in = 4; out != 0; /* dec in loop */) { + --out, --in; +#else + for (uint8_t out = shiftdiv, in = 0; out < 4; ++out, ++in) { +#endif + r.wa[out] = (v.wa[in] >> shiftmod) | carryover; + carryover = (v.wa[in] << (bits - shiftmod)); + } + } else { +#if PCG_LITTLE_ENDIAN + for (uint8_t out = 4-shiftdiv, in = 4; out != 0; /* dec in loop */) { + --out, --in; +#else + for (uint8_t out = shiftdiv, in = 0; out < 4; ++out, ++in) { +#endif + r.wa[out] = v.wa[in]; + } + } + + return r; +} + +} // namespace pcg_extras + +#endif // PCG_UINT128_HPP_INCLUDED From f9679a40b144b4e96ba0d7ea73e1252535e26896 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 02:57:52 +0300 Subject: [PATCH 075/108] Added README describing source of imported pcg-random library [#CLICKHOUSE-3174]. --- contrib/libpcg-random/README | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 contrib/libpcg-random/README diff --git a/contrib/libpcg-random/README b/contrib/libpcg-random/README new file mode 100644 index 00000000000..6b72706dce0 --- /dev/null +++ b/contrib/libpcg-random/README @@ -0,0 +1,2 @@ +https://github.com/imneme/pcg-cpp +0ca2e8ea6ba212bdfbc6219c2313c45917e34b8d From be79166199fc0ee9c04a622f3894f2d52ee14448 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 02:58:42 +0300 Subject: [PATCH 076/108] Using pcg-random instead of boost::random::taus88 [#CLICKHOUSE-3174]. --- cmake/lib_name.cmake | 1 + dbms/src/AggregateFunctions/CMakeLists.txt | 1 + dbms/src/AggregateFunctions/ReservoirSampler.h | 4 ++-- dbms/src/Server/CMakeLists.txt | 3 ++- 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/cmake/lib_name.cmake b/cmake/lib_name.cmake index ea1144ec305..9e56f549bed 100644 --- a/cmake/lib_name.cmake +++ b/cmake/lib_name.cmake @@ -4,3 +4,4 @@ set(CITYHASH_CONTRIB_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcityhash/in set(COMMON_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/libs/libcommon/include ${ClickHouse_BINARY_DIR}/libs/libcommon/include) set(DBMS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/dbms/src ${ClickHouse_BINARY_DIR}/dbms/src) set(DOUBLE_CONVERSION_CONTRIB_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libdouble-conversion) +set(PCG_RANDOM_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libpcg-random/include) diff --git a/dbms/src/AggregateFunctions/CMakeLists.txt b/dbms/src/AggregateFunctions/CMakeLists.txt index f3fb20b6101..4414d89f4a0 100644 --- a/dbms/src/AggregateFunctions/CMakeLists.txt +++ b/dbms/src/AggregateFunctions/CMakeLists.txt @@ -29,3 +29,4 @@ list(REMOVE_ITEM clickhouse_aggregate_functions_headers add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources}) target_link_libraries(clickhouse_aggregate_functions dbms) target_include_directories (clickhouse_aggregate_functions PRIVATE ${COMMON_INCLUDE_DIR}) +target_include_directories (clickhouse_aggregate_functions PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) diff --git a/dbms/src/AggregateFunctions/ReservoirSampler.h b/dbms/src/AggregateFunctions/ReservoirSampler.h index 1f3c909e2a9..9dae2900d11 100644 --- a/dbms/src/AggregateFunctions/ReservoirSampler.h +++ b/dbms/src/AggregateFunctions/ReservoirSampler.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include /// Implementing the Reservoir Sampling algorithm. Incrementally selects from the added objects a random subset of the sample_count size. @@ -196,7 +196,7 @@ private: size_t sample_count; size_t total_values = 0; Array samples; - boost::taus88 rng; + pcg32_fast rng; bool sorted = false; diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index 7741ba83e4d..864a37f4ff8 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -31,10 +31,11 @@ install (FILES clickhouse-client.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhous add_library (clickhouse-benchmark Benchmark.cpp) target_link_libraries (clickhouse-benchmark dbms ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_include_directories (clickhouse-benchmark PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) add_library (clickhouse-performance-test PerformanceTest.cpp) target_link_libraries (clickhouse-performance-test dbms ${Boost_PROGRAM_OPTIONS_LIBRARY}) - +target_include_directories (clickhouse-performance-test PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) add_executable(clickhouse main.cpp) target_include_directories(clickhouse PRIVATE ${COMMON_INCLUDE_DIR}) From b5c6bb64167b07dd504290f36d304b784d2e1333 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 03:01:29 +0300 Subject: [PATCH 077/108] Removed boost::random library [#CLICKHOUSE-3174]. --- .../libboost/boost_1_65_0/boost/random.hpp | 91 -- .../boost/random/additive_combine.hpp | 283 ------ .../boost/random/bernoulli_distribution.hpp | 197 ---- .../boost/random/beta_distribution.hpp | 184 ---- .../boost/random/binomial_distribution.hpp | 434 --------- .../boost/random/cauchy_distribution.hpp | 214 ----- .../boost/random/chi_squared_distribution.hpp | 209 ----- .../boost/random/detail/auto_link.hpp | 40 - .../boost/random/detail/config.hpp | 18 - .../boost/random/detail/const_mod.hpp | 216 ----- .../boost/random/detail/disable_warnings.hpp | 29 - .../boost/random/detail/enable_warnings.hpp | 22 - .../boost/random/detail/generator_bits.hpp | 36 - .../random/detail/generator_seed_seq.hpp | 40 - .../boost/random/detail/int_float_pair.hpp | 121 --- .../boost/random/detail/integer_log2.hpp | 84 -- .../boost/random/detail/iterator_mixin.hpp | 45 - .../boost/random/detail/large_arithmetic.hpp | 122 --- .../boost/random/detail/operators.hpp | 84 -- .../boost/random/detail/polynomial.hpp | 384 -------- .../boost/random/detail/ptr_helper.hpp | 67 -- .../boost_1_65_0/boost/random/detail/seed.hpp | 115 --- .../boost/random/detail/seed_impl.hpp | 398 -------- .../random/detail/signed_unsigned_tools.hpp | 89 -- .../boost/random/detail/uniform_int_float.hpp | 76 -- .../boost/random/detail/vector_io.hpp | 75 -- .../boost/random/discard_block.hpp | 241 ----- .../boost/random/discrete_distribution.hpp | 636 ------------- .../boost/random/exponential_distribution.hpp | 386 -------- .../random/extreme_value_distribution.hpp | 177 ---- .../boost/random/fisher_f_distribution.hpp | 183 ---- .../boost/random/gamma_distribution.hpp | 292 ------ .../boost/random/generate_canonical.hpp | 96 -- .../boost/random/geometric_distribution.hpp | 267 ------ .../random/hyperexponential_distribution.hpp | 883 ------------------ .../boost/random/independent_bits.hpp | 271 ------ .../boost/random/inversive_congruential.hpp | 267 ------ .../boost/random/lagged_fibonacci.hpp | 537 ----------- .../boost/random/laplace_distribution.hpp | 175 ---- .../boost/random/linear_congruential.hpp | 466 --------- .../boost/random/linear_feedback_shift.hpp | 217 ----- .../boost/random/lognormal_distribution.hpp | 254 ----- .../boost/random/mersenne_twister.hpp | 682 -------------- .../random/negative_binomial_distribution.hpp | 220 ----- .../non_central_chi_squared_distribution.hpp | 221 ----- .../boost/random/normal_distribution.hpp | 374 -------- .../piecewise_constant_distribution.hpp | 466 --------- .../random/piecewise_linear_distribution.hpp | 531 ----------- .../boost/random/poisson_distribution.hpp | 360 ------- .../boost/random/random_device.hpp | 143 --- .../boost/random/random_number_generator.hpp | 73 -- .../boost_1_65_0/boost/random/ranlux.hpp | 99 -- .../boost_1_65_0/boost/random/seed_seq.hpp | 118 --- .../boost/random/shuffle_order.hpp | 269 ------ .../boost/random/shuffle_output.hpp | 51 - .../boost/random/student_t_distribution.hpp | 180 ---- .../boost/random/subtract_with_carry.hpp | 613 ------------ .../boost_1_65_0/boost/random/taus88.hpp | 45 - .../boost_1_65_0/boost/random/traits.hpp | 107 --- .../boost/random/triangle_distribution.hpp | 232 ----- .../boost_1_65_0/boost/random/uniform_01.hpp | 257 ----- .../boost_1_65_0/boost/random/uniform_int.hpp | 99 -- .../boost/random/uniform_int_distribution.hpp | 419 --------- .../boost/random/uniform_on_sphere.hpp | 284 ------ .../boost/random/uniform_real.hpp | 82 -- .../random/uniform_real_distribution.hpp | 241 ----- .../boost/random/uniform_smallint.hpp | 307 ------ .../boost/random/variate_generator.hpp | 122 --- .../boost/random/weibull_distribution.hpp | 177 ---- .../boost_1_65_0/boost/random/xor_combine.hpp | 208 ----- 70 files changed, 16001 deletions(-) delete mode 100644 contrib/libboost/boost_1_65_0/boost/random.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/additive_combine.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/bernoulli_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/beta_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/binomial_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/cauchy_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/chi_squared_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/auto_link.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/config.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/const_mod.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/disable_warnings.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/enable_warnings.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/generator_bits.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/generator_seed_seq.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/int_float_pair.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/integer_log2.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/iterator_mixin.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/large_arithmetic.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/operators.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/polynomial.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/ptr_helper.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/seed.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/seed_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/signed_unsigned_tools.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/uniform_int_float.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/detail/vector_io.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/discard_block.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/discrete_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/exponential_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/extreme_value_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/fisher_f_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/gamma_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/generate_canonical.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/geometric_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/hyperexponential_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/independent_bits.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/inversive_congruential.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/lagged_fibonacci.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/laplace_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/linear_congruential.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/linear_feedback_shift.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/lognormal_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/mersenne_twister.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/negative_binomial_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/non_central_chi_squared_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/normal_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/piecewise_constant_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/piecewise_linear_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/poisson_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/random_device.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/random_number_generator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/ranlux.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/seed_seq.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/shuffle_order.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/shuffle_output.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/student_t_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/subtract_with_carry.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/taus88.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/traits.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/triangle_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/uniform_01.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/uniform_int.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/uniform_int_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/uniform_on_sphere.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/uniform_real.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/uniform_real_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/uniform_smallint.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/variate_generator.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/weibull_distribution.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/random/xor_combine.hpp diff --git a/contrib/libboost/boost_1_65_0/boost/random.hpp b/contrib/libboost/boost_1_65_0/boost/random.hpp deleted file mode 100644 index a164e7eb1ae..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random.hpp +++ /dev/null @@ -1,91 +0,0 @@ -/* boost random.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/random for documentation. - * - * $Id$ - * - * Revision history - * 2000-02-18 portability fixes (thanks to Beman Dawes) - * 2000-02-21 shuffle_output, inversive_congruential_schrage, - * generator_iterator, uniform_smallint - * 2000-02-23 generic modulus arithmetic helper, removed *_schrage classes, - * implemented Streamable and EqualityComparable concepts for - * generators, added Bernoulli distribution and Box-Muller - * transform - * 2000-03-01 cauchy, lognormal, triangle distributions; fixed - * uniform_smallint; renamed gaussian to normal distribution - * 2000-03-05 implemented iterator syntax for distribution functions - * 2000-04-21 removed some optimizations for better BCC/MSVC compatibility - * 2000-05-10 adapted to BCC and MSVC - * 2000-06-13 incorporated review results - * 2000-07-06 moved basic templates from namespace detail to random - * 2000-09-23 warning removals and int64 fixes (Ed Brey) - * 2000-09-24 added lagged_fibonacci generator (Matthias Troyer) - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_HPP -#define BOOST_RANDOM_HPP - -// generators -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// misc -#include -#include -#include -#include - -// distributions -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#endif // BOOST_RANDOM_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/additive_combine.hpp b/contrib/libboost/boost_1_65_0/boost/random/additive_combine.hpp deleted file mode 100644 index d786e18473a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/additive_combine.hpp +++ /dev/null @@ -1,283 +0,0 @@ -/* boost random/additive_combine.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_ADDITIVE_COMBINE_HPP -#define BOOST_RANDOM_ADDITIVE_COMBINE_HPP - -#include -#include -#include // for std::min and std::max -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * An instantiation of class template @c additive_combine_engine models a - * \pseudo_random_number_generator. It combines two multiplicative - * \linear_congruential_engine number generators, i.e. those with @c c = 0. - * It is described in - * - * @blockquote - * "Efficient and Portable Combined Random Number Generators", Pierre L'Ecuyer, - * Communications of the ACM, Vol. 31, No. 6, June 1988, pp. 742-749, 774 - * @endblockquote - * - * The template parameters MLCG1 and MLCG2 shall denote two different - * \linear_congruential_engine number generators, each with c = 0. Each - * invocation returns a random number - * X(n) := (MLCG1(n) - MLCG2(n)) mod (m1 - 1), - * where m1 denotes the modulus of MLCG1. - */ -template -class additive_combine_engine -{ -public: - typedef MLCG1 first_base; - typedef MLCG2 second_base; - typedef typename MLCG1::result_type result_type; - - // Required by old Boost.Random concept - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - /** - * Returns the smallest value that the generator can produce - */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return 1; } - /** - * Returns the largest value that the generator can produce - */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return MLCG1::modulus-1; } - - /** - * Constructs an @c additive_combine_engine using the - * default constructors of the two base generators. - */ - additive_combine_engine() : _mlcg1(), _mlcg2() { } - /** - * Constructs an @c additive_combine_engine, using seed as - * the constructor argument for both base generators. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(additive_combine_engine, - result_type, seed_arg) - { - _mlcg1.seed(seed_arg); - _mlcg2.seed(seed_arg); - } - /** - * Constructs an @c additive_combine_engine, using seq as - * the constructor argument for both base generators. - * - * @xmlwarning - * The semantics of this function are liable to change. - * A @c seed_seq is designed to generate all the seeds - * in one shot, but this seeds the two base engines - * independantly and probably ends up giving the same - * sequence to both. - * @endxmlwarning - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(additive_combine_engine, - SeedSeq, seq) - { - _mlcg1.seed(seq); - _mlcg2.seed(seq); - } - /** - * Constructs an @c additive_combine_engine, using - * @c seed1 and @c seed2 as the constructor argument to - * the first and second base generators, respectively. - */ - additive_combine_engine(typename MLCG1::result_type seed1, - typename MLCG2::result_type seed2) - : _mlcg1(seed1), _mlcg2(seed2) { } - /** - * Contructs an @c additive_combine_engine with - * values from the range defined by the input iterators first - * and last. first will be modified to point to the element - * after the last one used. - * - * Throws: @c std::invalid_argument if the input range is too small. - * - * Exception Safety: Basic - */ - template additive_combine_engine(It& first, It last) - : _mlcg1(first, last), _mlcg2(first, last) { } - - /** - * Seeds an @c additive_combine_engine using the default - * seeds of the two base generators. - */ - void seed() - { - _mlcg1.seed(); - _mlcg2.seed(); - } - - /** - * Seeds an @c additive_combine_engine, using @c seed as the - * seed for both base generators. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(additive_combine_engine, - result_type, seed_arg) - { - _mlcg1.seed(seed_arg); - _mlcg2.seed(seed_arg); - } - - /** - * Seeds an @c additive_combine_engine, using @c seq to - * seed both base generators. - * - * See the warning on the corresponding constructor. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(additive_combine_engine, - SeedSeq, seq) - { - _mlcg1.seed(seq); - _mlcg2.seed(seq); - } - - /** - * Seeds an @c additive_combine generator, using @c seed1 and @c seed2 as - * the seeds to the first and second base generators, respectively. - */ - void seed(typename MLCG1::result_type seed1, - typename MLCG2::result_type seed2) - { - _mlcg1.seed(seed1); - _mlcg2.seed(seed2); - } - - /** - * Seeds an @c additive_combine_engine with - * values from the range defined by the input iterators first - * and last. first will be modified to point to the element - * after the last one used. - * - * Throws: @c std::invalid_argument if the input range is too small. - * - * Exception Safety: Basic - */ - template void seed(It& first, It last) - { - _mlcg1.seed(first, last); - _mlcg2.seed(first, last); - } - - /** Returns the next value of the generator. */ - result_type operator()() { - result_type val1 = _mlcg1(); - result_type val2 = _mlcg2(); - if(val2 < val1) return val1 - val2; - else return val1 - val2 + MLCG1::modulus - 1; - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - _mlcg1.discard(z); - _mlcg2.discard(z); - } - - /** - * Writes the state of an @c additive_combine_engine to a @c - * std::ostream. The textual representation of an @c - * additive_combine_engine is the textual representation of - * the first base generator followed by the textual representation - * of the second base generator. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, additive_combine_engine, r) - { os << r._mlcg1 << ' ' << r._mlcg2; return os; } - - /** - * Reads the state of an @c additive_combine_engine from a - * @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, additive_combine_engine, r) - { is >> r._mlcg1 >> std::ws >> r._mlcg2; return is; } - - /** - * Returns: true iff the two @c additive_combine_engines will - * produce the same sequence of values. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(additive_combine_engine, x, y) - { return x._mlcg1 == y._mlcg1 && x._mlcg2 == y._mlcg2; } - /** - * Returns: true iff the two @c additive_combine_engines will - * produce different sequences of values. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(additive_combine_engine) - -private: - MLCG1 _mlcg1; - MLCG2 _mlcg2; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -template -const bool additive_combine_engine::has_fixed_range; -#endif - -/// \cond show_deprecated - -/** Provided for backwards compatibility. */ -template -class additive_combine : public additive_combine_engine -{ - typedef additive_combine_engine base_t; -public: - typedef typename base_t::result_type result_type; - additive_combine() {} - template - additive_combine(T& arg) : base_t(arg) {} - template - additive_combine(const T& arg) : base_t(arg) {} - template - additive_combine(It& first, It last) : base_t(first, last) {} -}; - -/// \endcond - -/** - * The specialization \ecuyer1988 was suggested in - * - * @blockquote - * "Efficient and Portable Combined Random Number Generators", Pierre L'Ecuyer, - * Communications of the ACM, Vol. 31, No. 6, June 1988, pp. 742-749, 774 - * @endblockquote - */ -typedef additive_combine_engine< - linear_congruential_engine, - linear_congruential_engine -> ecuyer1988; - -} // namespace random - -using random::ecuyer1988; - -} // namespace boost - -#endif // BOOST_RANDOM_ADDITIVE_COMBINE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/bernoulli_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/bernoulli_distribution.hpp deleted file mode 100644 index d66bae4389d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/bernoulli_distribution.hpp +++ /dev/null @@ -1,197 +0,0 @@ -/* boost random/bernoulli_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_BERNOULLI_DISTRIBUTION_HPP -#define BOOST_RANDOM_BERNOULLI_DISTRIBUTION_HPP - -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * Instantiations of class template \bernoulli_distribution model a - * \random_distribution. Such a random distribution produces bool values - * distributed with probabilities P(true) = p and P(false) = 1-p. p is - * the parameter of the distribution. - */ -template -class bernoulli_distribution -{ -public: - // In principle, this could work with both integer and floating-point - // types. Generating floating-point random numbers in the first - // place is probably more expensive, so use integer as input. - typedef int input_type; - typedef bool result_type; - - class param_type - { - public: - - typedef bernoulli_distribution distribution_type; - - /** - * Constructs the parameters of the distribution. - * - * Requires: 0 <= p <= 1 - */ - explicit param_type(RealType p_arg = RealType(0.5)) - : _p(p_arg) - { - BOOST_ASSERT(_p >= 0); - BOOST_ASSERT(_p <= 1); - } - - /** Returns the p parameter of the distribution. */ - RealType p() const { return _p; } - - /** Writes the parameters to a std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._p; - return os; - } - - /** Reads the parameters from a std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - is >> parm._p; - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._p == rhs._p; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _p; - }; - - /** - * Constructs a \bernoulli_distribution object. - * p is the parameter of the distribution. - * - * Requires: 0 <= p <= 1 - */ - explicit bernoulli_distribution(const RealType& p_arg = RealType(0.5)) - : _p(p_arg) - { - BOOST_ASSERT(_p >= 0); - BOOST_ASSERT(_p <= 1); - } - /** - * Constructs \bernoulli_distribution from its parameters - */ - explicit bernoulli_distribution(const param_type& parm) - : _p(parm.p()) {} - - // compiler-generated copy ctor and assignment operator are fine - - /** - * Returns: The "p" parameter of the distribution. - */ - RealType p() const { return _p; } - - /** Returns the smallest value that the distribution can produce. */ - bool min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return false; } - /** Returns the largest value that the distribution can produce. */ - bool max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return true; } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_p); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) { _p = parm.p(); } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** - * Returns: a random variate distributed according to the - * \bernoulli_distribution. - */ - template - bool operator()(Engine& eng) const - { - if(_p == RealType(0)) - return false; - else - return RealType(eng() - (eng.min)()) <= _p * RealType((eng.max)()-(eng.min)()); - } - - /** - * Returns: a random variate distributed according to the - * \bernoulli_distribution with parameters specified by param. - */ - template - bool operator()(Engine& eng, const param_type& parm) const - { - return bernoulli_distribution(parm)(eng); - } - - /** - * Writes the parameters of the distribution to a @c std::ostream. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, bernoulli_distribution, bd) - { - os << bd._p; - return os; - } - - /** - * Reads the parameters of the distribution from a @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, bernoulli_distribution, bd) - { - is >> bd._p; - return is; - } - - /** - * Returns true iff the two distributions will produce identical - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(bernoulli_distribution, lhs, rhs) - { return lhs._p == rhs._p; } - - /** - * Returns true iff the two distributions will produce different - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(bernoulli_distribution) - -private: - RealType _p; -}; - -} // namespace random - -using random::bernoulli_distribution; - -} // namespace boost - -#endif // BOOST_RANDOM_BERNOULLI_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/beta_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/beta_distribution.hpp deleted file mode 100644 index dabb72bfe2d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/beta_distribution.hpp +++ /dev/null @@ -1,184 +0,0 @@ -/* boost random/beta_distribution.hpp header file - * - * Copyright Steven Watanabe 2014 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_BETA_DISTRIBUTION_HPP -#define BOOST_RANDOM_BETA_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The beta distribution is a real-valued distribution which produces - * values in the range [0, 1]. It has two parameters, alpha and beta. - * - * It has \f$\displaystyle p(x) = \frac{x^{\alpha-1}(1-x)^{\beta-1}}{B(\alpha, \beta)}\f$. - */ -template -class beta_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef beta_distribution distribution_type; - - /** - * Constructs a @c param_type from the "alpha" and "beta" parameters - * of the distribution. - * - * Requires: alpha > 0, beta > 0 - */ - explicit param_type(RealType alpha_arg = RealType(1.0), - RealType beta_arg = RealType(1.0)) - : _alpha(alpha_arg), _beta(beta_arg) - { - assert(alpha_arg > 0); - assert(beta_arg > 0); - } - - /** Returns the "alpha" parameter of the distribtuion. */ - RealType alpha() const { return _alpha; } - /** Returns the "beta" parameter of the distribution. */ - RealType beta() const { return _beta; } - - /** Writes a @c param_type to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { os << parm._alpha << ' ' << parm._beta; return os; } - - /** Reads a @c param_type from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { is >> parm._alpha >> std::ws >> parm._beta; return is; } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._alpha == rhs._alpha && lhs._beta == rhs._beta; } - - /** Returns true if the two sets of parameters are the different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _alpha; - RealType _beta; - }; - - /** - * Constructs an @c beta_distribution from its "alpha" and "beta" parameters. - * - * Requires: alpha > 0, beta > 0 - */ - explicit beta_distribution(RealType alpha_arg = RealType(1.0), - RealType beta_arg = RealType(1.0)) - : _alpha(alpha_arg), _beta(beta_arg) - { - assert(alpha_arg > 0); - assert(beta_arg > 0); - } - /** Constructs an @c beta_distribution from its parameters. */ - explicit beta_distribution(const param_type& parm) - : _alpha(parm.alpha()), _beta(parm.beta()) - {} - - /** - * Returns a random variate distributed according to the - * beta distribution. - */ - template - RealType operator()(URNG& urng) const - { - RealType a = gamma_distribution(_alpha, RealType(1.0))(urng); - RealType b = gamma_distribution(_beta, RealType(1.0))(urng); - return a / (a + b); - } - - /** - * Returns a random variate distributed accordint to the beta - * distribution with parameters specified by @c param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return beta_distribution(parm)(urng); - } - - /** Returns the "alpha" parameter of the distribution. */ - RealType alpha() const { return _alpha; } - /** Returns the "beta" parameter of the distribution. */ - RealType beta() const { return _beta; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return RealType(0.0); } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return RealType(1.0); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_alpha, _beta); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _alpha = parm.alpha(); - _beta = parm.beta(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Writes an @c beta_distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, beta_distribution, wd) - { - os << wd.param(); - return os; - } - - /** Reads an @c beta_distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, beta_distribution, wd) - { - param_type parm; - if(is >> parm) { - wd.param(parm); - } - return is; - } - - /** - * Returns true if the two instances of @c beta_distribution will - * return identical sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(beta_distribution, lhs, rhs) - { return lhs._alpha == rhs._alpha && lhs._beta == rhs._beta; } - - /** - * Returns true if the two instances of @c beta_distribution will - * return different sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(beta_distribution) - -private: - RealType _alpha; - RealType _beta; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_BETA_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/binomial_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/binomial_distribution.hpp deleted file mode 100644 index 3efc905746c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/binomial_distribution.hpp +++ /dev/null @@ -1,434 +0,0 @@ -/* boost random/binomial_distribution.hpp header file - * - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_BINOMIAL_DISTRIBUTION_HPP_INCLUDED -#define BOOST_RANDOM_BINOMIAL_DISTRIBUTION_HPP_INCLUDED - -#include -#include -#include - -#include -#include - -#include - -namespace boost { -namespace random { - -namespace detail { - -template -struct binomial_table { - static const RealType table[10]; -}; - -template -const RealType binomial_table::table[10] = { - 0.08106146679532726, - 0.04134069595540929, - 0.02767792568499834, - 0.02079067210376509, - 0.01664469118982119, - 0.01387612882307075, - 0.01189670994589177, - 0.01041126526197209, - 0.009255462182712733, - 0.008330563433362871 -}; - -} - -/** - * The binomial distribution is an integer valued distribution with - * two parameters, @c t and @c p. The values of the distribution - * are within the range [0,t]. - * - * The distribution function is - * \f$\displaystyle P(k) = {t \choose k}p^k(1-p)^{t-k}\f$. - * - * The algorithm used is the BTRD algorithm described in - * - * @blockquote - * "The generation of binomial random variates", Wolfgang Hormann, - * Journal of Statistical Computation and Simulation, Volume 46, - * Issue 1 & 2 April 1993 , pages 101 - 110 - * @endblockquote - */ -template -class binomial_distribution { -public: - typedef IntType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef binomial_distribution distribution_type; - /** - * Construct a param_type object. @c t and @c p - * are the parameters of the distribution. - * - * Requires: t >=0 && 0 <= p <= 1 - */ - explicit param_type(IntType t_arg = 1, RealType p_arg = RealType (0.5)) - : _t(t_arg), _p(p_arg) - {} - /** Returns the @c t parameter of the distribution. */ - IntType t() const { return _t; } - /** Returns the @c p parameter of the distribution. */ - RealType p() const { return _p; } -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const param_type& parm) - { - os << parm._p << " " << parm._t; - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, param_type& parm) - { - is >> parm._p >> std::ws >> parm._t; - return is; - } -#endif - /** Returns true if the parameters have the same values. */ - friend bool operator==(const param_type& lhs, const param_type& rhs) - { - return lhs._t == rhs._t && lhs._p == rhs._p; - } - /** Returns true if the parameters have different values. */ - friend bool operator!=(const param_type& lhs, const param_type& rhs) - { - return !(lhs == rhs); - } - private: - IntType _t; - RealType _p; - }; - - /** - * Construct a @c binomial_distribution object. @c t and @c p - * are the parameters of the distribution. - * - * Requires: t >=0 && 0 <= p <= 1 - */ - explicit binomial_distribution(IntType t_arg = 1, - RealType p_arg = RealType(0.5)) - : _t(t_arg), _p(p_arg) - { - init(); - } - - /** - * Construct an @c binomial_distribution object from the - * parameters. - */ - explicit binomial_distribution(const param_type& parm) - : _t(parm.t()), _p(parm.p()) - { - init(); - } - - /** - * Returns a random variate distributed according to the - * binomial distribution. - */ - template - IntType operator()(URNG& urng) const - { - if(use_inversion()) { - if(0.5 < _p) { - return _t - invert(_t, 1-_p, urng); - } else { - return invert(_t, _p, urng); - } - } else if(0.5 < _p) { - return _t - generate(urng); - } else { - return generate(urng); - } - } - - /** - * Returns a random variate distributed according to the - * binomial distribution with parameters specified by @c param. - */ - template - IntType operator()(URNG& urng, const param_type& parm) const - { - return binomial_distribution(parm)(urng); - } - - /** Returns the @c t parameter of the distribution. */ - IntType t() const { return _t; } - /** Returns the @c p parameter of the distribution. */ - RealType p() const { return _p; } - - /** Returns the smallest value that the distribution can produce. */ - IntType min BOOST_PREVENT_MACRO_SUBSTITUTION() const { return 0; } - /** Returns the largest value that the distribution can produce. */ - IntType max BOOST_PREVENT_MACRO_SUBSTITUTION() const { return _t; } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_t, _p); } - /** Sets parameters of the distribution. */ - void param(const param_type& parm) - { - _t = parm.t(); - _p = parm.p(); - init(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const binomial_distribution& bd) - { - os << bd.param(); - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, binomial_distribution& bd) - { - bd.read(is); - return is; - } -#endif - - /** Returns true if the two distributions will produce the same - sequence of values, given equal generators. */ - friend bool operator==(const binomial_distribution& lhs, - const binomial_distribution& rhs) - { - return lhs._t == rhs._t && lhs._p == rhs._p; - } - /** Returns true if the two distributions could produce different - sequences of values, given equal generators. */ - friend bool operator!=(const binomial_distribution& lhs, - const binomial_distribution& rhs) - { - return !(lhs == rhs); - } - -private: - - /// @cond show_private - - template - void read(std::basic_istream& is) { - param_type parm; - if(is >> parm) { - param(parm); - } - } - - bool use_inversion() const - { - // BTRD is safe when np >= 10 - return m < 11; - } - - // computes the correction factor for the Stirling approximation - // for log(k!) - static RealType fc(IntType k) - { - if(k < 10) return detail::binomial_table::table[k]; - else { - RealType ikp1 = RealType(1) / (k + 1); - return (RealType(1)/12 - - (RealType(1)/360 - - (RealType(1)/1260)*(ikp1*ikp1))*(ikp1*ikp1))*ikp1; - } - } - - void init() - { - using std::sqrt; - using std::pow; - - RealType p = (0.5 < _p)? (1 - _p) : _p; - IntType t = _t; - - m = static_cast((t+1)*p); - - if(use_inversion()) { - q_n = pow((1 - p), static_cast(t)); - } else { - btrd.r = p/(1-p); - btrd.nr = (t+1)*btrd.r; - btrd.npq = t*p*(1-p); - RealType sqrt_npq = sqrt(btrd.npq); - btrd.b = 1.15 + 2.53 * sqrt_npq; - btrd.a = -0.0873 + 0.0248*btrd.b + 0.01*p; - btrd.c = t*p + 0.5; - btrd.alpha = (2.83 + 5.1/btrd.b) * sqrt_npq; - btrd.v_r = 0.92 - 4.2/btrd.b; - btrd.u_rv_r = 0.86*btrd.v_r; - } - } - - template - result_type generate(URNG& urng) const - { - using std::floor; - using std::abs; - using std::log; - - while(true) { - RealType u; - RealType v = uniform_01()(urng); - if(v <= btrd.u_rv_r) { - u = v/btrd.v_r - 0.43; - return static_cast(floor( - (2*btrd.a/(0.5 - abs(u)) + btrd.b)*u + btrd.c)); - } - - if(v >= btrd.v_r) { - u = uniform_01()(urng) - 0.5; - } else { - u = v/btrd.v_r - 0.93; - u = ((u < 0)? -0.5 : 0.5) - u; - v = uniform_01()(urng) * btrd.v_r; - } - - RealType us = 0.5 - abs(u); - IntType k = static_cast(floor((2*btrd.a/us + btrd.b)*u + btrd.c)); - if(k < 0 || k > _t) continue; - v = v*btrd.alpha/(btrd.a/(us*us) + btrd.b); - RealType km = abs(k - m); - if(km <= 15) { - RealType f = 1; - if(m < k) { - IntType i = m; - do { - ++i; - f = f*(btrd.nr/i - btrd.r); - } while(i != k); - } else if(m > k) { - IntType i = k; - do { - ++i; - v = v*(btrd.nr/i - btrd.r); - } while(i != m); - } - if(v <= f) return k; - else continue; - } else { - // final acceptance/rejection - v = log(v); - RealType rho = - (km/btrd.npq)*(((km/3. + 0.625)*km + 1./6)/btrd.npq + 0.5); - RealType t = -km*km/(2*btrd.npq); - if(v < t - rho) return k; - if(v > t + rho) continue; - - IntType nm = _t - m + 1; - RealType h = (m + 0.5)*log((m + 1)/(btrd.r*nm)) - + fc(m) + fc(_t - m); - - IntType nk = _t - k + 1; - if(v <= h + (_t+1)*log(static_cast(nm)/nk) - + (k + 0.5)*log(nk*btrd.r/(k+1)) - - fc(k) - - fc(_t - k)) - { - return k; - } else { - continue; - } - } - } - } - - template - IntType invert(IntType t, RealType p, URNG& urng) const - { - RealType q = 1 - p; - RealType s = p / q; - RealType a = (t + 1) * s; - RealType r = q_n; - RealType u = uniform_01()(urng); - IntType x = 0; - while(u > r) { - u = u - r; - ++x; - RealType r1 = ((a/x) - s) * r; - // If r gets too small then the round-off error - // becomes a problem. At this point, p(i) is - // decreasing exponentially, so if we just call - // it 0, it's close enough. Note that the - // minimum value of q_n is about 1e-7, so we - // may need to be a little careful to make sure that - // we don't terminate the first time through the loop - // for float. (Hence the test that r is decreasing) - if(r1 < std::numeric_limits::epsilon() && r1 < r) { - break; - } - r = r1; - } - return x; - } - - // parameters - IntType _t; - RealType _p; - - // common data - IntType m; - - union { - // for btrd - struct { - RealType r; - RealType nr; - RealType npq; - RealType b; - RealType a; - RealType c; - RealType alpha; - RealType v_r; - RealType u_rv_r; - } btrd; - // for inversion - RealType q_n; - }; - - /// @endcond -}; - -} - -// backwards compatibility -using random::binomial_distribution; - -} - -#include - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/cauchy_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/cauchy_distribution.hpp deleted file mode 100644 index 998e523447b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/cauchy_distribution.hpp +++ /dev/null @@ -1,214 +0,0 @@ -/* boost random/cauchy_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_CAUCHY_DISTRIBUTION_HPP -#define BOOST_RANDOM_CAUCHY_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -// Cauchy distribution: - -/** - * The cauchy distribution is a continuous distribution with two - * parameters, median and sigma. - * - * It has \f$\displaystyle p(x) = \frac{\sigma}{\pi(\sigma^2 + (x-m)^2)}\f$ - */ -template -class cauchy_distribution -{ -public: - typedef RealType input_type; - typedef RealType result_type; - - class param_type - { - public: - - typedef cauchy_distribution distribution_type; - - /** Constructs the parameters of the cauchy distribution. */ - explicit param_type(RealType median_arg = RealType(0.0), - RealType sigma_arg = RealType(1.0)) - : _median(median_arg), _sigma(sigma_arg) {} - - // backwards compatibility for Boost.Random - - /** Returns the median of the distribution. */ - RealType median() const { return _median; } - /** Returns the sigma parameter of the distribution. */ - RealType sigma() const { return _sigma; } - - // The new names in C++0x. - - /** Returns the median of the distribution. */ - RealType a() const { return _median; } - /** Returns the sigma parameter of the distribution. */ - RealType b() const { return _sigma; } - - /** Writes the parameters to a std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._median << " " << parm._sigma; - return os; - } - - /** Reads the parameters from a std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - is >> parm._median >> std::ws >> parm._sigma; - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._median == rhs._median && lhs._sigma == rhs._sigma; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _median; - RealType _sigma; - }; - - /** - * Constructs a \cauchy_distribution with the paramters @c median - * and @c sigma. - */ - explicit cauchy_distribution(RealType median_arg = RealType(0.0), - RealType sigma_arg = RealType(1.0)) - : _median(median_arg), _sigma(sigma_arg) { } - - /** - * Constructs a \cauchy_distribution from it's parameters. - */ - explicit cauchy_distribution(const param_type& parm) - : _median(parm.median()), _sigma(parm.sigma()) { } - - // compiler-generated copy ctor and assignment operator are fine - - // backwards compatibility for Boost.Random - - /** Returns: the "median" parameter of the distribution */ - RealType median() const { return _median; } - /** Returns: the "sigma" parameter of the distribution */ - RealType sigma() const { return _sigma; } - - // The new names in C++0x - - /** Returns: the "median" parameter of the distribution */ - RealType a() const { return _median; } - /** Returns: the "sigma" parameter of the distribution */ - RealType b() const { return _sigma; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return -(std::numeric_limits::infinity)(); } - - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return (std::numeric_limits::infinity)(); } - - param_type param() const { return param_type(_median, _sigma); } - - void param(const param_type& parm) - { - _median = parm.median(); - _sigma = parm.sigma(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** - * Returns: A random variate distributed according to the - * cauchy distribution. - */ - template - result_type operator()(Engine& eng) - { - // Can we have a boost::mathconst please? - const result_type pi = result_type(3.14159265358979323846); - using std::tan; - RealType val = uniform_01()(eng)-result_type(0.5); - return _median + _sigma * tan(pi*val); - } - - /** - * Returns: A random variate distributed according to the - * cauchy distribution with parameters specified by param. - */ - template - result_type operator()(Engine& eng, const param_type& parm) - { - return cauchy_distribution(parm)(eng); - } - - /** - * Writes the distribution to a @c std::ostream. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, cauchy_distribution, cd) - { - os << cd._median << " " << cd._sigma; - return os; - } - - /** - * Reads the distribution from a @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, cauchy_distribution, cd) - { - is >> cd._median >> std::ws >> cd._sigma; - return is; - } - - /** - * Returns true if the two distributions will produce - * identical sequences of values, given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(cauchy_distribution, lhs, rhs) - { return lhs._median == rhs._median && lhs._sigma == rhs._sigma; } - - /** - * Returns true if the two distributions may produce - * different sequences of values, given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(cauchy_distribution) - -private: - RealType _median; - RealType _sigma; -}; - -} // namespace random - -using random::cauchy_distribution; - -} // namespace boost - -#endif // BOOST_RANDOM_CAUCHY_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/chi_squared_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/chi_squared_distribution.hpp deleted file mode 100644 index f0fcce5c5dd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/chi_squared_distribution.hpp +++ /dev/null @@ -1,209 +0,0 @@ -/* boost random/chi_squared_distribution.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_CHI_SQUARED_DISTRIBUTION_HPP_INCLUDED -#define BOOST_RANDOM_CHI_SQUARED_DISTRIBUTION_HPP_INCLUDED - -#include -#include - -#include -#include - -namespace boost { -namespace random { - -/** - * The chi squared distribution is a real valued distribution with - * one parameter, @c n. The distribution produces values > 0. - * - * The distribution function is - * \f$\displaystyle P(x) = \frac{x^{(n/2)-1}e^{-x/2}}{\Gamma(n/2)2^{n/2}}\f$. - */ -template -class chi_squared_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef chi_squared_distribution distribution_type; - /** - * Construct a param_type object. @c n - * is the parameter of the distribution. - * - * Requires: t >=0 && 0 <= p <= 1 - */ - explicit param_type(RealType n_arg = RealType(1)) - : _n(n_arg) - {} - /** Returns the @c n parameter of the distribution. */ - RealType n() const { return _n; } -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const param_type& parm) - { - os << parm._n; - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, param_type& parm) - { - is >> parm._n; - return is; - } -#endif - /** Returns true if the parameters have the same values. */ - friend bool operator==(const param_type& lhs, const param_type& rhs) - { - return lhs._n == rhs._n; - } - /** Returns true if the parameters have different values. */ - friend bool operator!=(const param_type& lhs, const param_type& rhs) - { - return !(lhs == rhs); - } - private: - RealType _n; - }; - - /** - * Construct a @c chi_squared_distribution object. @c n - * is the parameter of the distribution. - * - * Requires: t >=0 && 0 <= p <= 1 - */ - explicit chi_squared_distribution(RealType n_arg = RealType(1)) - : _impl(static_cast(n_arg / 2)) - { - } - - /** - * Construct an @c chi_squared_distribution object from the - * parameters. - */ - explicit chi_squared_distribution(const param_type& parm) - : _impl(static_cast(parm.n() / 2)) - { - } - - /** - * Returns a random variate distributed according to the - * chi squared distribution. - */ - template - RealType operator()(URNG& urng) - { - return 2 * _impl(urng); - } - - /** - * Returns a random variate distributed according to the - * chi squared distribution with parameters specified by @c param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return chi_squared_distribution(parm)(urng); - } - - /** Returns the @c n parameter of the distribution. */ - RealType n() const { return 2 * _impl.alpha(); } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION() const { return 0; } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION() const - { return (std::numeric_limits::infinity)(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(n()); } - /** Sets parameters of the distribution. */ - void param(const param_type& parm) - { - typedef gamma_distribution impl_type; - typename impl_type::param_type impl_parm(static_cast(parm.n() / 2)); - _impl.param(impl_parm); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { _impl.reset(); } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const chi_squared_distribution& c2d) - { - os << c2d.param(); - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, - chi_squared_distribution& c2d) - { - c2d.read(is); - return is; - } -#endif - - /** Returns true if the two distributions will produce the same - sequence of values, given equal generators. */ - friend bool operator==(const chi_squared_distribution& lhs, - const chi_squared_distribution& rhs) - { - return lhs._impl == rhs._impl; - } - /** Returns true if the two distributions could produce different - sequences of values, given equal generators. */ - friend bool operator!=(const chi_squared_distribution& lhs, - const chi_squared_distribution& rhs) - { - return !(lhs == rhs); - } - -private: - - /// @cond show_private - - template - void read(std::basic_istream& is) { - param_type parm; - if(is >> parm) { - param(parm); - } - } - - gamma_distribution _impl; - - /// @endcond -}; - -} - -} - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/auto_link.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/auto_link.hpp deleted file mode 100644 index acbebdd4f01..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/auto_link.hpp +++ /dev/null @@ -1,40 +0,0 @@ -/* boost random auto_link.hpp header file - * - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DETAIL_AUTO_LINK_HPP -#define BOOST_RANDOM_DETAIL_AUTO_LINK_HPP - -#include - -#if defined(BOOST_ALL_DYN_LINK) || defined(BOOST_RANDOM_DYN_LINK) - #if defined(BOOST_RANDOM_SOURCE) - #define BOOST_RANDOM_DECL BOOST_SYMBOL_EXPORT - #else - #define BOOST_RANDOM_DECL BOOST_SYMBOL_IMPORT - #endif -#endif - -#ifndef BOOST_RANDOM_DECL - #define BOOST_RANDOM_DECL -#endif - -#if !defined(BOOST_RANDOM_NO_LIB) && !defined(BOOST_ALL_NO_LIB) && !defined(BOOST_RANDOM_SOURCE) - -#define BOOST_LIB_NAME boost_random - -#if defined(BOOST_RANDOM_DYN_LINK) || defined(BOOST_ALL_DYN_LINK) - #define BOOST_DYN_LINK -#endif - -#include - -#endif - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/config.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/config.hpp deleted file mode 100644 index 724ab194520..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/config.hpp +++ /dev/null @@ -1,18 +0,0 @@ -/* boost random/detail/config.hpp header file - * - * Copyright Steven Watanabe 2009 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#include - -#if (defined(BOOST_NO_OPERATORS_IN_NAMESPACE) || defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS)) \ - && !defined(BOOST_MSVC) - #define BOOST_RANDOM_NO_STREAM_OPERATORS -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/const_mod.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/const_mod.hpp deleted file mode 100644 index e0a43ab2499..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/const_mod.hpp +++ /dev/null @@ -1,216 +0,0 @@ -/* boost random/detail/const_mod.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_CONST_MOD_HPP -#define BOOST_RANDOM_CONST_MOD_HPP - -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { - -template -class const_mod -{ -public: - static IntType apply(IntType x) - { - if(((unsigned_m() - 1) & unsigned_m()) == 0) - return (unsigned_type(x)) & (unsigned_m() - 1); - else { - IntType suppress_warnings = (m == 0); - BOOST_ASSERT(suppress_warnings == 0); - return x % (m + suppress_warnings); - } - } - - static IntType add(IntType x, IntType c) - { - if(((unsigned_m() - 1) & unsigned_m()) == 0) - return (unsigned_type(x) + unsigned_type(c)) & (unsigned_m() - 1); - else if(c == 0) - return x; - else if(x < m - c) - return x + c; - else - return x - (m - c); - } - - static IntType mult(IntType a, IntType x) - { - if(((unsigned_m() - 1) & unsigned_m()) == 0) - return unsigned_type(a) * unsigned_type(x) & (unsigned_m() - 1); - else if(a == 0) - return 0; - else if(a == 1) - return x; - else if(m <= traits::const_max/a) // i.e. a*m <= max - return mult_small(a, x); - else if(traits::is_signed && (m%a < m/a)) - return mult_schrage(a, x); - else - return mult_general(a, x); - } - - static IntType mult_add(IntType a, IntType x, IntType c) - { - if(((unsigned_m() - 1) & unsigned_m()) == 0) - return (unsigned_type(a) * unsigned_type(x) + unsigned_type(c)) & (unsigned_m() - 1); - else if(a == 0) - return c; - else if(m <= (traits::const_max-c)/a) { // i.e. a*m+c <= max - IntType suppress_warnings = (m == 0); - BOOST_ASSERT(suppress_warnings == 0); - return (a*x+c) % (m + suppress_warnings); - } else - return add(mult(a, x), c); - } - - static IntType pow(IntType a, boost::uintmax_t exponent) - { - IntType result = 1; - while(exponent != 0) { - if(exponent % 2 == 1) { - result = mult(result, a); - } - a = mult(a, a); - exponent /= 2; - } - return result; - } - - static IntType invert(IntType x) - { return x == 0 ? 0 : (m == 0? invert_euclidian0(x) : invert_euclidian(x)); } - -private: - typedef integer_traits traits; - typedef typename make_unsigned::type unsigned_type; - - const_mod(); // don't instantiate - - static IntType mult_small(IntType a, IntType x) - { - IntType suppress_warnings = (m == 0); - BOOST_ASSERT(suppress_warnings == 0); - return a*x % (m + suppress_warnings); - } - - static IntType mult_schrage(IntType a, IntType value) - { - const IntType q = m / a; - const IntType r = m % a; - - BOOST_ASSERT(r < q); // check that overflow cannot happen - - return sub(a*(value%q), r*(value/q)); - } - - static IntType mult_general(IntType a, IntType b) - { - IntType suppress_warnings = (m == 0); - BOOST_ASSERT(suppress_warnings == 0); - IntType modulus = m + suppress_warnings; - BOOST_ASSERT(modulus == m); - if(::boost::uintmax_t(modulus) <= - (::std::numeric_limits< ::boost::uintmax_t>::max)() / modulus) - { - return static_cast(boost::uintmax_t(a) * b % modulus); - } else { - return static_cast(detail::mulmod(a, b, modulus)); - } - } - - static IntType sub(IntType a, IntType b) - { - if(a < b) - return m - (b - a); - else - return a - b; - } - - static unsigned_type unsigned_m() - { - if(m == 0) { - return unsigned_type((std::numeric_limits::max)()) + 1; - } else { - return unsigned_type(m); - } - } - - // invert c in the finite field (mod m) (m must be prime) - static IntType invert_euclidian(IntType c) - { - // we are interested in the gcd factor for c, because this is our inverse - BOOST_ASSERT(c > 0); - IntType l1 = 0; - IntType l2 = 1; - IntType n = c; - IntType p = m; - for(;;) { - IntType q = p / n; - l1 += q * l2; - p -= q * n; - if(p == 0) - return l2; - IntType q2 = n / p; - l2 += q2 * l1; - n -= q2 * p; - if(n == 0) - return m - l1; - } - } - - // invert c in the finite field (mod m) (c must be relatively prime to m) - static IntType invert_euclidian0(IntType c) - { - // we are interested in the gcd factor for c, because this is our inverse - BOOST_ASSERT(c > 0); - if(c == 1) return 1; - IntType l1 = 0; - IntType l2 = 1; - IntType n = c; - IntType p = m; - IntType max = (std::numeric_limits::max)(); - IntType q = max / n; - BOOST_ASSERT(max % n != n - 1 && "c must be relatively prime to m."); - l1 += q * l2; - p = max - q * n + 1; - for(;;) { - if(p == 0) - return l2; - IntType q2 = n / p; - l2 += q2 * l1; - n -= q2 * p; - if(n == 0) - return m - l1; - q = p / n; - l1 += q * l2; - p -= q * n; - } - } -}; - -} // namespace random -} // namespace boost - -#include - -#endif // BOOST_RANDOM_CONST_MOD_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/disable_warnings.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/disable_warnings.hpp deleted file mode 100644 index 4582dcb1a7d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/disable_warnings.hpp +++ /dev/null @@ -1,29 +0,0 @@ -/* boost random/detail/disable_warnings.hpp header file - * - * Copyright Steven Watanabe 2009 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -// No #include guard. This header is intended to be included multiple times. - -#include - -#ifdef BOOST_MSVC -#pragma warning(push) -#pragma warning(disable:4512) -#pragma warning(disable:4127) -#pragma warning(disable:4724) -#pragma warning(disable:4800) // 'int' : forcing value to bool 'true' or 'false' (performance warning) -#endif - -#if defined(BOOST_GCC) && BOOST_GCC >= 40600 -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wlogical-op" -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/enable_warnings.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/enable_warnings.hpp deleted file mode 100644 index 24f3bb3f583..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/enable_warnings.hpp +++ /dev/null @@ -1,22 +0,0 @@ -/* boost random/detail/enable_warnings.hpp header file - * - * Copyright Steven Watanabe 2009 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -// No #include guard. This header is intended to be included multiple times. - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -#if defined(BOOST_GCC) && BOOST_GCC >= 40600 -#pragma GCC diagnostic pop -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/generator_bits.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/generator_bits.hpp deleted file mode 100644 index 05276142cbc..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/generator_bits.hpp +++ /dev/null @@ -1,36 +0,0 @@ -/* boost random/detail/generator_bits.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_DETAIL_GENERATOR_BITS_HPP -#define BOOST_RANDOM_DETAIL_GENERATOR_BITS_HPP - -#include - -namespace boost { -namespace random { -namespace detail { - -// This is a temporary measure that retains backwards -// compatibility. -template -struct generator_bits { - static std::size_t value() { - return std::numeric_limits::digits; - } -}; - -} // namespace detail -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_DETAIL_GENERATOR_BITS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/generator_seed_seq.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/generator_seed_seq.hpp deleted file mode 100644 index 7e134834649..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/generator_seed_seq.hpp +++ /dev/null @@ -1,40 +0,0 @@ -/* boost random/mersenne_twister.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_DETAIL_GENERATOR_SEED_SEQ_HPP_INCLUDED -#define BOOST_RANDOM_DETAIL_GENERATOR_SEED_SEQ_HPP_INCLUDED - -namespace boost { -namespace random { -namespace detail { - -template -class generator_seed_seq { -public: - generator_seed_seq(Generator& g) : gen(&g) {} - template - void generate(It first, It last) { - for(; first != last; ++first) { - *first = (*gen)(); - } - } -private: - Generator* gen; -}; - -} -} -} - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/int_float_pair.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/int_float_pair.hpp deleted file mode 100644 index de253a0ecbf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/int_float_pair.hpp +++ /dev/null @@ -1,121 +0,0 @@ -/* boost random/detail/int_float_pair.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2010-2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_DETAIL_INT_FLOAT_PAIR_HPP -#define BOOST_RANDOM_DETAIL_INT_FLOAT_PAIR_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { -namespace detail { - -template -inline typename boost::make_unsigned::type -generate_one_digit(Engine& eng, std::size_t bits) -{ - typedef typename Engine::result_type base_result; - typedef typename boost::make_unsigned::type base_unsigned; - - base_unsigned range = - detail::subtract()((eng.max)(), (eng.min)()); - base_unsigned y0_mask = (base_unsigned(2) << (bits - 1)) - 1; - base_unsigned y0 = (range + 1) & ~y0_mask; - base_unsigned u; - do { - u = detail::subtract()(eng(), (eng.min)()); - } while(y0 != 0 && u > base_unsigned(y0 - 1)); - return u & y0_mask; -} - -template -std::pair generate_int_float_pair(Engine& eng, boost::mpl::true_) -{ - typedef typename Engine::result_type base_result; - typedef typename boost::make_unsigned::type base_unsigned; - - base_unsigned range = - detail::subtract()((eng.max)(), (eng.min)()); - - std::size_t m = - (range == (std::numeric_limits::max)()) ? - std::numeric_limits::digits : - detail::integer_log2(range + 1); - - int bucket = 0; - // process as many full digits as possible into the int part - for(std::size_t i = 0; i < w/m; ++i) { - base_unsigned u = generate_one_digit(eng, m); - bucket = (bucket << m) | u; - } - RealType r; - - const std::size_t digits = std::numeric_limits::digits; - { - base_unsigned u = generate_one_digit(eng, m); - base_unsigned mask = (base_unsigned(1) << (w%m)) - 1; - bucket = (bucket << (w%m)) | (mask & u); - const RealType mult = RealType(1)/RealType(base_unsigned(1) << (m - w%m)); - // zero out unused bits - if (m - w%m > digits) { - u &= ~(base_unsigned(1) << (m - digits)); - } - r = RealType(u >> (w%m)) * mult; - } - for(std::size_t i = m - w%m; i + m < digits; ++i) { - base_unsigned u = generate_one_digit(eng, m); - r += u; - r *= RealType(0.5)/RealType(base_unsigned(1) << (m - 1)); - } - if (m - w%m < digits) - { - const std::size_t remaining = (digits - m + w%m) % m; - base_unsigned u = generate_one_digit(eng, m); - r += u & ((base_unsigned(2) << (remaining - 1)) - 1); - const RealType mult = RealType(0.5)/RealType(base_unsigned(1) << (remaining - 1)); - r *= mult; - } - return std::make_pair(r, bucket); -} - -template -inline std::pair generate_int_float_pair(Engine& eng, boost::mpl::false_) -{ - int bucket = uniform_int_distribution<>(0, (1 << w) - 1)(eng); - RealType r = uniform_01()(eng); - return std::make_pair(r, bucket); -} - -template -inline std::pair generate_int_float_pair(Engine& eng) -{ - typedef typename Engine::result_type base_result; - return generate_int_float_pair(eng, - boost::is_integral()); -} - -} // namespace detail -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_DETAIL_INT_FLOAT_PAIR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/integer_log2.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/integer_log2.hpp deleted file mode 100644 index 248243a4b2e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/integer_log2.hpp +++ /dev/null @@ -1,84 +0,0 @@ -/* boost random/detail/integer_log2.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_DETAIL_INTEGER_LOG2_HPP -#define BOOST_RANDOM_DETAIL_INTEGER_LOG2_HPP - -#include -#include -#include - -namespace boost { -namespace random { -namespace detail { - -#if !defined(BOOST_NO_CXX11_CONSTEXPR) -#define BOOST_RANDOM_DETAIL_CONSTEXPR constexpr -#elif defined(BOOST_MSVC) -#define BOOST_RANDOM_DETAIL_CONSTEXPR __forceinline -#elif defined(__GNUC__) && __GNUC__ >= 4 -#define BOOST_RANDOM_DETAIL_CONSTEXPR inline __attribute__((__const__)) __attribute__((__always_inline__)) -#else -#define BOOST_RANDOM_DETAIL_CONSTEXPR inline -#endif - -template -struct integer_log2_impl -{ -#if defined(BOOST_NO_CXX11_CONSTEXPR) - template - BOOST_RANDOM_DETAIL_CONSTEXPR static int apply(T t, int accum) - { - int update = ((t >> Shift) != 0) * Shift; - return integer_log2_impl::apply(t >> update, accum + update); - } -#else - template - BOOST_RANDOM_DETAIL_CONSTEXPR static int apply2(T t, int accum, int update) - { - return integer_log2_impl::apply(t >> update, accum + update); - } - - template - BOOST_RANDOM_DETAIL_CONSTEXPR static int apply(T t, int accum) - { - return apply2(t, accum, ((t >> Shift) != 0) * Shift); - } -#endif -}; - -template<> -struct integer_log2_impl<1> -{ - template - BOOST_RANDOM_DETAIL_CONSTEXPR static int apply(T t, int accum) - { - return int(t >> 1) + accum; - } -}; - -template -BOOST_RANDOM_DETAIL_CONSTEXPR int integer_log2(T t) -{ - return integer_log2_impl< - ::boost::detail::max_pow2_less< - ::std::numeric_limits::digits, 4 - >::value - >::apply(t, 0); -} - -} // namespace detail -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_DETAIL_INTEGER_LOG2_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/iterator_mixin.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/iterator_mixin.hpp deleted file mode 100644 index 57ff75cecab..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/iterator_mixin.hpp +++ /dev/null @@ -1,45 +0,0 @@ -/* boost random/detail/iterator_mixin.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * Revision history - */ - -#ifndef BOOST_ITERATOR_MIXIN_HPP -#define BOOST_ITERATOR_MIXIN_HPP - -#include - -namespace boost { - -// must be in boost namespace, otherwise the inline friend trick fails -template -class generator_iterator_mixin_adapter - : incrementable, equality_comparable -{ -public: - typedef std::input_iterator_tag iterator_category; - typedef ResultType value_type; - typedef std::ptrdiff_t difference_type; - typedef const value_type * pointer; - typedef const value_type & reference; - Generator& operator++() { v = cast()(); return cast(); } - const value_type& operator*() const { return v; } - -protected: - // instantiate from derived classes only - generator_iterator_mixin_adapter() { } - void iterator_init() { operator++(); } -private: - Generator & cast() { return static_cast(*this); } - value_type v; -}; - -} // namespace boost - -#endif // BOOST_ITERATOR_MIXIN_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/large_arithmetic.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/large_arithmetic.hpp deleted file mode 100644 index 66f6b4e6156..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/large_arithmetic.hpp +++ /dev/null @@ -1,122 +0,0 @@ -/* boost random/detail/large_arithmetic.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DETAIL_LARGE_ARITHMETIC_HPP -#define BOOST_RANDOM_DETAIL_LARGE_ARITHMETIC_HPP - -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { -namespace detail { - -struct div_t { - boost::uintmax_t quotient; - boost::uintmax_t remainder; -}; - -inline div_t muldivmod(boost::uintmax_t a, boost::uintmax_t b, boost::uintmax_t m) -{ - const int bits = - ::std::numeric_limits< ::boost::uintmax_t>::digits / 2; - const ::boost::uintmax_t mask = (::boost::uintmax_t(1) << bits) - 1; - typedef ::boost::uint_t::fast digit_t; - - int shift = std::numeric_limits< ::boost::uintmax_t>::digits - 1 - - detail::integer_log2(m); - - a <<= shift; - m <<= shift; - - digit_t product[4] = { 0, 0, 0, 0 }; - digit_t a_[2] = { digit_t(a & mask), digit_t((a >> bits) & mask) }; - digit_t b_[2] = { digit_t(b & mask), digit_t((b >> bits) & mask) }; - digit_t m_[2] = { digit_t(m & mask), digit_t((m >> bits) & mask) }; - - // multiply a * b - for(int i = 0; i < 2; ++i) { - digit_t carry = 0; - for(int j = 0; j < 2; ++j) { - ::boost::uint64_t temp = ::boost::uintmax_t(a_[i]) * b_[j] + - carry + product[i + j]; - product[i + j] = digit_t(temp & mask); - carry = digit_t(temp >> bits); - } - if(carry != 0) { - product[i + 2] += carry; - } - } - - digit_t quotient[2]; - - if(m == 0) { - div_t result = { - ((::boost::uintmax_t(product[3]) << bits) | product[2]), - ((::boost::uintmax_t(product[1]) << bits) | product[0]) >> shift, - }; - return result; - } - - // divide product / m - for(int i = 3; i >= 2; --i) { - ::boost::uintmax_t temp = - ::boost::uintmax_t(product[i]) << bits | product[i - 1]; - - digit_t q = digit_t((product[i] == m_[1]) ? mask : temp / m_[1]); - - ::boost::uintmax_t rem = - ((temp - ::boost::uintmax_t(q) * m_[1]) << bits) + product[i - 2]; - - ::boost::uintmax_t diff = m_[0] * ::boost::uintmax_t(q); - - int error = 0; - if(diff > rem) { - if(diff - rem > m) { - error = 2; - } else { - error = 1; - } - } - q -= error; - rem = rem + error * m - diff; - - quotient[i - 2] = q; - product[i] = 0; - product[i-1] = static_cast((rem >> bits) & mask); - product[i-2] = static_cast(rem & mask); - } - - div_t result = { - ((::boost::uintmax_t(quotient[1]) << bits) | quotient[0]), - ((::boost::uintmax_t(product[1]) << bits) | product[0]) >> shift, - }; - return result; -} - -inline boost::uintmax_t muldiv(boost::uintmax_t a, boost::uintmax_t b, boost::uintmax_t m) -{ return detail::muldivmod(a, b, m).quotient; } - -inline boost::uintmax_t mulmod(boost::uintmax_t a, boost::uintmax_t b, boost::uintmax_t m) -{ return detail::muldivmod(a, b, m).remainder; } - -} // namespace detail -} // namespace random -} // namespace boost - -#include - -#endif // BOOST_RANDOM_DETAIL_LARGE_ARITHMETIC_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/operators.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/operators.hpp deleted file mode 100644 index 597343c9070..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/operators.hpp +++ /dev/null @@ -1,84 +0,0 @@ -/* boost random/detail/operators.hpp header file - * - * Copyright Steven Watanabe 2010-2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DETAIL_OPERATORS_HPP -#define BOOST_RANDOM_DETAIL_OPERATORS_HPP - -#include -#include - -#if BOOST_WORKAROUND(BOOST_MSVC, <= 1310) \ - || BOOST_WORKAROUND(__SUNPRO_CC, BOOST_TESTED_AT(0x5100)) - -#define BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, T, t) \ - template \ - friend std::basic_ostream& \ - operator<<(std::basic_ostream& os, const T& t) { \ - t.print(os, t); \ - return os; \ - } \ - template \ - static std::basic_ostream& \ - print(std::basic_ostream& os, const T& t) - -#define BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, T, t) \ - template \ - friend std::basic_istream& \ - operator>>(std::basic_istream& is, T& t) { \ - t.read(is, t); \ - return is; \ - } \ - template \ - static std::basic_istream& \ - read(std::basic_istream& is, T& t) - -#endif - -#if defined(__BORLANDC__) - -#define BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(T, lhs, rhs) \ - bool operator==(const T& rhs) const \ - { return T::is_equal(*this, rhs); } \ - static bool is_equal(const T& lhs, const T& rhs) - -#define BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(T) \ - bool operator!=(const T& rhs) const \ - { return !T::is_equal(*this, rhs); } - -#endif - -#ifndef BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR -#define BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, T, t) \ - template \ - friend std::basic_ostream& \ - operator<<(std::basic_ostream& os, const T& t) -#endif - -#ifndef BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR -#define BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, T, t) \ - template \ - friend std::basic_istream& \ - operator>>(std::basic_istream& is, T& t) -#endif - -#ifndef BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR -#define BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(T, lhs, rhs) \ - friend bool operator==(const T& lhs, const T& rhs) -#endif - -#ifndef BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR -#define BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(T) \ - friend bool operator!=(const T& lhs, const T& rhs) \ - { return !(lhs == rhs); } -#endif - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/polynomial.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/polynomial.hpp deleted file mode 100644 index a8c4b269f1f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/polynomial.hpp +++ /dev/null @@ -1,384 +0,0 @@ -/* boost random/detail/polynomial.hpp header file - * - * Copyright Steven Watanabe 2014 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DETAIL_POLYNOMIAL_HPP -#define BOOST_RANDOM_DETAIL_POLYNOMIAL_HPP - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { -namespace detail { - -class polynomial_ops { -public: - typedef unsigned long digit_t; - - static void add(std::size_t size, const digit_t * lhs, - const digit_t * rhs, digit_t * output) - { - for(std::size_t i = 0; i < size; ++i) { - output[i] = lhs[i] ^ rhs[i]; - } - } - - static void add_shifted_inplace(std::size_t size, const digit_t * lhs, - digit_t * output, std::size_t shift) - { - if(shift == 0) { - add(size, lhs, output, output); - return; - } - std::size_t bits = std::numeric_limits::digits; - digit_t prev = 0; - for(std::size_t i = 0; i < size; ++i) { - digit_t tmp = lhs[i]; - output[i] ^= (tmp << shift) | (prev >> (bits-shift)); - prev = tmp; - } - output[size] ^= (prev >> (bits-shift)); - } - - static void multiply_simple(std::size_t size, const digit_t * lhs, - const digit_t * rhs, digit_t * output) - { - std::size_t bits = std::numeric_limits::digits; - for(std::size_t i = 0; i < 2*size; ++i) { - output[i] = 0; - } - for(std::size_t i = 0; i < size; ++i) { - for(std::size_t j = 0; j < bits; ++j) { - if((lhs[i] & (digit_t(1) << j)) != 0) { - add_shifted_inplace(size, rhs, output + i, j); - } - } - } - } - - // memory requirements: (size - cutoff) * 4 + next_smaller - static void multiply_karatsuba(std::size_t size, - const digit_t * lhs, const digit_t * rhs, - digit_t * output) - { - if(size < 64) { - multiply_simple(size, lhs, rhs, output); - return; - } - // split in half - std::size_t cutoff = size/2; - multiply_karatsuba(cutoff, lhs, rhs, output); - multiply_karatsuba(size - cutoff, lhs + cutoff, rhs + cutoff, - output + cutoff*2); - std::vector local1(size - cutoff); - std::vector local2(size - cutoff); - // combine the digits for the inner multiply - add(cutoff, lhs, lhs + cutoff, &local1[0]); - if(size & 1) local1[cutoff] = lhs[size - 1]; - add(cutoff, rhs + cutoff, rhs, &local2[0]); - if(size & 1) local2[cutoff] = rhs[size - 1]; - std::vector local3((size - cutoff) * 2); - multiply_karatsuba(size - cutoff, &local1[0], &local2[0], &local3[0]); - add(cutoff * 2, output, &local3[0], &local3[0]); - add((size - cutoff) * 2, output + cutoff*2, &local3[0], &local3[0]); - // Finally, add the inner result - add((size - cutoff) * 2, output + cutoff, &local3[0], output + cutoff); - } - - static void multiply_add_karatsuba(std::size_t size, - const digit_t * lhs, const digit_t * rhs, - digit_t * output) - { - std::vector buf(size * 2); - multiply_karatsuba(size, lhs, rhs, &buf[0]); - add(size * 2, &buf[0], output, output); - } - - static void multiply(const digit_t * lhs, std::size_t lhs_size, - const digit_t * rhs, std::size_t rhs_size, - digit_t * output) - { - std::fill_n(output, lhs_size + rhs_size, digit_t(0)); - multiply_add(lhs, lhs_size, rhs, rhs_size, output); - } - - static void multiply_add(const digit_t * lhs, std::size_t lhs_size, - const digit_t * rhs, std::size_t rhs_size, - digit_t * output) - { - // split into pieces that can be passed to - // karatsuba multiply. - while(lhs_size != 0) { - if(lhs_size < rhs_size) { - std::swap(lhs, rhs); - std::swap(lhs_size, rhs_size); - } - - multiply_add_karatsuba(rhs_size, lhs, rhs, output); - - lhs += rhs_size; - lhs_size -= rhs_size; - output += rhs_size; - } - } - - static void copy_bits(const digit_t * x, std::size_t low, std::size_t high, - digit_t * out) - { - const std::size_t bits = std::numeric_limits::digits; - std::size_t offset = low/bits; - x += offset; - low -= offset*bits; - high -= offset*bits; - std::size_t n = (high-low)/bits; - if(low == 0) { - for(std::size_t i = 0; i < n; ++i) { - out[i] = x[i]; - } - } else { - for(std::size_t i = 0; i < n; ++i) { - out[i] = (x[i] >> low) | (x[i+1] << (bits-low)); - } - } - if((high-low)%bits) { - digit_t low_mask = (digit_t(1) << ((high-low)%bits)) - 1; - digit_t result = (x[n] >> low); - if(low != 0 && (n+1)*bits < high) { - result |= (x[n+1] << (bits-low)); - } - out[n] = (result & low_mask); - } - } - - static void shift_left(digit_t * val, std::size_t size, std::size_t shift) - { - const std::size_t bits = std::numeric_limits::digits; - BOOST_ASSERT(shift > 0); - BOOST_ASSERT(shift < bits); - digit_t prev = 0; - for(std::size_t i = 0; i < size; ++i) { - digit_t tmp = val[i]; - val[i] = (prev >> (bits - shift)) | (val[i] << shift); - prev = tmp; - } - } - - static digit_t sqr(digit_t val) { - const std::size_t bits = std::numeric_limits::digits; - digit_t mask = (digit_t(1) << bits/2) - 1; - for(std::size_t i = bits; i > 1; i /= 2) { - val = ((val & ~mask) << i/2) | (val & mask); - mask = mask & (mask >> i/4); - mask = mask | (mask << i/2); - } - return val; - } - - static void sqr(digit_t * val, std::size_t size) - { - const std::size_t bits = std::numeric_limits::digits; - digit_t mask = (digit_t(1) << bits/2) - 1; - for(std::size_t i = 0; i < size; ++i) { - digit_t x = val[size - i - 1]; - val[(size - i - 1) * 2] = sqr(x & mask); - val[(size - i - 1) * 2 + 1] = sqr(x >> bits/2); - } - } - - // optimized for the case when the modulus has few bits set. - struct sparse_mod { - sparse_mod(const digit_t * divisor, std::size_t divisor_bits) - { - const std::size_t bits = std::numeric_limits::digits; - _remainder_bits = divisor_bits - 1; - for(std::size_t i = 0; i < divisor_bits; ++i) { - if(divisor[i/bits] & (digit_t(1) << i%bits)) { - _bit_indices.push_back(i); - } - } - BOOST_ASSERT(_bit_indices.back() == divisor_bits - 1); - _bit_indices.pop_back(); - if(_bit_indices.empty()) { - _block_bits = divisor_bits; - _lower_bits = 0; - } else { - _block_bits = divisor_bits - _bit_indices.back() - 1; - _lower_bits = _bit_indices.back() + 1; - } - - _partial_quotient.resize((_block_bits + bits - 1)/bits); - } - void operator()(digit_t * dividend, std::size_t dividend_bits) - { - const std::size_t bits = std::numeric_limits::digits; - while(dividend_bits > _remainder_bits) { - std::size_t block_start = (std::max)(dividend_bits - _block_bits, _remainder_bits); - std::size_t block_size = (dividend_bits - block_start + bits - 1) / bits; - copy_bits(dividend, block_start, dividend_bits, &_partial_quotient[0]); - for(std::size_t i = 0; i < _bit_indices.size(); ++i) { - std::size_t pos = _bit_indices[i] + block_start - _remainder_bits; - add_shifted_inplace(block_size, &_partial_quotient[0], dividend + pos/bits, pos%bits); - } - add_shifted_inplace(block_size, &_partial_quotient[0], dividend + block_start/bits, block_start%bits); - dividend_bits = block_start; - } - } - std::vector _partial_quotient; - std::size_t _remainder_bits; - std::size_t _block_bits; - std::size_t _lower_bits; - std::vector _bit_indices; - }; - - // base should have the same number of bits as mod - // base, and mod should both be able to hold a power - // of 2 >= mod_bits. out needs to be twice as large. - static void mod_pow_x(boost::uintmax_t exponent, const digit_t * mod, std::size_t mod_bits, digit_t * out) - { - const std::size_t bits = std::numeric_limits::digits; - const std::size_t n = (mod_bits + bits - 1) / bits; - const std::size_t highbit = mod_bits - 1; - if(exponent == 0) { - out[0] = 1; - std::fill_n(out + 1, n - 1, digit_t(0)); - return; - } - boost::uintmax_t i = std::numeric_limits::digits - 1; - while(((boost::uintmax_t(1) << i) & exponent) == 0) { - --i; - } - out[0] = 2; - std::fill_n(out + 1, n - 1, digit_t(0)); - sparse_mod m(mod, mod_bits); - while(i--) { - sqr(out, n); - m(out, 2 * mod_bits - 1); - if((boost::uintmax_t(1) << i) & exponent) { - shift_left(out, n, 1); - if(out[highbit / bits] & (digit_t(1) << highbit%bits)) - add(n, out, mod, out); - } - } - } -}; - -class polynomial -{ - typedef polynomial_ops::digit_t digit_t; -public: - polynomial() : _size(0) {} - class reference { - public: - reference(digit_t &value, int idx) - : _value(value), _idx(idx) {} - operator bool() const { return (_value & (digit_t(1) << _idx)) != 0; } - reference& operator=(bool b) - { - if(b) { - _value |= (digit_t(1) << _idx); - } else { - _value &= ~(digit_t(1) << _idx); - } - return *this; - } - reference &operator^=(bool b) - { - _value ^= (digit_t(b) << _idx); - return *this; - } - - reference &operator=(const reference &other) - { - return *this = static_cast(other); - } - private: - digit_t &_value; - int _idx; - }; - reference operator[](std::size_t i) - { - static const std::size_t bits = std::numeric_limits::digits; - ensure_bit(i); - return reference(_storage[i/bits], i%bits); - } - bool operator[](std::size_t i) const - { - static const std::size_t bits = std::numeric_limits::digits; - if(i < size()) - return (_storage[i/bits] & (digit_t(1) << (i%bits))) != 0; - else - return false; - } - std::size_t size() const - { - return _size; - } - void resize(std::size_t n) - { - static const std::size_t bits = std::numeric_limits::digits; - _storage.resize((n + bits - 1)/bits); - // clear the high order bits in case we're shrinking. - if(n%bits) { - _storage.back() &= ((digit_t(1) << (n%bits)) - 1); - } - _size = n; - } - friend polynomial operator*(const polynomial &lhs, const polynomial &rhs); - friend polynomial mod_pow_x(boost::uintmax_t exponent, polynomial mod); -private: - std::vector _storage; - std::size_t _size; - void ensure_bit(std::size_t i) - { - if(i >= size()) { - resize(i + 1); - } - } - void normalize() - { - while(size() && (*this)[size() - 1] == 0) - resize(size() - 1); - } -}; - -inline polynomial operator*(const polynomial &lhs, const polynomial &rhs) -{ - polynomial result; - result._storage.resize(lhs._storage.size() + rhs._storage.size()); - polynomial_ops::multiply(&lhs._storage[0], lhs._storage.size(), - &rhs._storage[0], rhs._storage.size(), - &result._storage[0]); - result._size = lhs._size + rhs._size; - return result; -} - -inline polynomial mod_pow_x(boost::uintmax_t exponent, polynomial mod) -{ - polynomial result; - mod.normalize(); - std::size_t mod_size = mod.size(); - result._storage.resize(mod._storage.size() * 2); - result._size = mod.size() * 2; - polynomial_ops::mod_pow_x(exponent, &mod._storage[0], mod_size, &result._storage[0]); - result.resize(mod.size() - 1); - return result; -} - -} -} -} - -#endif // BOOST_RANDOM_DETAIL_POLYNOMIAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/ptr_helper.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/ptr_helper.hpp deleted file mode 100644 index f1b983d9367..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/ptr_helper.hpp +++ /dev/null @@ -1,67 +0,0 @@ -/* boost random/detail/ptr_helper.hpp header file - * - * Copyright Jens Maurer 2002 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_DETAIL_PTR_HELPER_HPP -#define BOOST_RANDOM_DETAIL_PTR_HELPER_HPP - -#include - - -namespace boost { -namespace random { -namespace detail { - -// type_traits could help here, but I don't want to depend on type_traits. -template -struct ptr_helper -{ - typedef T value_type; - typedef T& reference_type; - typedef const T& rvalue_type; - static reference_type ref(T& r) { return r; } - static const T& ref(const T& r) { return r; } -}; - -template -struct ptr_helper -{ - typedef T value_type; - typedef T& reference_type; - typedef T& rvalue_type; - static reference_type ref(T& r) { return r; } - static const T& ref(const T& r) { return r; } -}; - -template -struct ptr_helper -{ - typedef T value_type; - typedef T& reference_type; - typedef T* rvalue_type; - static reference_type ref(T * p) { return *p; } - static const T& ref(const T * p) { return *p; } -}; - -} // namespace detail -} // namespace random -} // namespace boost - -// -// BOOST_RANDOM_PTR_HELPER_SPEC -- -// -// Helper macro for broken compilers defines specializations of -// ptr_helper. -// -# define BOOST_RANDOM_PTR_HELPER_SPEC(T) - -#endif // BOOST_RANDOM_DETAIL_PTR_HELPER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/seed.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/seed.hpp deleted file mode 100644 index 557482ad4b1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/seed.hpp +++ /dev/null @@ -1,115 +0,0 @@ -/* boost random/detail/seed.hpp header file - * - * Copyright Steven Watanabe 2009 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DETAIL_SEED_HPP -#define BOOST_RANDOM_DETAIL_SEED_HPP - -#include - -// Sun seems to have trouble with the use of SFINAE for the -// templated constructor. So does Borland. -#if !defined(BOOST_NO_SFINAE) && !defined(__SUNPRO_CC) && !defined(__BORLANDC__) - -#include -#include -#include - -namespace boost { -namespace random { -namespace detail { - -template -struct disable_seed : boost::disable_if > {}; - -template -struct disable_constructor : disable_seed {}; - -template -struct disable_constructor {}; - -#define BOOST_RANDOM_DETAIL_GENERATOR_CONSTRUCTOR(Self, Generator, gen) \ - template \ - explicit Self(Generator& gen, typename ::boost::random::detail::disable_constructor::type* = 0) - -#define BOOST_RANDOM_DETAIL_GENERATOR_SEED(Self, Generator, gen) \ - template \ - void seed(Generator& gen, typename ::boost::random::detail::disable_seed::type* = 0) - -#define BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(Self, SeedSeq, seq) \ - template \ - explicit Self(SeedSeq& seq, typename ::boost::random::detail::disable_constructor::type* = 0) - -#define BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(Self, SeedSeq, seq) \ - template \ - void seed(SeedSeq& seq, typename ::boost::random::detail::disable_seed::type* = 0) - -#define BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(Self, T, x) \ - explicit Self(const T& x) - -#define BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(Self, T, x) \ - void seed(const T& x) -} -} -} - -#else - -#include -#include - -#define BOOST_RANDOM_DETAIL_GENERATOR_CONSTRUCTOR(Self, Generator, gen) \ - Self(Self& other) { *this = other; } \ - Self(const Self& other) { *this = other; } \ - template \ - explicit Self(Generator& gen) { \ - boost_random_constructor_impl(gen, ::boost::is_arithmetic());\ - } \ - template \ - void boost_random_constructor_impl(Generator& gen, ::boost::mpl::false_) - -#define BOOST_RANDOM_DETAIL_GENERATOR_SEED(Self, Generator, gen) \ - template \ - void seed(Generator& gen) { \ - boost_random_seed_impl(gen, ::boost::is_arithmetic());\ - }\ - template\ - void boost_random_seed_impl(Generator& gen, ::boost::mpl::false_) - -#define BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(Self, SeedSeq, seq) \ - Self(Self& other) { *this = other; } \ - Self(const Self& other) { *this = other; } \ - template \ - explicit Self(SeedSeq& seq) { \ - boost_random_constructor_impl(seq, ::boost::is_arithmetic());\ - } \ - template \ - void boost_random_constructor_impl(SeedSeq& seq, ::boost::mpl::false_) - -#define BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(Self, SeedSeq, seq) \ - template \ - void seed(SeedSeq& seq) { \ - boost_random_seed_impl(seq, ::boost::is_arithmetic()); \ - } \ - template \ - void boost_random_seed_impl(SeedSeq& seq, ::boost::mpl::false_) - -#define BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(Self, T, x) \ - explicit Self(const T& x) { boost_random_constructor_impl(x, ::boost::mpl::true_()); }\ - void boost_random_constructor_impl(const T& x, ::boost::mpl::true_) - -#define BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(Self, T, x) \ - void seed(const T& x) { boost_random_seed_impl(x, ::boost::mpl::true_()); }\ - void boost_random_seed_impl(const T& x, ::boost::mpl::true_) - -#endif - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/seed_impl.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/seed_impl.hpp deleted file mode 100644 index 918a2944579..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/seed_impl.hpp +++ /dev/null @@ -1,398 +0,0 @@ -/* boost random/detail/seed.hpp header file - * - * Copyright Steven Watanabe 2009 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DETAIL_SEED_IMPL_HPP -#define BOOST_RANDOM_DETAIL_SEED_IMPL_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { -namespace detail { - -// finds the seed type of an engine, given its -// result_type. If the result_type is integral -// the seed type is the same. If the result_type -// is floating point, the seed type is uint32_t -template -struct seed_type -{ - typedef typename boost::mpl::if_, - T, - boost::uint32_t - >::type type; -}; - -template -struct const_pow_impl -{ - template - static T call(T arg, int n, T result) - { - return const_pow_impl::call(T(arg * arg), n / 2, - n%2 == 0? result : T(result * arg)); - } -}; - -template<> -struct const_pow_impl<0> -{ - template - static T call(T, int, T result) - { - return result; - } -}; - -// requires N is an upper bound on n -template -inline T const_pow(T arg, int n) { return const_pow_impl::call(arg, n, T(1)); } - -template -inline T pow2(int n) -{ - typedef unsigned int_type; - const int max_bits = std::numeric_limits::digits; - T multiplier = T(int_type(1) << (max_bits - 1)) * 2; - return (int_type(1) << (n % max_bits)) * - const_pow::digits / max_bits>(multiplier, n / max_bits); -} - -template -void generate_from_real(Engine& eng, Iter begin, Iter end) -{ - using std::fmod; - typedef typename Engine::result_type RealType; - const int Bits = detail::generator_bits::value(); - int remaining_bits = 0; - boost::uint_least32_t saved_bits = 0; - RealType multiplier = pow2( Bits); - RealType mult32 = RealType(4294967296.0); // 2^32 - while(true) { - RealType val = eng() * multiplier; - int available_bits = Bits; - // Make sure the compiler can optimize this out - // if it isn't possible. - if(Bits < 32 && available_bits < 32 - remaining_bits) { - saved_bits |= boost::uint_least32_t(val) << remaining_bits; - remaining_bits += Bits; - } else { - // If Bits < 32, then remaining_bits != 0, since - // if remaining_bits == 0, available_bits < 32 - 0, - // and we won't get here to begin with. - if(Bits < 32 || remaining_bits != 0) { - boost::uint_least32_t divisor = - (boost::uint_least32_t(1) << (32 - remaining_bits)); - boost::uint_least32_t extra_bits = boost::uint_least32_t(fmod(val, mult32)) & (divisor - 1); - val = val / divisor; - *begin++ = saved_bits | (extra_bits << remaining_bits); - if(begin == end) return; - available_bits -= 32 - remaining_bits; - remaining_bits = 0; - } - // If Bits < 32 we should never enter this loop - if(Bits >= 32) { - for(; available_bits >= 32; available_bits -= 32) { - boost::uint_least32_t word = boost::uint_least32_t(fmod(val, mult32)); - val /= mult32; - *begin++ = word; - if(begin == end) return; - } - } - remaining_bits = available_bits; - saved_bits = static_cast(val); - } - } -} - -template -void generate_from_int(Engine& eng, Iter begin, Iter end) -{ - typedef typename Engine::result_type IntType; - typedef typename boost::random::traits::make_unsigned::type unsigned_type; - int remaining_bits = 0; - boost::uint_least32_t saved_bits = 0; - unsigned_type range = boost::random::detail::subtract()((eng.max)(), (eng.min)()); - - int bits = - (range == (std::numeric_limits::max)()) ? - std::numeric_limits::digits : - detail::integer_log2(range + 1); - - { - int discarded_bits = detail::integer_log2(bits); - unsigned_type excess = (range + 1) >> (bits - discarded_bits); - if(excess != 0) { - int extra_bits = detail::integer_log2((excess - 1) ^ excess); - bits = bits - discarded_bits + extra_bits; - } - } - - unsigned_type mask = (static_cast(2) << (bits - 1)) - 1; - unsigned_type limit = ((range + 1) & ~mask) - 1; - - while(true) { - unsigned_type val; - do { - val = boost::random::detail::subtract()(eng(), (eng.min)()); - } while(limit != range && val > limit); - val &= mask; - int available_bits = bits; - if(available_bits == 32) { - *begin++ = static_cast(val) & 0xFFFFFFFFu; - if(begin == end) return; - } else if(available_bits % 32 == 0) { - for(int i = 0; i < available_bits / 32; ++i) { - boost::uint_least32_t word = boost::uint_least32_t(val) & 0xFFFFFFFFu; - int suppress_warning = (bits >= 32); - BOOST_ASSERT(suppress_warning == 1); - val >>= (32 * suppress_warning); - *begin++ = word; - if(begin == end) return; - } - } else if(bits < 32 && available_bits < 32 - remaining_bits) { - saved_bits |= boost::uint_least32_t(val) << remaining_bits; - remaining_bits += bits; - } else { - if(bits < 32 || remaining_bits != 0) { - boost::uint_least32_t extra_bits = boost::uint_least32_t(val) & ((boost::uint_least32_t(1) << (32 - remaining_bits)) - 1); - val >>= 32 - remaining_bits; - *begin++ = saved_bits | (extra_bits << remaining_bits); - if(begin == end) return; - available_bits -= 32 - remaining_bits; - remaining_bits = 0; - } - if(bits >= 32) { - for(; available_bits >= 32; available_bits -= 32) { - boost::uint_least32_t word = boost::uint_least32_t(val) & 0xFFFFFFFFu; - int suppress_warning = (bits >= 32); - BOOST_ASSERT(suppress_warning == 1); - val >>= (32 * suppress_warning); - *begin++ = word; - if(begin == end) return; - } - } - remaining_bits = available_bits; - saved_bits = static_cast(val); - } - } -} - -template -void generate_impl(Engine& eng, Iter first, Iter last, boost::mpl::true_) -{ - return detail::generate_from_int(eng, first, last); -} - -template -void generate_impl(Engine& eng, Iter first, Iter last, boost::mpl::false_) -{ - return detail::generate_from_real(eng, first, last); -} - -template -void generate(Engine& eng, Iter first, Iter last) -{ - return detail::generate_impl(eng, first, last, boost::random::traits::is_integral()); -} - - - -template -IntType seed_one_int(SeedSeq& seq) -{ - static const int log = ::boost::mpl::if_c<(m == 0), - ::boost::mpl::int_<(::std::numeric_limits::digits)>, - ::boost::static_log2 >::type::value; - static const int k = - (log + ((~(static_cast(2) << (log - 1)) & m)? 32 : 31)) / 32; - ::boost::uint_least32_t array[log / 32 + 4]; - seq.generate(&array[0], &array[0] + k + 3); - IntType s = 0; - for(int j = 0; j < k; ++j) { - IntType digit = const_mod::apply(IntType(array[j+3])); - IntType mult = IntType(1) << 32*j; - s = const_mod::mult_add(mult, digit, s); - } - return s; -} - -template -IntType get_one_int(Iter& first, Iter last) -{ - static const int log = ::boost::mpl::if_c<(m == 0), - ::boost::mpl::int_<(::std::numeric_limits::digits)>, - ::boost::static_log2 >::type::value; - static const int k = - (log + ((~(static_cast(2) << (log - 1)) & m)? 32 : 31)) / 32; - IntType s = 0; - for(int j = 0; j < k; ++j) { - if(first == last) { - boost::throw_exception(::std::invalid_argument("Not enough elements in call to seed.")); - } - IntType digit = const_mod::apply(IntType(*first++)); - IntType mult = IntType(1) << 32*j; - s = const_mod::mult_add(mult, digit, s); - } - return s; -} - -// TODO: work in-place whenever possible -template -void seed_array_int_impl(SeedSeq& seq, UIntType (&x)[n]) -{ - boost::uint_least32_t storage[((w+31)/32) * n]; - seq.generate(&storage[0], &storage[0] + ((w+31)/32) * n); - for(std::size_t j = 0; j < n; j++) { - UIntType val = 0; - for(std::size_t k = 0; k < (w+31)/32; ++k) { - val += static_cast(storage[(w+31)/32*j + k]) << 32*k; - } - x[j] = val & ::boost::low_bits_mask_t::sig_bits; - } -} - -template -inline void seed_array_int_impl(SeedSeq& seq, IntType (&x)[n], boost::mpl::true_) -{ - BOOST_STATIC_ASSERT_MSG(boost::is_integral::value, "Sorry but this routine has not been ported to non built-in integers as it relies on a reinterpret_cast."); - typedef typename boost::make_unsigned::type unsigned_array[n]; - seed_array_int_impl(seq, reinterpret_cast(x)); -} - -template -inline void seed_array_int_impl(SeedSeq& seq, IntType (&x)[n], boost::mpl::false_) -{ - seed_array_int_impl(seq, x); -} - -template -inline void seed_array_int(SeedSeq& seq, IntType (&x)[n]) -{ - seed_array_int_impl(seq, x, boost::random::traits::is_signed()); -} - -template -void fill_array_int_impl(Iter& first, Iter last, UIntType (&x)[n]) -{ - for(std::size_t j = 0; j < n; j++) { - UIntType val = 0; - for(std::size_t k = 0; k < (w+31)/32; ++k) { - if(first == last) { - boost::throw_exception(std::invalid_argument("Not enough elements in call to seed.")); - } - val += static_cast(*first++) << 32*k; - } - x[j] = val & ::boost::low_bits_mask_t::sig_bits; - } -} - -template -inline void fill_array_int_impl(Iter& first, Iter last, IntType (&x)[n], boost::mpl::true_) -{ - BOOST_STATIC_ASSERT_MSG(boost::is_integral::value, "Sorry but this routine has not been ported to non built-in integers as it relies on a reinterpret_cast."); - typedef typename boost::make_unsigned::type unsigned_array[n]; - fill_array_int_impl(first, last, reinterpret_cast(x)); -} - -template -inline void fill_array_int_impl(Iter& first, Iter last, IntType (&x)[n], boost::mpl::false_) -{ - fill_array_int_impl(first, last, x); -} - -template -inline void fill_array_int(Iter& first, Iter last, IntType (&x)[n]) -{ - fill_array_int_impl(first, last, x, boost::random::traits::is_signed()); -} - -template -void seed_array_real_impl(const boost::uint_least32_t* storage, RealType (&x)[n]) -{ - boost::uint_least32_t mask = ~((~boost::uint_least32_t(0)) << (w%32)); - RealType two32 = 4294967296.0; - const RealType divisor = RealType(1)/detail::pow2(w); - unsigned int j; - for(j = 0; j < n; ++j) { - RealType val = RealType(0); - RealType mult = divisor; - for(int k = 0; k < w/32; ++k) { - val += *storage++ * mult; - mult *= two32; - } - if(mask != 0) { - val += (*storage++ & mask) * mult; - } - BOOST_ASSERT(val >= 0); - BOOST_ASSERT(val < 1); - x[j] = val; - } -} - -template -void seed_array_real(SeedSeq& seq, RealType (&x)[n]) -{ - using std::pow; - boost::uint_least32_t storage[((w+31)/32) * n]; - seq.generate(&storage[0], &storage[0] + ((w+31)/32) * n); - seed_array_real_impl(storage, x); -} - -template -void fill_array_real(Iter& first, Iter last, RealType (&x)[n]) -{ - boost::uint_least32_t mask = ~((~boost::uint_least32_t(0)) << (w%32)); - RealType two32 = 4294967296.0; - const RealType divisor = RealType(1)/detail::pow2(w); - unsigned int j; - for(j = 0; j < n; ++j) { - RealType val = RealType(0); - RealType mult = divisor; - for(int k = 0; k < w/32; ++k, ++first) { - if(first == last) boost::throw_exception(std::invalid_argument("Not enough elements in call to seed.")); - val += *first * mult; - mult *= two32; - } - if(mask != 0) { - if(first == last) boost::throw_exception(std::invalid_argument("Not enough elements in call to seed.")); - val += (*first & mask) * mult; - ++first; - } - BOOST_ASSERT(val >= 0); - BOOST_ASSERT(val < 1); - x[j] = val; - } -} - -} -} -} - -#include - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/signed_unsigned_tools.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/signed_unsigned_tools.hpp deleted file mode 100644 index 1979908a312..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/signed_unsigned_tools.hpp +++ /dev/null @@ -1,89 +0,0 @@ -/* boost random/detail/signed_unsigned_tools.hpp header file - * - * Copyright Jens Maurer 2006 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - */ - -#ifndef BOOST_RANDOM_DETAIL_SIGNED_UNSIGNED_TOOLS -#define BOOST_RANDOM_DETAIL_SIGNED_UNSIGNED_TOOLS - -#include -#include -#include - -namespace boost { -namespace random { -namespace detail { - - -/* - * Compute x - y, we know that x >= y, return an unsigned value. - */ - -template::is_signed && std::numeric_limits::is_bounded> -struct subtract { }; - -template -struct subtract -{ - typedef T result_type; - result_type operator()(T x, T y) { return x - y; } -}; - -template -struct subtract -{ - typedef typename boost::random::traits::make_unsigned_or_unbounded::type result_type; - result_type operator()(T x, T y) - { - if (y >= 0) // because x >= y, it follows that x >= 0, too - return result_type(x) - result_type(y); - if (x >= 0) // y < 0 - // avoid the nasty two's complement case for y == min() - return result_type(x) + result_type(-(y+1)) + 1; - // both x and y are negative: no signed overflow - return result_type(x - y); - } -}; - -/* - * Compute x + y, x is unsigned, result fits in type of "y". - */ - -template::is_signed && (std::numeric_limits::digits >= std::numeric_limits::digits))> -struct add { }; - -template -struct add -{ - typedef T2 result_type; - result_type operator()(T1 x, T2 y) { return T2(x) + y; } -}; - -template -struct add -{ - typedef T2 result_type; - result_type operator()(T1 x, T2 y) - { - if (y >= 0) - return T2(x) + y; - // y < 0 - if (x > T1(-(y+1))) // result >= 0 after subtraction - // avoid the nasty two's complement edge case for y == min() - return T2(x - T1(-(y+1)) - 1); - // abs(x) < abs(y), thus T2 able to represent x - return T2(x) + y; - } -}; - -} // namespace detail -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_DETAIL_SIGNED_UNSIGNED_TOOLS - diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/uniform_int_float.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/uniform_int_float.hpp deleted file mode 100644 index 393c455bcde..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/uniform_int_float.hpp +++ /dev/null @@ -1,76 +0,0 @@ -/* boost random/detail/uniform_int_float.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_DETAIL_UNIFORM_INT_FLOAT_HPP -#define BOOST_RANDOM_DETAIL_UNIFORM_INT_FLOAT_HPP - -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { -namespace detail { - -template -class uniform_int_float -{ -public: - typedef URNG base_type; - typedef typename base_type::result_type base_result; - - typedef typename boost::uint_t< - (std::numeric_limits::digits < - std::numeric_limits::digits)? - std::numeric_limits::digits : - std::numeric_limits::digits - >::fast result_type; - - uniform_int_float(base_type& rng) - : _rng(rng) {} - - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return 0; } - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { - std::size_t digits = std::numeric_limits::digits; - if(detail::generator_bits::value() < digits) { - digits = detail::generator_bits::value(); - } - return (result_type(2) << (digits - 1)) - 1; - } - base_type& base() { return _rng; } - const base_type& base() const { return _rng; } - - result_type operator()() - { - base_result range = static_cast((max)())+1; - return static_cast(_rng() * range); - } - -private: - base_type& _rng; -}; - -} // namespace detail -} // namespace random -} // namespace boost - -#include - -#endif // BOOST_RANDOM_DETAIL_UNIFORM_INT_FLOAT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/detail/vector_io.hpp b/contrib/libboost/boost_1_65_0/boost/random/detail/vector_io.hpp deleted file mode 100644 index 24508c210e1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/detail/vector_io.hpp +++ /dev/null @@ -1,75 +0,0 @@ -/* boost random/vector_io.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DETAIL_VECTOR_IO_HPP -#define BOOST_RANDOM_DETAIL_VECTOR_IO_HPP - -#include -#include -#include - -namespace boost { -namespace random { -namespace detail { - -template -void print_vector(std::basic_ostream& os, - const std::vector& vec) -{ - typename std::vector::const_iterator - iter = vec.begin(), - end = vec.end(); - os << os.widen('['); - if(iter != end) { - os << *iter; - ++iter; - for(; iter != end; ++iter) - { - os << os.widen(' ') << *iter; - } - } - os << os.widen(']'); -} - -template -void read_vector(std::basic_istream& is, std::vector& vec) -{ - CharT ch; - if(!(is >> ch)) { - return; - } - if(ch != is.widen('[')) { - is.putback(ch); - is.setstate(std::ios_base::failbit); - return; - } - T val; - while(is >> std::ws >> val) { - vec.push_back(val); - } - if(is.fail()) { - is.clear(); - if(!(is >> ch)) { - return; - } - if(ch != is.widen(']')) { - is.putback(ch); - is.setstate(std::ios_base::failbit); - } - } -} - -} -} -} - -#endif // BOOST_RANDOM_DETAIL_VECTOR_IO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/discard_block.hpp b/contrib/libboost/boost_1_65_0/boost/random/discard_block.hpp deleted file mode 100644 index f3c6b82b87f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/discard_block.hpp +++ /dev/null @@ -1,241 +0,0 @@ -/* boost random/discard_block.hpp header file - * - * Copyright Jens Maurer 2002 - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-03-02 created - */ - -#ifndef BOOST_RANDOM_DISCARD_BLOCK_HPP -#define BOOST_RANDOM_DISCARD_BLOCK_HPP - -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace boost { -namespace random { - -/** - * The class template \discard_block_engine is a model of - * \pseudo_random_number_generator. It modifies - * another generator by discarding parts of its output. - * Out of every block of @c p results, the first @c r - * will be returned and the rest discarded. - * - * Requires: 0 < p <= r - */ -template -class discard_block_engine -{ - typedef typename detail::seed_type< - typename UniformRandomNumberGenerator::result_type>::type seed_type; -public: - typedef UniformRandomNumberGenerator base_type; - typedef typename base_type::result_type result_type; - - BOOST_STATIC_CONSTANT(std::size_t, block_size = p); - BOOST_STATIC_CONSTANT(std::size_t, used_block = r); - - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - BOOST_STATIC_CONSTANT(std::size_t, total_block = p); - BOOST_STATIC_CONSTANT(std::size_t, returned_block = r); - - BOOST_STATIC_ASSERT(total_block >= returned_block); - - /** Uses the default seed for the base generator. */ - discard_block_engine() : _rng(), _n(0) { } - /** Constructs a new \discard_block_engine with a copy of rng. */ - explicit discard_block_engine(const base_type & rng) : _rng(rng), _n(0) { } - -#ifndef BOOST_NO_CXX11_RVALUE_REFERENCES - /** Constructs a new \discard_block_engine with rng. */ - explicit discard_block_engine(base_type && rng) : _rng(rng), _n(0) { } -#endif - - /** - * Creates a new \discard_block_engine and seeds the underlying - * generator with @c value - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(discard_block_engine, - seed_type, value) - { _rng.seed(value); _n = 0; } - - /** - * Creates a new \discard_block_engine and seeds the underlying - * generator with @c seq - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(discard_block_engine, SeedSeq, seq) - { _rng.seed(seq); _n = 0; } - - /** - * Creates a new \discard_block_engine and seeds the underlying - * generator with first and last. - */ - template discard_block_engine(It& first, It last) - : _rng(first, last), _n(0) { } - - /** default seeds the underlying generator. */ - void seed() { _rng.seed(); _n = 0; } - /** Seeds the underlying generator with s. */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(discard_block_engine, seed_type, s) - { _rng.seed(s); _n = 0; } - /** Seeds the underlying generator with seq. */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(discard_block_engine, SeedSeq, seq) - { _rng.seed(seq); _n = 0; } - /** Seeds the underlying generator with first and last. */ - template void seed(It& first, It last) - { _rng.seed(first, last); _n = 0; } - - /** Returns the underlying engine. */ - const base_type& base() const { return _rng; } - - /** Returns the next value of the generator. */ - result_type operator()() - { - if(_n >= returned_block) { - // discard values of random number generator - // Don't use discard, since we still need to - // be somewhat compatible with TR1. - // _rng.discard(total_block - _n); - for(std::size_t i = 0; i < total_block - _n; ++i) { - _rng(); - } - _n = 0; - } - ++_n; - return _rng(); - } - - void discard(boost::uintmax_t z) - { - for(boost::uintmax_t j = 0; j < z; ++j) { - (*this)(); - } - } - - template - void generate(It first, It last) - { detail::generate(*this, first, last); } - - /** - * Returns the smallest value that the generator can produce. - * This is the same as the minimum of the underlying generator. - */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (base_type::min)(); } - /** - * Returns the largest value that the generator can produce. - * This is the same as the maximum of the underlying generator. - */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (base_type::max)(); } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes a \discard_block_engine to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const discard_block_engine& s) - { - os << s._rng << ' ' << s._n; - return os; - } - - /** Reads a \discard_block_engine from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, discard_block_engine& s) - { - is >> s._rng >> std::ws >> s._n; - return is; - } -#endif - - /** Returns true if the two generators will produce identical sequences. */ - friend bool operator==(const discard_block_engine& x, - const discard_block_engine& y) - { return x._rng == y._rng && x._n == y._n; } - /** Returns true if the two generators will produce different sequences. */ - friend bool operator!=(const discard_block_engine& x, - const discard_block_engine& y) - { return !(x == y); } - -private: - base_type _rng; - std::size_t _n; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool discard_block_engine::has_fixed_range; -template -const std::size_t discard_block_engine::total_block; -template -const std::size_t discard_block_engine::returned_block; -template -const std::size_t discard_block_engine::block_size; -template -const std::size_t discard_block_engine::used_block; -#endif - -/// \cond \show_deprecated - -template -class discard_block : public discard_block_engine -{ - typedef discard_block_engine base_t; -public: - typedef typename base_t::result_type result_type; - discard_block() {} - template - discard_block(T& arg) : base_t(arg) {} - template - discard_block(const T& arg) : base_t(arg) {} - template - discard_block(It& first, It last) : base_t(first, last) {} - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (this->base().min)(); } - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (this->base().max)(); } -}; - -/// \endcond - -namespace detail { - - template - struct generator_bits; - - template - struct generator_bits > { - static std::size_t value() { return generator_bits::value(); } - }; - - template - struct generator_bits > { - static std::size_t value() { return generator_bits::value(); } - }; - -} - -} // namespace random - -} // namespace boost - -#endif // BOOST_RANDOM_DISCARD_BLOCK_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/discrete_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/discrete_distribution.hpp deleted file mode 100644 index 3d14d4a02c7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/discrete_distribution.hpp +++ /dev/null @@ -1,636 +0,0 @@ -/* boost random/discrete_distribution.hpp header file - * - * Copyright Steven Watanabe 2009-2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_DISCRETE_DISTRIBUTION_HPP_INCLUDED -#define BOOST_RANDOM_DISCRETE_DISTRIBUTION_HPP_INCLUDED - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST -#include -#endif - -#include -#include - -#include - -namespace boost { -namespace random { -namespace detail { - -template -struct integer_alias_table { - WeightType get_weight(IntType bin) const { - WeightType result = _average; - if(bin < _excess) ++result; - return result; - } - template - WeightType init_average(Iter begin, Iter end) { - WeightType weight_average = 0; - IntType excess = 0; - IntType n = 0; - // weight_average * n + excess == current partial sum - // This is a bit messy, but it's guaranteed not to overflow - for(Iter iter = begin; iter != end; ++iter) { - ++n; - if(*iter < weight_average) { - WeightType diff = weight_average - *iter; - weight_average -= diff / n; - if(diff % n > excess) { - --weight_average; - excess += n - diff % n; - } else { - excess -= diff % n; - } - } else { - WeightType diff = *iter - weight_average; - weight_average += diff / n; - if(diff % n < n - excess) { - excess += diff % n; - } else { - ++weight_average; - excess -= n - diff % n; - } - } - } - _alias_table.resize(static_cast(n)); - _average = weight_average; - _excess = excess; - return weight_average; - } - void init_empty() - { - _alias_table.clear(); - _alias_table.push_back(std::make_pair(static_cast(1), - static_cast(0))); - _average = static_cast(1); - _excess = static_cast(0); - } - bool operator==(const integer_alias_table& other) const - { - return _alias_table == other._alias_table && - _average == other._average && _excess == other._excess; - } - static WeightType normalize(WeightType val, WeightType average) - { - return val; - } - static void normalize(std::vector&) {} - template - WeightType test(URNG &urng) const - { - return uniform_int_distribution(0, _average)(urng); - } - bool accept(IntType result, WeightType val) const - { - return result < _excess || val < _average; - } - static WeightType try_get_sum(const std::vector& weights) - { - WeightType result = static_cast(0); - for(typename std::vector::const_iterator - iter = weights.begin(), end = weights.end(); - iter != end; ++iter) - { - if((std::numeric_limits::max)() - result > *iter) { - return static_cast(0); - } - result += *iter; - } - return result; - } - template - static WeightType generate_in_range(URNG &urng, WeightType max) - { - return uniform_int_distribution( - static_cast(0), max-1)(urng); - } - typedef std::vector > alias_table_t; - alias_table_t _alias_table; - WeightType _average; - IntType _excess; -}; - -template -struct real_alias_table { - WeightType get_weight(IntType) const - { - return WeightType(1.0); - } - template - WeightType init_average(Iter first, Iter last) - { - std::size_t size = std::distance(first, last); - WeightType weight_sum = - std::accumulate(first, last, static_cast(0)); - _alias_table.resize(size); - return weight_sum / size; - } - void init_empty() - { - _alias_table.clear(); - _alias_table.push_back(std::make_pair(static_cast(1), - static_cast(0))); - } - bool operator==(const real_alias_table& other) const - { - return _alias_table == other._alias_table; - } - static WeightType normalize(WeightType val, WeightType average) - { - return val / average; - } - static void normalize(std::vector& weights) - { - WeightType sum = - std::accumulate(weights.begin(), weights.end(), - static_cast(0)); - for(typename std::vector::iterator - iter = weights.begin(), - end = weights.end(); - iter != end; ++iter) - { - *iter /= sum; - } - } - template - WeightType test(URNG &urng) const - { - return uniform_01()(urng); - } - bool accept(IntType, WeightType) const - { - return true; - } - static WeightType try_get_sum(const std::vector& weights) - { - return static_cast(1); - } - template - static WeightType generate_in_range(URNG &urng, WeightType) - { - return uniform_01()(urng); - } - typedef std::vector > alias_table_t; - alias_table_t _alias_table; -}; - -template -struct select_alias_table; - -template<> -struct select_alias_table { - template - struct apply { - typedef integer_alias_table type; - }; -}; - -template<> -struct select_alias_table { - template - struct apply { - typedef real_alias_table type; - }; -}; - -} - -/** - * The class @c discrete_distribution models a \random_distribution. - * It produces integers in the range [0, n) with the probability - * of producing each value is specified by the parameters of the - * distribution. - */ -template -class discrete_distribution { -public: - typedef WeightType input_type; - typedef IntType result_type; - - class param_type { - public: - - typedef discrete_distribution distribution_type; - - /** - * Constructs a @c param_type object, representing a distribution - * with \f$p(0) = 1\f$ and \f$p(k|k>0) = 0\f$. - */ - param_type() : _probabilities(1, static_cast(1)) {} - /** - * If @c first == @c last, equivalent to the default constructor. - * Otherwise, the values of the range represent weights for the - * possible values of the distribution. - */ - template - param_type(Iter first, Iter last) : _probabilities(first, last) - { - normalize(); - } -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * If wl.size() == 0, equivalent to the default constructor. - * Otherwise, the values of the @c initializer_list represent - * weights for the possible values of the distribution. - */ - param_type(const std::initializer_list& wl) - : _probabilities(wl) - { - normalize(); - } -#endif - /** - * If the range is empty, equivalent to the default constructor. - * Otherwise, the elements of the range represent - * weights for the possible values of the distribution. - */ - template - explicit param_type(const Range& range) - : _probabilities(boost::begin(range), boost::end(range)) - { - normalize(); - } - - /** - * If nw is zero, equivalent to the default constructor. - * Otherwise, the range of the distribution is [0, nw), - * and the weights are found by calling fw with values - * evenly distributed between \f$\mbox{xmin} + \delta/2\f$ and - * \f$\mbox{xmax} - \delta/2\f$, where - * \f$\delta = (\mbox{xmax} - \mbox{xmin})/\mbox{nw}\f$. - */ - template - param_type(std::size_t nw, double xmin, double xmax, Func fw) - { - std::size_t n = (nw == 0) ? 1 : nw; - double delta = (xmax - xmin) / n; - BOOST_ASSERT(delta > 0); - for(std::size_t k = 0; k < n; ++k) { - _probabilities.push_back(fw(xmin + k*delta + delta/2)); - } - normalize(); - } - - /** - * Returns a vector containing the probabilities of each possible - * value of the distribution. - */ - std::vector probabilities() const - { - return _probabilities; - } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - detail::print_vector(os, parm._probabilities); - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - std::vector temp; - detail::read_vector(is, temp); - if(is) { - parm._probabilities.swap(temp); - } - return is; - } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { - return lhs._probabilities == rhs._probabilities; - } - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - private: - /// @cond show_private - friend class discrete_distribution; - explicit param_type(const discrete_distribution& dist) - : _probabilities(dist.probabilities()) - {} - void normalize() - { - impl_type::normalize(_probabilities); - } - std::vector _probabilities; - /// @endcond - }; - - /** - * Creates a new @c discrete_distribution object that has - * \f$p(0) = 1\f$ and \f$p(i|i>0) = 0\f$. - */ - discrete_distribution() - { - _impl.init_empty(); - } - /** - * Constructs a discrete_distribution from an iterator range. - * If @c first == @c last, equivalent to the default constructor. - * Otherwise, the values of the range represent weights for the - * possible values of the distribution. - */ - template - discrete_distribution(Iter first, Iter last) - { - init(first, last); - } -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * Constructs a @c discrete_distribution from a @c std::initializer_list. - * If the @c initializer_list is empty, equivalent to the default - * constructor. Otherwise, the values of the @c initializer_list - * represent weights for the possible values of the distribution. - * For example, given the distribution - * - * @code - * discrete_distribution<> dist{1, 4, 5}; - * @endcode - * - * The probability of a 0 is 1/10, the probability of a 1 is 2/5, - * the probability of a 2 is 1/2, and no other values are possible. - */ - discrete_distribution(std::initializer_list wl) - { - init(wl.begin(), wl.end()); - } -#endif - /** - * Constructs a discrete_distribution from a Boost.Range range. - * If the range is empty, equivalent to the default constructor. - * Otherwise, the values of the range represent weights for the - * possible values of the distribution. - */ - template - explicit discrete_distribution(const Range& range) - { - init(boost::begin(range), boost::end(range)); - } - /** - * Constructs a discrete_distribution that approximates a function. - * If nw is zero, equivalent to the default constructor. - * Otherwise, the range of the distribution is [0, nw), - * and the weights are found by calling fw with values - * evenly distributed between \f$\mbox{xmin} + \delta/2\f$ and - * \f$\mbox{xmax} - \delta/2\f$, where - * \f$\delta = (\mbox{xmax} - \mbox{xmin})/\mbox{nw}\f$. - */ - template - discrete_distribution(std::size_t nw, double xmin, double xmax, Func fw) - { - std::size_t n = (nw == 0) ? 1 : nw; - double delta = (xmax - xmin) / n; - BOOST_ASSERT(delta > 0); - std::vector weights; - for(std::size_t k = 0; k < n; ++k) { - weights.push_back(fw(xmin + k*delta + delta/2)); - } - init(weights.begin(), weights.end()); - } - /** - * Constructs a discrete_distribution from its parameters. - */ - explicit discrete_distribution(const param_type& parm) - { - param(parm); - } - - /** - * Returns a value distributed according to the parameters of the - * discrete_distribution. - */ - template - IntType operator()(URNG& urng) const - { - BOOST_ASSERT(!_impl._alias_table.empty()); - IntType result; - WeightType test; - do { - result = uniform_int_distribution((min)(), (max)())(urng); - test = _impl.test(urng); - } while(!_impl.accept(result, test)); - if(test < _impl._alias_table[static_cast(result)].first) { - return result; - } else { - return(_impl._alias_table[static_cast(result)].second); - } - } - - /** - * Returns a value distributed according to the parameters - * specified by param. - */ - template - IntType operator()(URNG& urng, const param_type& parm) const - { - if(WeightType limit = impl_type::try_get_sum(parm._probabilities)) { - WeightType val = impl_type::generate_in_range(urng, limit); - WeightType sum = 0; - std::size_t result = 0; - for(typename std::vector::const_iterator - iter = parm._probabilities.begin(), - end = parm._probabilities.end(); - iter != end; ++iter, ++result) - { - sum += *iter; - if(sum > val) { - return result; - } - } - // This shouldn't be reachable, but round-off error - // can prevent any match from being found when val is - // very close to 1. - return static_cast(parm._probabilities.size() - 1); - } else { - // WeightType is integral and sum(parm._probabilities) - // would overflow. Just use the easy solution. - return discrete_distribution(parm)(urng); - } - } - - /** Returns the smallest value that the distribution can produce. */ - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return 0; } - /** Returns the largest value that the distribution can produce. */ - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return static_cast(_impl._alias_table.size() - 1); } - - /** - * Returns a vector containing the probabilities of each - * value of the distribution. For example, given - * - * @code - * discrete_distribution<> dist = { 1, 4, 5 }; - * std::vector p = dist.param(); - * @endcode - * - * the vector, p will contain {0.1, 0.4, 0.5}. - * - * If @c WeightType is integral, then the weights - * will be returned unchanged. - */ - std::vector probabilities() const - { - std::vector result(_impl._alias_table.size(), static_cast(0)); - std::size_t i = 0; - for(typename impl_type::alias_table_t::const_iterator - iter = _impl._alias_table.begin(), - end = _impl._alias_table.end(); - iter != end; ++iter, ++i) - { - WeightType val = iter->first; - result[i] += val; - result[static_cast(iter->second)] += _impl.get_weight(i) - val; - } - impl_type::normalize(result); - return(result); - } - - /** Returns the parameters of the distribution. */ - param_type param() const - { - return param_type(*this); - } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - init(parm._probabilities.begin(), parm._probabilities.end()); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() {} - - /** Writes a distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, discrete_distribution, dd) - { - os << dd.param(); - return os; - } - - /** Reads a distribution from a @c std::istream */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, discrete_distribution, dd) - { - param_type parm; - if(is >> parm) { - dd.param(parm); - } - return is; - } - - /** - * Returns true if the two distributions will return the - * same sequence of values, when passed equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(discrete_distribution, lhs, rhs) - { - return lhs._impl == rhs._impl; - } - /** - * Returns true if the two distributions may return different - * sequences of values, when passed equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(discrete_distribution) - -private: - - /// @cond show_private - - template - void init(Iter first, Iter last, std::input_iterator_tag) - { - std::vector temp(first, last); - init(temp.begin(), temp.end()); - } - template - void init(Iter first, Iter last, std::forward_iterator_tag) - { - std::vector > below_average; - std::vector > above_average; - WeightType weight_average = _impl.init_average(first, last); - WeightType normalized_average = _impl.get_weight(0); - std::size_t i = 0; - for(; first != last; ++first, ++i) { - WeightType val = impl_type::normalize(*first, weight_average); - std::pair elem(val, static_cast(i)); - if(val < normalized_average) { - below_average.push_back(elem); - } else { - above_average.push_back(elem); - } - } - - typename impl_type::alias_table_t::iterator - b_iter = below_average.begin(), - b_end = below_average.end(), - a_iter = above_average.begin(), - a_end = above_average.end() - ; - while(b_iter != b_end && a_iter != a_end) { - _impl._alias_table[static_cast(b_iter->second)] = - std::make_pair(b_iter->first, a_iter->second); - a_iter->first -= (_impl.get_weight(b_iter->second) - b_iter->first); - if(a_iter->first < normalized_average) { - *b_iter = *a_iter++; - } else { - ++b_iter; - } - } - for(; b_iter != b_end; ++b_iter) { - _impl._alias_table[static_cast(b_iter->second)].first = - _impl.get_weight(b_iter->second); - } - for(; a_iter != a_end; ++a_iter) { - _impl._alias_table[static_cast(a_iter->second)].first = - _impl.get_weight(a_iter->second); - } - } - template - void init(Iter first, Iter last) - { - if(first == last) { - _impl.init_empty(); - } else { - typename std::iterator_traits::iterator_category category; - init(first, last, category); - } - } - typedef typename detail::select_alias_table< - (::boost::is_integral::value) - >::template apply::type impl_type; - impl_type _impl; - /// @endcond -}; - -} -} - -#include - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/exponential_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/exponential_distribution.hpp deleted file mode 100644 index b4b95c5472b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/exponential_distribution.hpp +++ /dev/null @@ -1,386 +0,0 @@ -/* boost random/exponential_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Copyright Jason Rhinelander 2016 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_EXPONENTIAL_DISTRIBUTION_HPP -#define BOOST_RANDOM_EXPONENTIAL_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -namespace detail { - -// tables for the ziggurat algorithm -template -struct exponential_table { - static const RealType table_x[257]; - static const RealType table_y[257]; -}; - -template -const RealType exponential_table::table_x[257] = { - 8.6971174701310497140, 7.6971174701310497140, 6.9410336293772123602, 6.4783784938325698538, - 6.1441646657724730491, 5.8821443157953997963, 5.6664101674540337371, 5.4828906275260628694, - 5.3230905057543986131, 5.1814872813015010392, 5.0542884899813047117, 4.9387770859012514838, - 4.8329397410251125881, 4.7352429966017412526, 4.6444918854200854873, 4.5597370617073515513, - 4.4802117465284221949, 4.4052876934735729805, 4.3344436803172730116, 4.2672424802773661873, - 4.2033137137351843802, 4.1423408656640511251, 4.0840513104082974638, 4.0282085446479365106, - 3.9746060666737884793, 3.9230625001354895926, 3.8734176703995089983, 3.8255294185223367372, - 3.7792709924116678992, 3.7345288940397975350, 3.6912010902374189454, 3.6491955157608538478, - 3.6084288131289096339, 3.5688252656483374051, 3.5303158891293438633, 3.4928376547740601814, - 3.4563328211327607625, 3.4207483572511205323, 3.3860354424603017887, 3.3521490309001100106, - 3.3190474709707487166, 3.2866921715990692095, 3.2550473085704501813, 3.2240795652862645207, - 3.1937579032122407483, 3.1640533580259734580, 3.1349388580844407393, 3.1063890623398246660, - 3.0783802152540905188, 3.0508900166154554479, 3.0238975044556767713, 2.9973829495161306949, - 2.9713277599210896472, 2.9457143948950456386, 2.9205262865127406647, 2.8957477686001416838, - 2.8713640120155362592, 2.8473609656351888266, 2.8237253024500354905, 2.8004443702507381944, - 2.7775061464397572041, 2.7548991965623453650, 2.7326126361947007411, 2.7106360958679293686, - 2.6889596887418041593, 2.6675739807732670816, 2.6464699631518093905, 2.6256390267977886123, - 2.6050729387408355373, 2.5847638202141406911, 2.5647041263169053687, 2.5448866271118700928, - 2.5253043900378279427, 2.5059507635285939648, 2.4868193617402096807, 2.4679040502973649846, - 2.4491989329782498908, 2.4306983392644199088, 2.4123968126888708336, 2.3942890999214583288, - 2.3763701405361408194, 2.3586350574093374601, 2.3410791477030346875, 2.3236978743901964559, - 2.3064868582835798692, 2.2894418705322694265, 2.2725588255531546952, 2.2558337743672190441, - 2.2392628983129087111, 2.2228425031110364013, 2.2065690132576635755, 2.1904389667232199235, - 2.1744490099377744673, 2.1585958930438856781, 2.1428764653998416425, 2.1272876713173679737, - 2.1118265460190418108, 2.0964902118017147637, 2.0812758743932248696, 2.0661808194905755036, - 2.0512024094685848641, 2.0363380802487695916, 2.0215853383189260770, 2.0069417578945183144, - 1.9924049782135764992, 1.9779727009573602295, 1.9636426877895480401, 1.9494127580071845659, - 1.9352807862970511135, 1.9212447005915276767, 1.9073024800183871196, 1.8934521529393077332, - 1.8796917950722108462, 1.8660195276928275962, 1.8524335159111751661, 1.8389319670188793980, - 1.8255131289035192212, 1.8121752885263901413, 1.7989167704602903934, 1.7857359354841254047, - 1.7726311792313049959, 1.7596009308890742369, 1.7466436519460739352, 1.7337578349855711926, - 1.7209420025219350428, 1.7081947058780575683, 1.6955145241015377061, 1.6829000629175537544, - 1.6703499537164519163, 1.6578628525741725325, 1.6454374393037234057, 1.6330724165359912048, - 1.6207665088282577216, 1.6085184617988580769, 1.5963270412864831349, 1.5841910325326886695, - 1.5721092393862294810, 1.5600804835278879161, 1.5481036037145133070, 1.5361774550410318943, - 1.5243009082192260050, 1.5124728488721167573, 1.5006921768428164936, 1.4889578055167456003, - 1.4772686611561334579, 1.4656236822457450411, 1.4540218188487932264, 1.4424620319720121876, - 1.4309432929388794104, 1.4194645827699828254, 1.4080248915695353509, 1.3966232179170417110, - 1.3852585682631217189, 1.3739299563284902176, 1.3626364025050864742, 1.3513769332583349176, - 1.3401505805295045843, 1.3289563811371163220, 1.3177933761763245480, 1.3066606104151739482, - 1.2955571316866007210, 1.2844819902750125450, 1.2734342382962410994, 1.2624129290696153434, - 1.2514171164808525098, 1.2404458543344064544, 1.2294981956938491599, 1.2185731922087903071, - 1.2076698934267612830, 1.1967873460884031665, 1.1859245934042023557, 1.1750806743109117687, - 1.1642546227056790397, 1.1534454666557748056, 1.1426522275816728928, 1.1318739194110786733, - 1.1211095477013306083, 1.1103581087274114281, 1.0996185885325976575, 1.0888899619385472598, - 1.0781711915113727024, 1.0674612264799681530, 1.0567590016025518414, 1.0460634359770445503, - 1.0353734317905289496, 1.0246878730026178052, 1.0140056239570971074, 1.0033255279156973717, - 0.99264640550727647009, 0.98196705308506317914, 0.97128624098390397896, 0.96060271166866709917, - 0.94991517776407659940, 0.93922231995526297952, 0.92852278474721113999, 0.91781518207004493915, - 0.90709808271569100600, 0.89637001558989069006, 0.88562946476175228052, 0.87487486629102585352, - 0.86410460481100519511, 0.85331700984237406386, 0.84251035181036928333, 0.83168283773427388393, - 0.82083260655441252290, 0.80995772405741906620, 0.79905617735548788109, 0.78812586886949324977, - 0.77716460975913043936, 0.76617011273543541328, 0.75513998418198289808, 0.74407171550050873971, - 0.73296267358436604916, 0.72181009030875689912, 0.71061105090965570413, 0.69936248110323266174, - 0.68806113277374858613, 0.67670356802952337911, 0.66528614139267855405, 0.65380497984766565353, - 0.64225596042453703448, 0.63063468493349100113, 0.61893645139487678178, 0.60715622162030085137, - 0.59528858429150359384, 0.58332771274877027785, 0.57126731653258903915, 0.55910058551154127652, - 0.54682012516331112550, 0.53441788123716615385, 0.52188505159213564105, 0.50921198244365495319, - 0.49638804551867159754, 0.48340149165346224782, 0.47023927508216945338, 0.45688684093142071279, - 0.44332786607355296305, 0.42954394022541129589, 0.41551416960035700100, 0.40121467889627836229, - 0.38661797794112021568, 0.37169214532991786118, 0.35639976025839443721, 0.34069648106484979674, - 0.32452911701691008547, 0.30783295467493287307, 0.29052795549123115167, 0.27251318547846547924, - 0.25365836338591284433, 0.23379048305967553619, 0.21267151063096745264, 0.18995868962243277774, - 0.16512762256418831796, 0.13730498094001380420, 0.10483850756582017915, 0.063852163815003480173, - 0 -}; - -template -const RealType exponential_table::table_y[257] = { - 0, 0.00045413435384149675545, 0.00096726928232717452884, 0.0015362997803015723824, - 0.0021459677437189061793, 0.0027887987935740759640, 0.0034602647778369039855, 0.0041572951208337952532, - 0.0048776559835423925804, 0.0056196422072054831710, 0.0063819059373191794422, 0.0071633531836349841425, - 0.0079630774380170392396, 0.0087803149858089752347, 0.0096144136425022094101, 0.010464810181029979488, - 0.011331013597834597488, 0.012212592426255380661, 0.013109164931254991070, 0.014020391403181937334, - 0.014945968011691148079, 0.015885621839973162490, 0.016839106826039946359, 0.017806200410911360563, - 0.018786700744696029497, 0.019780424338009741737, 0.020787204072578117603, 0.021806887504283582125, - 0.022839335406385238829, 0.023884420511558170348, 0.024942026419731782971, 0.026012046645134218076, - 0.027094383780955798424, 0.028188948763978634421, 0.029295660224637394015, 0.030414443910466605492, - 0.031545232172893605499, 0.032687963508959533317, 0.033842582150874329031, 0.035009037697397411067, - 0.036187284781931419754, 0.037377282772959360128, 0.038578995503074859626, 0.039792391023374122670, - 0.041017441380414820816, 0.042254122413316231413, 0.043502413568888183301, 0.044762297732943280694, - 0.046033761076175166762, 0.047316792913181548703, 0.048611385573379494401, 0.049917534282706374944, - 0.051235237055126279830, 0.052564494593071689595, 0.053905310196046085104, 0.055257689676697038322, - 0.056621641283742874438, 0.057997175631200659098, 0.059384305633420264487, 0.060783046445479636051, - 0.062193415408540996150, 0.063615431999807331076, 0.065049117786753755036, 0.066494496385339779043, - 0.067951593421936607770, 0.069420436498728751675, 0.070901055162371828426, 0.072393480875708743023, - 0.073897746992364746308, 0.075413888734058408453, 0.076941943170480510100, 0.078481949201606426042, - 0.080033947542319910023, 0.081597980709237420930, 0.083174093009632380354, 0.084762330532368125386, - 0.086362741140756912277, 0.087975374467270219300, 0.089600281910032864534, 0.091237516631040162057, - 0.092887133556043546523, 0.094549189376055853718, 0.096223742550432800103, 0.097910853311492199618, - 0.099610583670637128826, 0.10132299742595363588, 0.10304816017125771553, 0.10478613930657016928, - 0.10653700405000166218, 0.10830082545103379867, 0.11007767640518539026, 0.11186763167005629731, - 0.11367076788274431301, 0.11548716357863353664, 0.11731689921155557057, 0.11916005717532768467, - 0.12101672182667483729, 0.12288697950954513498, 0.12477091858083096578, 0.12666862943751066518, - 0.12858020454522817870, 0.13050573846833078225, 0.13244532790138752023, 0.13439907170221363078, - 0.13636707092642885841, 0.13834942886358021406, 0.14034625107486244210, 0.14235764543247220043, - 0.14438372216063476473, 0.14642459387834493787, 0.14848037564386679222, 0.15055118500103990354, - 0.15263714202744286154, 0.15473836938446807312, 0.15685499236936522013, 0.15898713896931420572, - 0.16113493991759203183, 0.16329852875190180795, 0.16547804187493600915, 0.16767361861725019322, - 0.16988540130252766513, 0.17211353531532005700, 0.17435816917135348788, 0.17661945459049489581, - 0.17889754657247831241, 0.18119260347549629488, 0.18350478709776746150, 0.18583426276219711495, - 0.18818119940425430485, 0.19054576966319540013, 0.19292814997677133873, 0.19532852067956322315, - 0.19774706610509886464, 0.20018397469191127727, 0.20263943909370901930, 0.20511365629383770880, - 0.20760682772422204205, 0.21011915938898825914, 0.21265086199297827522, 0.21520215107537867786, - 0.21777324714870053264, 0.22036437584335949720, 0.22297576805812018050, 0.22560766011668406495, - 0.22826029393071670664, 0.23093391716962742173, 0.23362878343743333945, 0.23634515245705964715, - 0.23908329026244917002, 0.24184346939887722761, 0.24462596913189210901, 0.24743107566532763894, - 0.25025908236886230967, 0.25311029001562948171, 0.25598500703041538015, 0.25888354974901621678, - 0.26180624268936295243, 0.26475341883506220209, 0.26772541993204481808, 0.27072259679906003167, - 0.27374530965280298302, 0.27679392844851734458, 0.27986883323697289920, 0.28297041453878076010, - 0.28609907373707684673, 0.28925522348967773308, 0.29243928816189258772, 0.29565170428126120948, - 0.29889292101558177099, 0.30216340067569352897, 0.30546361924459023541, 0.30879406693456016794, - 0.31215524877417956945, 0.31554768522712893632, 0.31897191284495723773, 0.32242848495608914289, - 0.32591797239355619822, 0.32944096426413633091, 0.33299806876180896713, 0.33658991402867758144, - 0.34021714906678004560, 0.34388044470450243010, 0.34758049462163698567, 0.35131801643748334681, - 0.35509375286678745925, 0.35890847294874976196, 0.36276297335481777335, 0.36665807978151414890, - 0.37059464843514599421, 0.37457356761590215193, 0.37859575940958081092, 0.38266218149600982112, - 0.38677382908413768115, 0.39093173698479710717, 0.39513698183329015336, 0.39939068447523107877, - 0.40369401253053026739, 0.40804818315203238238, 0.41245446599716116772, 0.41691418643300289465, - 0.42142872899761659635, 0.42599954114303435739, 0.43062813728845883923, 0.43531610321563659758, - 0.44006510084235387501, 0.44487687341454851593, 0.44975325116275498919, 0.45469615747461548049, - 0.45970761564213768669, 0.46478975625042618067, 0.46994482528395999841, 0.47517519303737738299, - 0.48048336393045423016, 0.48587198734188493564, 0.49134386959403255500, 0.49690198724154955294, - 0.50254950184134769289, 0.50828977641064283495, 0.51412639381474855788, 0.52006317736823356823, - 0.52610421398361972602, 0.53225388026304326945, 0.53851687200286186590, 0.54489823767243963663, - 0.55140341654064131685, 0.55803828226258748140, 0.56480919291240022434, 0.57172304866482579008, - 0.57878735860284503057, 0.58601031847726802755, 0.59340090169173341521, 0.60096896636523224742, - 0.60872538207962206507, 0.61668218091520762326, 0.62485273870366592605, 0.63325199421436607968, - 0.64189671642726607018, 0.65080583341457104881, 0.66000084107899974178, 0.66950631673192477684, - 0.67935057226476538741, 0.68956649611707798890, 0.70019265508278816709, 0.71127476080507597882, - 0.72286765959357200702, 0.73503809243142351530, 0.74786862198519510742, 0.76146338884989624862, - 0.77595685204011559675, 0.79152763697249565519, 0.80842165152300838005, 0.82699329664305033399, - 0.84778550062398962096, 0.87170433238120363669, 0.90046992992574643800, 0.93814368086217467916, - 1 -}; - -template -struct unit_exponential_distribution -{ - template - RealType operator()(Engine& eng) { - const double * const table_x = exponential_table::table_x; - const double * const table_y = exponential_table::table_y; - RealType shift(0); - for(;;) { - std::pair vals = generate_int_float_pair(eng); - int i = vals.second; - RealType x = vals.first * RealType(table_x[i]); - if(x < RealType(table_x[i + 1])) return shift + x; - // For i=0 we need to generate from the tail, but because this is an exponential - // distribution, the tail looks exactly like the body, so we can simply repeat with a - // shift: - if (i == 0) shift += RealType(table_x[1]); - else { - RealType y01 = uniform_01()(eng); - RealType y = RealType(table_y[i]) + y01 * RealType(table_y[i+1] - table_y[i]); - - // All we care about is whether these are < or > 0; these values are equal to - // (lbound) or proportional to (ubound) `y` minus the lower/upper bound. - RealType y_above_ubound = RealType(table_x[i] - table_x[i+1]) * y01 - (RealType(table_x[i]) - x), - y_above_lbound = y - (RealType(table_y[i+1]) + (RealType(table_x[i+1]) - x) * RealType(table_y[i+1])); - - if (y_above_ubound < 0 // if above the upper bound reject immediately - && - ( - y_above_lbound < 0 // If below the lower bound accept immediately - || - y < f(x) // Otherwise it's between the bounds and we need a full check - ) - ) { - return x + shift; - } - } - } - } - static RealType f(RealType x) { - using std::exp; - return exp(-x); - } -}; - -} // namespace detail - - -/** - * The exponential distribution is a model of \random_distribution with - * a single parameter lambda. - * - * It has \f$\displaystyle p(x) = \lambda e^{-\lambda x}\f$ - * - * The implementation uses the "ziggurat" algorithm, as described in - * - * @blockquote - * "The Ziggurat Method for Generating Random Variables", - * George Marsaglia and Wai Wan Tsang, Journal of Statistical Software - * Volume 5, Number 8 (2000), 1-7. - * @endblockquote - */ -template -class exponential_distribution -{ -public: - typedef RealType input_type; - typedef RealType result_type; - - class param_type - { - public: - - typedef exponential_distribution distribution_type; - - /** - * Constructs parameters with a given lambda. - * - * Requires: lambda > 0 - */ - param_type(RealType lambda_arg = RealType(1.0)) - : _lambda(lambda_arg) { BOOST_ASSERT(_lambda > RealType(0)); } - - /** Returns the lambda parameter of the distribution. */ - RealType lambda() const { return _lambda; } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._lambda; - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - is >> parm._lambda; - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._lambda == rhs._lambda; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _lambda; - }; - - /** - * Constructs an exponential_distribution with a given lambda. - * - * Requires: lambda > 0 - */ - explicit exponential_distribution(RealType lambda_arg = RealType(1.0)) - : _lambda(lambda_arg) { BOOST_ASSERT(_lambda > RealType(0)); } - - /** - * Constructs an exponential_distribution from its parameters - */ - explicit exponential_distribution(const param_type& parm) - : _lambda(parm.lambda()) {} - - // compiler-generated copy ctor and assignment operator are fine - - /** Returns the lambda parameter of the distribution. */ - RealType lambda() const { return _lambda; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return RealType(0); } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return (std::numeric_limits::infinity)(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_lambda); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) { _lambda = parm.lambda(); } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** - * Returns a random variate distributed according to the - * exponential distribution. - */ - template - result_type operator()(Engine& eng) const - { - detail::unit_exponential_distribution impl; - return impl(eng) / _lambda; - } - - /** - * Returns a random variate distributed according to the exponential - * distribution with parameters specified by param. - */ - template - result_type operator()(Engine& eng, const param_type& parm) const - { - return exponential_distribution(parm)(eng); - } - - /** Writes the distribution to a std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, exponential_distribution, ed) - { - os << ed._lambda; - return os; - } - - /** Reads the distribution from a std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, exponential_distribution, ed) - { - is >> ed._lambda; - return is; - } - - /** - * Returns true iff the two distributions will produce identical - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(exponential_distribution, lhs, rhs) - { return lhs._lambda == rhs._lambda; } - - /** - * Returns true iff the two distributions will produce different - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(exponential_distribution) - -private: - result_type _lambda; -}; - -} // namespace random - -using random::exponential_distribution; - -} // namespace boost - -#endif // BOOST_RANDOM_EXPONENTIAL_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/extreme_value_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/extreme_value_distribution.hpp deleted file mode 100644 index 419c3da651d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/extreme_value_distribution.hpp +++ /dev/null @@ -1,177 +0,0 @@ -/* boost random/extreme_value_distribution.hpp header file - * - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_EXTREME_VALUE_DISTRIBUTION_HPP -#define BOOST_RANDOM_EXTREME_VALUE_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The extreme value distribution is a real valued distribution with two - * parameters a and b. - * - * It has \f$\displaystyle p(x) = \frac{1}{b}e^{\frac{a-x}{b} - e^\frac{a-x}{b}}\f$. - */ -template -class extreme_value_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef extreme_value_distribution distribution_type; - - /** - * Constructs a @c param_type from the "a" and "b" parameters - * of the distribution. - * - * Requires: b > 0 - */ - explicit param_type(RealType a_arg = 1.0, RealType b_arg = 1.0) - : _a(a_arg), _b(b_arg) - {} - - /** Returns the "a" parameter of the distribtuion. */ - RealType a() const { return _a; } - /** Returns the "b" parameter of the distribution. */ - RealType b() const { return _b; } - - /** Writes a @c param_type to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { os << parm._a << ' ' << parm._b; return os; } - - /** Reads a @c param_type from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { is >> parm._a >> std::ws >> parm._b; return is; } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._a == rhs._a && lhs._b == rhs._b; } - - /** Returns true if the two sets of parameters are the different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _a; - RealType _b; - }; - - /** - * Constructs an @c extreme_value_distribution from its "a" and "b" parameters. - * - * Requires: b > 0 - */ - explicit extreme_value_distribution(RealType a_arg = 1.0, RealType b_arg = 1.0) - : _a(a_arg), _b(b_arg) - {} - /** Constructs an @c extreme_value_distribution from its parameters. */ - explicit extreme_value_distribution(const param_type& parm) - : _a(parm.a()), _b(parm.b()) - {} - - /** - * Returns a random variate distributed according to the - * @c extreme_value_distribution. - */ - template - RealType operator()(URNG& urng) const - { - using std::log; - return _a - log(-log(uniform_01()(urng))) * _b; - } - - /** - * Returns a random variate distributed accordint to the extreme - * value distribution with parameters specified by @c param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return extreme_value_distribution(parm)(urng); - } - - /** Returns the "a" parameter of the distribution. */ - RealType a() const { return _a; } - /** Returns the "b" parameter of the distribution. */ - RealType b() const { return _b; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return -std::numeric_limits::infinity(); } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return std::numeric_limits::infinity(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_a, _b); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _a = parm.a(); - _b = parm.b(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Writes an @c extreme_value_distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, extreme_value_distribution, wd) - { - os << wd.param(); - return os; - } - - /** Reads an @c extreme_value_distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, extreme_value_distribution, wd) - { - param_type parm; - if(is >> parm) { - wd.param(parm); - } - return is; - } - - /** - * Returns true if the two instances of @c extreme_value_distribution will - * return identical sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(extreme_value_distribution, lhs, rhs) - { return lhs._a == rhs._a && lhs._b == rhs._b; } - - /** - * Returns true if the two instances of @c extreme_value_distribution will - * return different sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(extreme_value_distribution) - -private: - RealType _a; - RealType _b; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_EXTREME_VALUE_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/fisher_f_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/fisher_f_distribution.hpp deleted file mode 100644 index cdf14da8f6c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/fisher_f_distribution.hpp +++ /dev/null @@ -1,183 +0,0 @@ -/* boost random/fisher_f_distribution.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_FISHER_F_DISTRIBUTION_HPP -#define BOOST_RANDOM_FISHER_F_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The Fisher F distribution is a real valued distribution with two - * parameters m and n. - * - * It has \f$\displaystyle p(x) = - * \frac{\Gamma((m+n)/2)}{\Gamma(m/2)\Gamma(n/2)} - * \left(\frac{m}{n}\right)^{m/2} - * x^{(m/2)-1} \left(1+\frac{mx}{n}\right)^{-(m+n)/2} - * \f$. - */ -template -class fisher_f_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef fisher_f_distribution distribution_type; - - /** - * Constructs a @c param_type from the "m" and "n" parameters - * of the distribution. - * - * Requires: m > 0 and n > 0 - */ - explicit param_type(RealType m_arg = RealType(1.0), - RealType n_arg = RealType(1.0)) - : _m(m_arg), _n(n_arg) - {} - - /** Returns the "m" parameter of the distribtuion. */ - RealType m() const { return _m; } - /** Returns the "n" parameter of the distribution. */ - RealType n() const { return _n; } - - /** Writes a @c param_type to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { os << parm._m << ' ' << parm._n; return os; } - - /** Reads a @c param_type from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { is >> parm._m >> std::ws >> parm._n; return is; } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._m == rhs._m && lhs._n == rhs._n; } - - /** Returns true if the two sets of parameters are the different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _m; - RealType _n; - }; - - /** - * Constructs a @c fisher_f_distribution from its "m" and "n" parameters. - * - * Requires: m > 0 and n > 0 - */ - explicit fisher_f_distribution(RealType m_arg = RealType(1.0), - RealType n_arg = RealType(1.0)) - : _impl_m(m_arg), _impl_n(n_arg) - {} - /** Constructs an @c fisher_f_distribution from its parameters. */ - explicit fisher_f_distribution(const param_type& parm) - : _impl_m(parm.m()), _impl_n(parm.n()) - {} - - /** - * Returns a random variate distributed according to the - * F distribution. - */ - template - RealType operator()(URNG& urng) - { - return (_impl_m(urng) * n()) / (_impl_n(urng) * m()); - } - - /** - * Returns a random variate distributed according to the - * F distribution with parameters specified by @c param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return fisher_f_distribution(parm)(urng); - } - - /** Returns the "m" parameter of the distribution. */ - RealType m() const { return _impl_m.n(); } - /** Returns the "n" parameter of the distribution. */ - RealType n() const { return _impl_n.n(); } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return 0; } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return std::numeric_limits::infinity(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(m(), n()); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - typedef chi_squared_distribution impl_type; - typename impl_type::param_type m_param(parm.m()); - _impl_m.param(m_param); - typename impl_type::param_type n_param(parm.n()); - _impl_n.param(n_param); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Writes an @c fisher_f_distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, fisher_f_distribution, fd) - { - os << fd.param(); - return os; - } - - /** Reads an @c fisher_f_distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, fisher_f_distribution, fd) - { - param_type parm; - if(is >> parm) { - fd.param(parm); - } - return is; - } - - /** - * Returns true if the two instances of @c fisher_f_distribution will - * return identical sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(fisher_f_distribution, lhs, rhs) - { return lhs._impl_m == rhs._impl_m && lhs._impl_n == rhs._impl_n; } - - /** - * Returns true if the two instances of @c fisher_f_distribution will - * return different sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(fisher_f_distribution) - -private: - chi_squared_distribution _impl_m; - chi_squared_distribution _impl_n; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_EXTREME_VALUE_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/gamma_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/gamma_distribution.hpp deleted file mode 100644 index c9bda4ca9d3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/gamma_distribution.hpp +++ /dev/null @@ -1,292 +0,0 @@ -/* boost random/gamma_distribution.hpp header file - * - * Copyright Jens Maurer 2002 - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_GAMMA_DISTRIBUTION_HPP -#define BOOST_RANDOM_GAMMA_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -// The algorithm is taken from Knuth - -/** - * The gamma distribution is a continuous distribution with two - * parameters alpha and beta. It produces values > 0. - * - * It has - * \f$\displaystyle p(x) = x^{\alpha-1}\frac{e^{-x/\beta}}{\beta^\alpha\Gamma(\alpha)}\f$. - */ -template -class gamma_distribution -{ -public: - typedef RealType input_type; - typedef RealType result_type; - - class param_type - { - public: - typedef gamma_distribution distribution_type; - - /** - * Constructs a @c param_type object from the "alpha" and "beta" - * parameters. - * - * Requires: alpha > 0 && beta > 0 - */ - param_type(const RealType& alpha_arg = RealType(1.0), - const RealType& beta_arg = RealType(1.0)) - : _alpha(alpha_arg), _beta(beta_arg) - { - } - - /** Returns the "alpha" parameter of the distribution. */ - RealType alpha() const { return _alpha; } - /** Returns the "beta" parameter of the distribution. */ - RealType beta() const { return _beta; } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const param_type& parm) - { - os << parm._alpha << ' ' << parm._beta; - return os; - } - - /** Reads the parameters from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, param_type& parm) - { - is >> parm._alpha >> std::ws >> parm._beta; - return is; - } -#endif - - /** Returns true if the two sets of parameters are the same. */ - friend bool operator==(const param_type& lhs, const param_type& rhs) - { - return lhs._alpha == rhs._alpha && lhs._beta == rhs._beta; - } - /** Returns true if the two sets fo parameters are different. */ - friend bool operator!=(const param_type& lhs, const param_type& rhs) - { - return !(lhs == rhs); - } - private: - RealType _alpha; - RealType _beta; - }; - -#ifndef BOOST_NO_LIMITS_COMPILE_TIME_CONSTANTS - BOOST_STATIC_ASSERT(!std::numeric_limits::is_integer); -#endif - - /** - * Creates a new gamma_distribution with parameters "alpha" and "beta". - * - * Requires: alpha > 0 && beta > 0 - */ - explicit gamma_distribution(const result_type& alpha_arg = result_type(1.0), - const result_type& beta_arg = result_type(1.0)) - : _exp(result_type(1)), _alpha(alpha_arg), _beta(beta_arg) - { - BOOST_ASSERT(_alpha > result_type(0)); - BOOST_ASSERT(_beta > result_type(0)); - init(); - } - - /** Constructs a @c gamma_distribution from its parameters. */ - explicit gamma_distribution(const param_type& parm) - : _exp(result_type(1)), _alpha(parm.alpha()), _beta(parm.beta()) - { - init(); - } - - // compiler-generated copy ctor and assignment operator are fine - - /** Returns the "alpha" paramter of the distribution. */ - RealType alpha() const { return _alpha; } - /** Returns the "beta" parameter of the distribution. */ - RealType beta() const { return _beta; } - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return 0; } - /* Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return (std::numeric_limits::infinity)(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_alpha, _beta); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _alpha = parm.alpha(); - _beta = parm.beta(); - init(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { _exp.reset(); } - - /** - * Returns a random variate distributed according to - * the gamma distribution. - */ - template - result_type operator()(Engine& eng) - { -#ifndef BOOST_NO_STDC_NAMESPACE - // allow for Koenig lookup - using std::tan; using std::sqrt; using std::exp; using std::log; - using std::pow; -#endif - if(_alpha == result_type(1)) { - return _exp(eng) * _beta; - } else if(_alpha > result_type(1)) { - // Can we have a boost::mathconst please? - const result_type pi = result_type(3.14159265358979323846); - for(;;) { - result_type y = tan(pi * uniform_01()(eng)); - result_type x = sqrt(result_type(2)*_alpha-result_type(1))*y - + _alpha-result_type(1); - if(x <= result_type(0)) - continue; - if(uniform_01()(eng) > - (result_type(1)+y*y) * exp((_alpha-result_type(1)) - *log(x/(_alpha-result_type(1))) - - sqrt(result_type(2)*_alpha - -result_type(1))*y)) - continue; - return x * _beta; - } - } else /* alpha < 1.0 */ { - for(;;) { - result_type u = uniform_01()(eng); - result_type y = _exp(eng); - result_type x, q; - if(u < _p) { - x = exp(-y/_alpha); - q = _p*exp(-x); - } else { - x = result_type(1)+y; - q = _p + (result_type(1)-_p) * pow(x,_alpha-result_type(1)); - } - if(u >= q) - continue; - return x * _beta; - } - } - } - - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return gamma_distribution(parm)(urng); - } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes a @c gamma_distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const gamma_distribution& gd) - { - os << gd.param(); - return os; - } - - /** Reads a @c gamma_distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, gamma_distribution& gd) - { - gd.read(is); - return is; - } -#endif - - /** - * Returns true if the two distributions will produce identical - * sequences of random variates given equal generators. - */ - friend bool operator==(const gamma_distribution& lhs, - const gamma_distribution& rhs) - { - return lhs._alpha == rhs._alpha - && lhs._beta == rhs._beta - && lhs._exp == rhs._exp; - } - - /** - * Returns true if the two distributions can produce different - * sequences of random variates, given equal generators. - */ - friend bool operator!=(const gamma_distribution& lhs, - const gamma_distribution& rhs) - { - return !(lhs == rhs); - } - -private: - /// \cond hide_private_members - - template - void read(std::basic_istream& is) - { - param_type parm; - if(is >> parm) { - param(parm); - } - } - - void init() - { -#ifndef BOOST_NO_STDC_NAMESPACE - // allow for Koenig lookup - using std::exp; -#endif - _p = exp(result_type(1)) / (_alpha + exp(result_type(1))); - } - /// \endcond - - exponential_distribution _exp; - result_type _alpha; - result_type _beta; - // some data precomputed from the parameters - result_type _p; -}; - - -} // namespace random - -using random::gamma_distribution; - -} // namespace boost - -#endif // BOOST_RANDOM_GAMMA_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/generate_canonical.hpp b/contrib/libboost/boost_1_65_0/boost/random/generate_canonical.hpp deleted file mode 100644 index 112784742b1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/generate_canonical.hpp +++ /dev/null @@ -1,96 +0,0 @@ -/* boost random/generate_canonical.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_GENERATE_CANONICAL_HPP -#define BOOST_RANDOM_GENERATE_CANONICAL_HPP - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -namespace detail { - -template -RealType generate_canonical_impl(URNG& g, boost::mpl::true_ /*is_integral*/) -{ - using std::pow; - typedef typename URNG::result_type base_result; - std::size_t digits = std::numeric_limits::digits; - RealType R = RealType((g.max)()) - RealType((g.min)()) + 1; - RealType mult = R; - RealType limit = - pow(RealType(2), - RealType((std::min)(static_cast(bits), digits))); - RealType S = RealType(detail::subtract()(g(), (g.min)())); - while(mult < limit) { - RealType inc = RealType(detail::subtract()(g(), (g.min)())); - S += inc * mult; - mult *= R; - } - return S / mult; -} - -template -RealType generate_canonical_impl(URNG& g, boost::mpl::false_ /*is_integral*/) -{ - using std::pow; - using std::floor; - BOOST_ASSERT((g.min)() == 0); - BOOST_ASSERT((g.max)() == 1); - std::size_t digits = std::numeric_limits::digits; - std::size_t engine_bits = detail::generator_bits::value(); - std::size_t b = (std::min)(bits, digits); - RealType R = pow(RealType(2), RealType(engine_bits)); - RealType mult = R; - RealType limit = pow(RealType(2), RealType(b)); - RealType S = RealType(g() - (g.min)()); - while(mult < limit) { - RealType inc(floor((RealType(g()) - RealType((g.min)())) * R)); - S += inc * mult; - mult *= R; - } - return S / mult; -} - -} - -/** - * Returns a value uniformly distributed in the range [0, 1) - * with at least @c bits random bits. - */ -template -RealType generate_canonical(URNG& g) -{ - RealType result = detail::generate_canonical_impl( - g, boost::random::traits::is_integral()); - BOOST_ASSERT(result >= 0); - BOOST_ASSERT(result <= 1); - if(result == 1) { - result -= std::numeric_limits::epsilon() / 2; - BOOST_ASSERT(result != 1); - } - return result; -} - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_GENERATE_CANONICAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/geometric_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/geometric_distribution.hpp deleted file mode 100644 index 90374cff71b..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/geometric_distribution.hpp +++ /dev/null @@ -1,267 +0,0 @@ -/* boost random/geometric_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_GEOMETRIC_DISTRIBUTION_HPP -#define BOOST_RANDOM_GEOMETRIC_DISTRIBUTION_HPP - -#include // std::log -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * An instantiation of the class template @c geometric_distribution models - * a \random_distribution. The distribution produces positive - * integers which are the number of bernoulli trials - * with probability @c p required to get one that fails. - * - * For the geometric distribution, \f$p(i) = p(1-p)^{i}\f$. - * - * @xmlwarning - * This distribution has been updated to match the C++ standard. - * Its behavior has changed from the original - * boost::geometric_distribution. A backwards compatible - * wrapper is provided in namespace boost. - * @endxmlwarning - */ -template -class geometric_distribution -{ -public: - typedef RealType input_type; - typedef IntType result_type; - - class param_type - { - public: - - typedef geometric_distribution distribution_type; - - /** Constructs the parameters with p. */ - explicit param_type(RealType p_arg = RealType(0.5)) - : _p(p_arg) - { - BOOST_ASSERT(RealType(0) < _p && _p < RealType(1)); - } - - /** Returns the p parameter of the distribution. */ - RealType p() const { return _p; } - - /** Writes the parameters to a std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._p; - return os; - } - - /** Reads the parameters from a std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - double p_in; - if(is >> p_in) { - if(p_in > RealType(0) && p_in < RealType(1)) { - parm._p = p_in; - } else { - is.setstate(std::ios_base::failbit); - } - } - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._p == rhs._p; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - - private: - RealType _p; - }; - - /** - * Contructs a new geometric_distribution with the paramter @c p. - * - * Requires: 0 < p < 1 - */ - explicit geometric_distribution(const RealType& p_arg = RealType(0.5)) - : _p(p_arg) - { - BOOST_ASSERT(RealType(0) < _p && _p < RealType(1)); - init(); - } - - /** Constructs a new geometric_distribution from its parameters. */ - explicit geometric_distribution(const param_type& parm) - : _p(parm.p()) - { - init(); - } - - // compiler-generated copy ctor and assignment operator are fine - - /** Returns: the distribution parameter @c p */ - RealType p() const { return _p; } - - /** Returns the smallest value that the distribution can produce. */ - IntType min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return IntType(0); } - - /** Returns the largest value that the distribution can produce. */ - IntType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return (std::numeric_limits::max)(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_p); } - - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _p = parm.p(); - init(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** - * Returns a random variate distributed according to the - * geometric_distribution. - */ - template - result_type operator()(Engine& eng) const - { - using std::log; - using std::floor; - RealType x = RealType(1) - boost::uniform_01()(eng); - return IntType(floor(log(x) / _log_1mp)); - } - - /** - * Returns a random variate distributed according to the - * geometric distribution with parameters specified by param. - */ - template - result_type operator()(Engine& eng, const param_type& parm) const - { return geometric_distribution(parm)(eng); } - - /** Writes the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, geometric_distribution, gd) - { - os << gd._p; - return os; - } - - /** Reads the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, geometric_distribution, gd) - { - param_type parm; - if(is >> parm) { - gd.param(parm); - } - return is; - } - - /** - * Returns true if the two distributions will produce identical - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(geometric_distribution, lhs, rhs) - { return lhs._p == rhs._p; } - - /** - * Returns true if the two distributions may produce different - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(geometric_distribution) - -private: - - /// \cond show_private - - void init() - { - using std::log; - _log_1mp = log(1 - _p); - } - - RealType _p; - RealType _log_1mp; - - /// \endcond -}; - -} // namespace random - -/// \cond show_deprecated - -/** - * Provided for backwards compatibility. This class is - * deprecated. It provides the old behavior of geometric_distribution - * with \f$p(i) = (1-p) p^{i-1}\f$. - */ -template -class geometric_distribution -{ -public: - typedef RealType input_type; - typedef IntType result_type; - - explicit geometric_distribution(RealType p_arg = RealType(0.5)) - : _impl(1 - p_arg) {} - - RealType p() const { return 1 - _impl.p(); } - - void reset() {} - - template - IntType operator()(Engine& eng) const { return _impl(eng) + IntType(1); } - - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, geometric_distribution, gd) - { - os << gd.p(); - return os; - } - - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, geometric_distribution, gd) - { - RealType val; - if(is >> val) { - typename impl_type::param_type impl_param(1 - val); - gd._impl.param(impl_param); - } - return is; - } - -private: - typedef random::geometric_distribution impl_type; - impl_type _impl; -}; - -/// \endcond - -} // namespace boost - -#endif // BOOST_RANDOM_GEOMETRIC_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/hyperexponential_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/hyperexponential_distribution.hpp deleted file mode 100644 index 046d0fc4016..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/hyperexponential_distribution.hpp +++ /dev/null @@ -1,883 +0,0 @@ -/* boost random/hyperexponential_distribution.hpp header file - * - * Copyright Marco Guazzone 2014 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * Much of the code here taken by boost::math::hyperexponential_distribution. - * To this end, we would like to thank Paul Bristow and John Maddock for their - * valuable feedback. - * - * \author Marco Guazzone (marco.guazzone@gmail.com) - */ - -#ifndef BOOST_RANDOM_HYPEREXPONENTIAL_DISTRIBUTION_HPP -#define BOOST_RANDOM_HYPEREXPONENTIAL_DISTRIBUTION_HPP - - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST -# include -#endif // BOOST_NO_CXX11_HDR_INITIALIZER_LIST -#include -#include -#include -#include - - -namespace boost { namespace random { - -namespace hyperexp_detail { - -template -std::vector& normalize(std::vector& v) -{ - if (v.size() == 0) - { - return v; - } - - const T sum = std::accumulate(v.begin(), v.end(), static_cast(0)); - T final_sum = 0; - - const typename std::vector::iterator end = --v.end(); - for (typename std::vector::iterator it = v.begin(); - it != end; - ++it) - { - *it /= sum; - final_sum += *it; - } - *end = 1-final_sum; // avoids round off errors thus ensuring the probabilities really sum to 1 - - return v; -} - -template -bool check_probabilities(std::vector const& probabilities) -{ - const std::size_t n = probabilities.size(); - RealT sum = 0; - for (std::size_t i = 0; i < n; ++i) - { - if (probabilities[i] < 0 - || probabilities[i] > 1 - || !(boost::math::isfinite)(probabilities[i])) - { - return false; - } - sum += probabilities[i]; - } - - //NOTE: the check below seems to fail on some architectures. - // So we commented it. - //// - We try to keep phase probabilities correctly normalized in the distribution constructors - //// - However in practice we have to allow for a very slight divergence from a sum of exactly 1: - ////if (std::abs(sum-1) > (std::numeric_limits::epsilon()*2)) - //// This is from Knuth "The Art of Computer Programming: Vol.2, 3rd Ed", and can be used to - //// check is two numbers are approximately equal - //const RealT one = 1; - //const RealT tol = std::numeric_limits::epsilon()*2.0; - //if (std::abs(sum-one) > (std::max(std::abs(sum), std::abs(one))*tol)) - //{ - // return false; - //} - - return true; -} - -template -bool check_rates(std::vector const& rates) -{ - const std::size_t n = rates.size(); - for (std::size_t i = 0; i < n; ++i) - { - if (rates[i] <= 0 - || !(boost::math::isfinite)(rates[i])) - { - return false; - } - } - return true; -} - -template -bool check_params(std::vector const& probabilities, std::vector const& rates) -{ - if (probabilities.size() != rates.size()) - { - return false; - } - - return check_probabilities(probabilities) - && check_rates(rates); -} - -} // Namespace hyperexp_detail - - -/** - * The hyperexponential distribution is a real-valued continuous distribution - * with two parameters, the phase probability vector \c probs and the - * rate vector \c rates. - * - * A \f$k\f$-phase hyperexponential distribution is a mixture of \f$k\f$ - * exponential distributions. - * For this reason, it is also referred to as mixed exponential - * distribution or parallel \f$k\f$-phase exponential - * distribution. - * - * A \f$k\f$-phase hyperexponential distribution is characterized by two - * parameters, namely a phase probability vector \f$\mathbf{\alpha}=(\alpha_1,\ldots,\alpha_k)\f$ and a rate vector \f$\mathbf{\lambda}=(\lambda_1,\ldots,\lambda_k)\f$. - * - * A \f$k\f$-phase hyperexponential distribution is frequently used in - * queueing theory to model the distribution of the superposition of - * \f$k\f$ independent events, like, for instance, the service time distribution - * of a queueing station with \f$k\f$ servers in parallel where the \f$i\f$-th - * server is chosen with probability \f$\alpha_i\f$ and its service time - * distribution is an exponential distribution with rate \f$\lambda_i\f$ - * (Allen,1990; Papadopolous et al.,1993; Trivedi,2002). - * - * For instance, CPUs service-time distribution in a computing system has often - * been observed to possess such a distribution (Rosin,1965). - * Also, the arrival of different types of customer to a single queueing station - * is often modeled as a hyperexponential distribution (Papadopolous et al.,1993). - * Similarly, if a product manufactured in several parallel assemply lines and - * the outputs are merged, the failure density of the overall product is likely - * to be hyperexponential (Trivedi,2002). - * - * Finally, since the hyperexponential distribution exhibits a high Coefficient - * of Variation (CoV), that is a CoV > 1, it is especially suited to fit - * empirical data with large CoV (Feitelson,2014; Wolski et al.,2013) and to - * approximate long-tail probability distributions (Feldmann et al.,1998). - * - * See (Boost,2014) for more information and examples. - * - * A \f$k\f$-phase hyperexponential distribution has a probability density - * function - * \f[ - * f(x) = \sum_{i=1}^k \alpha_i \lambda_i e^{-x\lambda_i} - * \f] - * where: - * - \f$k\f$ is the number of phases and also the size of the input - * vector parameters, - * - \f$\mathbf{\alpha}=(\alpha_1,\ldots,\alpha_k)\f$ is the phase probability - * vector parameter, and - * - \f$\mathbf{\lambda}=(\lambda_1,\ldots,\lambda_k)\f$ is the rate vector - * parameter. - * . - * - * Given a \f$k\f$-phase hyperexponential distribution with phase probability - * vector \f$\mathbf{\alpha}\f$ and rate vector \f$\mathbf{\lambda}\f$, the - * random variate generation algorithm consists of the following steps (Tyszer,1999): - * -# Generate a random variable \f$U\f$ uniformly distribution on the interval \f$(0,1)\f$. - * -# Use \f$U\f$ to select the appropriate \f$\lambda_i\f$ (e.g., the - * alias method can possibly be used for this step). - * -# Generate an exponentially distributed random variable \f$X\f$ with rate parameter \f$\lambda_i\f$. - * -# Return \f$X\f$. - * . - * - * References: - * -# A.O. Allen, Probability, Statistics, and Queuing Theory with Computer Science Applications, Second Edition, Academic Press, 1990. - * -# Boost C++ Libraries, Boost.Math / Statistical Distributions: Hyperexponential Distribution, Online: http://www.boost.org/doc/libs/release/libs/math/doc/html/dist.html , 2014. - * -# D.G. Feitelson, Workload Modeling for Computer Systems Performance Evaluation, Cambridge University Press, 2014 - * -# A. Feldmann and W. Whitt, Fitting mixtures of exponentials to long-tail distributions to analyze network performance models, Performance Evaluation 31(3-4):245, doi:10.1016/S0166-5316(97)00003-5, 1998. - * -# H.T. Papadopolous, C. Heavey and J. Browne, Queueing Theory in Manufacturing Systems Analysis and Design, Chapman & Hall/CRC, 1993, p. 35. - * -# R.F. Rosin, Determining a computing center environment, Communications of the ACM 8(7):463-468, 1965. - * -# K.S. Trivedi, Probability and Statistics with Reliability, Queueing, and Computer Science Applications, John Wiley & Sons, Inc., 2002. - * -# J. Tyszer, Object-Oriented Computer Simulation of Discrete-Event Systems, Springer, 1999. - * -# Wikipedia, Hyperexponential Distribution, Online: http://en.wikipedia.org/wiki/Hyperexponential_distribution , 2014. - * -# Wolfram Mathematica, Hyperexponential Distribution, Online: http://reference.wolfram.com/language/ref/HyperexponentialDistribution.html , 2014. - * . - * - * \author Marco Guazzone (marco.guazzone@gmail.com) - */ -template -class hyperexponential_distribution -{ - public: typedef RealT result_type; - public: typedef RealT input_type; - - - /** - * The parameters of a hyperexponential distribution. - * - * Stores the phase probability vector and the rate vector - * of the hyperexponential distribution. - * - * \author Marco Guazzone (marco.guazzone@gmail.com) - */ - public: class param_type - { - public: typedef hyperexponential_distribution distribution_type; - - /** - * Constructs a \c param_type with the default parameters - * of the distribution. - */ - public: param_type() - : probs_(1, 1), - rates_(1, 1) - { - } - - /** - * Constructs a \c param_type from the phase probability vector - * and rate vector parameters of the distribution. - * - * The phase probability vector parameter is given by the range - * defined by [\a prob_first, \a prob_last) iterator pair, and the - * rate vector parameter is given by the range defined by - * [\a rate_first, \a rate_last) iterator pair. - * - * \tparam ProbIterT Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * \tparam RateIterT Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * - * \param prob_first The iterator to the beginning of the range of non-negative real elements representing the phase probabilities; if elements don't sum to 1, they are normalized. - * \param prob_last The iterator to the ending of the range of non-negative real elements representing the phase probabilities; if elements don't sum to 1, they are normalized. - * \param rate_first The iterator to the beginning of the range of non-negative real elements representing the rates. - * \param rate_last The iterator to the ending of the range of non-negative real elements representing the rates. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - public: template - param_type(ProbIterT prob_first, ProbIterT prob_last, - RateIterT rate_first, RateIterT rate_last) - : probs_(prob_first, prob_last), - rates_(rate_first, rate_last) - { - hyperexp_detail::normalize(probs_); - - assert( hyperexp_detail::check_params(probs_, rates_) ); - } - - /** - * Constructs a \c param_type from the phase probability vector - * and rate vector parameters of the distribution. - * - * The phase probability vector parameter is given by the range - * defined by \a prob_range, and the rate vector parameter is - * given by the range defined by \a rate_range. - * - * \tparam ProbRangeT Must meet the requirements of Range concept. - * \tparam RateRangeT Must meet the requirements of Range concept. - * - * \param prob_range The range of non-negative real elements representing the phase probabilities; if elements don't sum to 1, they are normalized. - * \param rate_range The range of positive real elements representing the rates. - * - * \note - * The final \c disable_if parameter is an implementation detail that - * differentiates between this two argument constructor and the - * iterator-based two argument constructor described below. - */ - // We SFINAE this out of existance if either argument type is - // incrementable as in that case the type is probably an iterator: - public: template - param_type(ProbRangeT const& prob_range, - RateRangeT const& rate_range, - typename boost::disable_if_c::value || boost::has_pre_increment::value>::type* = 0) - : probs_(boost::begin(prob_range), boost::end(prob_range)), - rates_(boost::begin(rate_range), boost::end(rate_range)) - { - hyperexp_detail::normalize(probs_); - - assert( hyperexp_detail::check_params(probs_, rates_) ); - } - - /** - * Constructs a \c param_type from the rate vector parameter of - * the distribution and with equal phase probabilities. - * - * The rate vector parameter is given by the range defined by - * [\a rate_first, \a rate_last) iterator pair, and the phase - * probability vector parameter is set to the equal phase - * probabilities (i.e., to a vector of the same length \f$k\f$ of the - * rate vector and with each element set to \f$1.0/k\f$). - * - * \tparam RateIterT Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * \tparam RateIterT2 Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * - * \param rate_first The iterator to the beginning of the range of non-negative real elements representing the rates. - * \param rate_last The iterator to the ending of the range of non-negative real elements representing the rates. - * - * \note - * The final \c disable_if parameter is an implementation detail that - * differentiates between this two argument constructor and the - * range-based two argument constructor described above. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - // We SFINAE this out of existance if the argument type is - // incrementable as in that case the type is probably an iterator. - public: template - param_type(RateIterT rate_first, - RateIterT rate_last, - typename boost::enable_if_c::value>::type* = 0) - : probs_(std::distance(rate_first, rate_last), 1), // will be normalized below - rates_(rate_first, rate_last) - { - assert(probs_.size() == rates_.size()); - } - - /** - * Constructs a @c param_type from the "rates" parameters - * of the distribution and with equal phase probabilities. - * - * The rate vector parameter is given by the range defined by - * \a rate_range, and the phase probability vector parameter is - * set to the equal phase probabilities (i.e., to a vector of the same - * length \f$k\f$ of the rate vector and with each element set - * to \f$1.0/k\f$). - * - * \tparam RateRangeT Must meet the requirements of Range concept. - * - * \param rate_range The range of positive real elements representing the rates. - */ - public: template - param_type(RateRangeT const& rate_range) - : probs_(boost::size(rate_range), 1), // Will be normalized below - rates_(boost::begin(rate_range), boost::end(rate_range)) - { - hyperexp_detail::normalize(probs_); - - assert( hyperexp_detail::check_params(probs_, rates_) ); - } - -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * Constructs a \c param_type from the phase probability vector - * and rate vector parameters of the distribution. - * - * The phase probability vector parameter is given by the - * brace-init-list (ISO,2014,sec. 8.5.4 [dcl.init.list]) - * defined by \a l1, and the rate vector parameter is given by the - * brace-init-list (ISO,2014,sec. 8.5.4 [dcl.init.list]) - * defined by \a l2. - * - * \param l1 The initializer list for inizializing the phase probability vector. - * \param l2 The initializer list for inizializing the rate vector. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - public: param_type(std::initializer_list l1, std::initializer_list l2) - : probs_(l1.begin(), l1.end()), - rates_(l2.begin(), l2.end()) - { - hyperexp_detail::normalize(probs_); - - assert( hyperexp_detail::check_params(probs_, rates_) ); - } - - /** - * Constructs a \c param_type from the rate vector parameter - * of the distribution and with equal phase probabilities. - * - * The rate vector parameter is given by the - * brace-init-list (ISO,2014,sec. 8.5.4 [dcl.init.list]) - * defined by \a l1, and the phase probability vector parameter is - * set to the equal phase probabilities (i.e., to a vector of the same - * length \f$k\f$ of the rate vector and with each element set - * to \f$1.0/k\f$). - * - * \param l1 The initializer list for inizializing the rate vector. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - public: param_type(std::initializer_list l1) - : probs_(std::distance(l1.begin(), l1.end()), 1), // Will be normalized below - rates_(l1.begin(), l1.end()) - { - hyperexp_detail::normalize(probs_); - - assert( hyperexp_detail::check_params(probs_, rates_) ); - } -#endif // BOOST_NO_CXX11_HDR_INITIALIZER_LIST - - /** - * Gets the phase probability vector parameter of the distribtuion. - * - * \return The phase probability vector parameter of the distribution. - * - * \note - * The returned probabilities are the normalized version of the ones - * passed at construction time. - */ - public: std::vector probabilities() const - { - return probs_; - } - - /** - * Gets the rate vector parameter of the distribtuion. - * - * \return The rate vector parameter of the distribution. - */ - public: std::vector rates() const - { - return rates_; - } - - /** Writes a \c param_type to a \c std::ostream. */ - public: BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, param) - { - detail::print_vector(os, param.probs_); - os << ' '; - detail::print_vector(os, param.rates_); - - return os; - } - - /** Reads a \c param_type from a \c std::istream. */ - public: BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, param) - { - // NOTE: if \c std::ios_base::exceptions is set, the code below may - // throw in case of a I/O failure. - // To prevent leaving the state of \c param inconsistent: - // - if an exception is thrown, the state of \c param is left - // unchanged (i.e., is the same as the one at the beginning - // of the function's execution), and - // - the state of \c param only after reading the whole input. - - std::vector probs; - std::vector rates; - - // Reads probability and rate vectors - detail::read_vector(is, probs); - if (!is) - { - return is; - } - is >> std::ws; - detail::read_vector(is, rates); - if (!is) - { - return is; - } - - // Update the state of the param_type object - if (probs.size() > 0) - { - param.probs_.swap(probs); - probs.clear(); - } - if (rates.size() > 0) - { - param.rates_.swap(rates); - rates.clear(); - } - - bool fail = false; - - // Adjust vector sizes (if needed) - if (param.probs_.size() != param.rates_.size() - || param.probs_.size() == 0) - { - fail = true; - - const std::size_t np = param.probs_.size(); - const std::size_t nr = param.rates_.size(); - - if (np > nr) - { - param.rates_.resize(np, 1); - } - else if (nr > np) - { - param.probs_.resize(nr, 1); - } - else - { - param.probs_.resize(1, 1); - param.rates_.resize(1, 1); - } - } - - // Normalize probabilities - // NOTE: this cannot be done earlier since the probability vector - // can be changed due to size conformance - hyperexp_detail::normalize(param.probs_); - - // Set the error state in the underlying stream in case of invalid input - if (fail) - { - // This throws an exception if ios_base::exception(failbit) is enabled - is.setstate(std::ios_base::failbit); - } - - //post: vector size conformance - assert(param.probs_.size() == param.rates_.size()); - - return is; - } - - /** Returns true if the two sets of parameters are the same. */ - public: BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { - return lhs.probs_ == rhs.probs_ - && lhs.rates_ == rhs.rates_; - } - - /** Returns true if the two sets of parameters are the different. */ - public: BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - - private: std::vector probs_; ///< The phase probability vector parameter of the distribution - private: std::vector rates_; ///< The rate vector parameter of the distribution - }; // param_type - - - /** - * Constructs a 1-phase \c hyperexponential_distribution (i.e., an - * exponential distribution) with rate 1. - */ - public: hyperexponential_distribution() - : dd_(std::vector(1, 1)), - rates_(1, 1) - { - // empty - } - - /** - * Constructs a \c hyperexponential_distribution from the phase - * probability vector and rate vector parameters of the - * distribution. - * - * The phase probability vector parameter is given by the range - * defined by [\a prob_first, \a prob_last) iterator pair, and the - * rate vector parameter is given by the range defined by - * [\a rate_first, \a rate_last) iterator pair. - * - * \tparam ProbIterT Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * \tparam RateIterT Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * - * \param prob_first The iterator to the beginning of the range of non-negative real elements representing the phase probabilities; if elements don't sum to 1, they are normalized. - * \param prob_last The iterator to the ending of the range of non-negative real elements representing the phase probabilities; if elements don't sum to 1, they are normalized. - * \param rate_first The iterator to the beginning of the range of non-negative real elements representing the rates. - * \param rate_last The iterator to the ending of the range of non-negative real elements representing the rates. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - public: template - hyperexponential_distribution(ProbIterT prob_first, ProbIterT prob_last, - RateIterT rate_first, RateIterT rate_last) - : dd_(prob_first, prob_last), - rates_(rate_first, rate_last) - { - assert( hyperexp_detail::check_params(dd_.probabilities(), rates_) ); - } - - /** - * Constructs a \c hyperexponential_distribution from the phase - * probability vector and rate vector parameters of the - * distribution. - * - * The phase probability vector parameter is given by the range - * defined by \a prob_range, and the rate vector parameter is - * given by the range defined by \a rate_range. - * - * \tparam ProbRangeT Must meet the requirements of Range concept. - * \tparam RateRangeT Must meet the requirements of Range concept. - * - * \param prob_range The range of non-negative real elements representing the phase probabilities; if elements don't sum to 1, they are normalized. - * \param rate_range The range of positive real elements representing the rates. - * - * \note - * The final \c disable_if parameter is an implementation detail that - * differentiates between this two argument constructor and the - * iterator-based two argument constructor described below. - */ - // We SFINAE this out of existance if either argument type is - // incrementable as in that case the type is probably an iterator: - public: template - hyperexponential_distribution(ProbRangeT const& prob_range, - RateRangeT const& rate_range, - typename boost::disable_if_c::value || boost::has_pre_increment::value>::type* = 0) - : dd_(prob_range), - rates_(boost::begin(rate_range), boost::end(rate_range)) - { - assert( hyperexp_detail::check_params(dd_.probabilities(), rates_) ); - } - - /** - * Constructs a \c hyperexponential_distribution from the rate - * vector parameter of the distribution and with equal phase - * probabilities. - * - * The rate vector parameter is given by the range defined by - * [\a rate_first, \a rate_last) iterator pair, and the phase - * probability vector parameter is set to the equal phase - * probabilities (i.e., to a vector of the same length \f$k\f$ of the - * rate vector and with each element set to \f$1.0/k\f$). - * - * \tparam RateIterT Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * \tparam RateIterT2 Must meet the requirements of \c InputIterator concept (ISO,2014,sec. 24.2.3 [input.iterators]). - * - * \param rate_first The iterator to the beginning of the range of non-negative real elements representing the rates. - * \param rate_last The iterator to the ending of the range of non-negative real elements representing the rates. - * - * \note - * The final \c disable_if parameter is an implementation detail that - * differentiates between this two argument constructor and the - * range-based two argument constructor described above. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - // We SFINAE this out of existance if the argument type is - // incrementable as in that case the type is probably an iterator. - public: template - hyperexponential_distribution(RateIterT rate_first, - RateIterT rate_last, - typename boost::enable_if_c::value>::type* = 0) - : dd_(std::vector(std::distance(rate_first, rate_last), 1)), - rates_(rate_first, rate_last) - { - assert( hyperexp_detail::check_params(dd_.probabilities(), rates_) ); - } - - /** - * Constructs a @c param_type from the "rates" parameters - * of the distribution and with equal phase probabilities. - * - * The rate vector parameter is given by the range defined by - * \a rate_range, and the phase probability vector parameter is - * set to the equal phase probabilities (i.e., to a vector of the same - * length \f$k\f$ of the rate vector and with each element set - * to \f$1.0/k\f$). - * - * \tparam RateRangeT Must meet the requirements of Range concept. - * - * \param rate_range The range of positive real elements representing the rates. - */ - public: template - hyperexponential_distribution(RateRangeT const& rate_range) - : dd_(std::vector(boost::size(rate_range), 1)), - rates_(boost::begin(rate_range), boost::end(rate_range)) - { - assert( hyperexp_detail::check_params(dd_.probabilities(), rates_) ); - } - - /** - * Constructs a \c hyperexponential_distribution from its parameters. - * - * \param param The parameters of the distribution. - */ - public: explicit hyperexponential_distribution(param_type const& param) - : dd_(param.probabilities()), - rates_(param.rates()) - { - assert( hyperexp_detail::check_params(dd_.probabilities(), rates_) ); - } - -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * Constructs a \c hyperexponential_distribution from the phase - * probability vector and rate vector parameters of the - * distribution. - * - * The phase probability vector parameter is given by the - * brace-init-list (ISO,2014,sec. 8.5.4 [dcl.init.list]) - * defined by \a l1, and the rate vector parameter is given by the - * brace-init-list (ISO,2014,sec. 8.5.4 [dcl.init.list]) - * defined by \a l2. - * - * \param l1 The initializer list for inizializing the phase probability vector. - * \param l2 The initializer list for inizializing the rate vector. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - public: hyperexponential_distribution(std::initializer_list const& l1, std::initializer_list const& l2) - : dd_(l1.begin(), l1.end()), - rates_(l2.begin(), l2.end()) - { - assert( hyperexp_detail::check_params(dd_.probabilities(), rates_) ); - } - - /** - * Constructs a \c hyperexponential_distribution from the rate - * vector parameter of the distribution and with equal phase - * probabilities. - * - * The rate vector parameter is given by the - * brace-init-list (ISO,2014,sec. 8.5.4 [dcl.init.list]) - * defined by \a l1, and the phase probability vector parameter is - * set to the equal phase probabilities (i.e., to a vector of the same - * length \f$k\f$ of the rate vector and with each element set - * to \f$1.0/k\f$). - * - * \param l1 The initializer list for inizializing the rate vector. - * - * References: - * -# ISO, ISO/IEC 14882-2014: Information technology - Programming languages - C++, 2014 - * . - */ - public: hyperexponential_distribution(std::initializer_list const& l1) - : dd_(std::vector(std::distance(l1.begin(), l1.end()), 1)), - rates_(l1.begin(), l1.end()) - { - assert( hyperexp_detail::check_params(dd_.probabilities(), rates_) ); - } -#endif - - /** - * Gets a random variate distributed according to the - * hyperexponential distribution. - * - * \tparam URNG Must meet the requirements of \uniform_random_number_generator. - * - * \param urng A uniform random number generator object. - * - * \return A random variate distributed according to the hyperexponential distribution. - */ - public: template\ - RealT operator()(URNG& urng) const - { - const int i = dd_(urng); - - return boost::random::exponential_distribution(rates_[i])(urng); - } - - /** - * Gets a random variate distributed according to the hyperexponential - * distribution with parameters specified by \c param. - * - * \tparam URNG Must meet the requirements of \uniform_random_number_generator. - * - * \param urng A uniform random number generator object. - * \param param A distribution parameter object. - * - * \return A random variate distributed according to the hyperexponential distribution. - * distribution with parameters specified by \c param. - */ - public: template - RealT operator()(URNG& urng, const param_type& param) const - { - return hyperexponential_distribution(param)(urng); - } - - /** Returns the number of phases of the distribution. */ - public: std::size_t num_phases() const - { - return rates_.size(); - } - - /** Returns the phase probability vector parameter of the distribution. */ - public: std::vector probabilities() const - { - return dd_.probabilities(); - } - - /** Returns the rate vector parameter of the distribution. */ - public: std::vector rates() const - { - return rates_; - } - - /** Returns the smallest value that the distribution can produce. */ - public: RealT min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { - return 0; - } - - /** Returns the largest value that the distribution can produce. */ - public: RealT max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { - return std::numeric_limits::infinity(); - } - - /** Returns the parameters of the distribution. */ - public: param_type param() const - { - std::vector probs = dd_.probabilities(); - - return param_type(probs.begin(), probs.end(), rates_.begin(), rates_.end()); - } - - /** Sets the parameters of the distribution. */ - public: void param(param_type const& param) - { - dd_.param(typename boost::random::discrete_distribution::param_type(param.probabilities())); - rates_ = param.rates(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - public: void reset() - { - // empty - } - - /** Writes an @c hyperexponential_distribution to a @c std::ostream. */ - public: BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, hyperexponential_distribution, hd) - { - os << hd.param(); - return os; - } - - /** Reads an @c hyperexponential_distribution from a @c std::istream. */ - public: BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, hyperexponential_distribution, hd) - { - param_type param; - if(is >> param) - { - hd.param(param); - } - return is; - } - - /** - * Returns true if the two instances of @c hyperexponential_distribution will - * return identical sequences of values given equal generators. - */ - public: BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(hyperexponential_distribution, lhs, rhs) - { - return lhs.dd_ == rhs.dd_ - && lhs.rates_ == rhs.rates_; - } - - /** - * Returns true if the two instances of @c hyperexponential_distribution will - * return different sequences of values given equal generators. - */ - public: BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(hyperexponential_distribution) - - - private: boost::random::discrete_distribution dd_; ///< The \c discrete_distribution used to sample the phase probability and choose the rate - private: std::vector rates_; ///< The rate vector parameter of the distribution -}; // hyperexponential_distribution - -}} // namespace boost::random - - -#endif // BOOST_RANDOM_HYPEREXPONENTIAL_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/independent_bits.hpp b/contrib/libboost/boost_1_65_0/boost/random/independent_bits.hpp deleted file mode 100644 index dec63b3f46d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/independent_bits.hpp +++ /dev/null @@ -1,271 +0,0 @@ -/* boost random/independent_bits.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_INDEPENDENT_BITS_HPP -#define BOOST_RANDOM_INDEPENDENT_BITS_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * An instantiation of class template @c independent_bits_engine - * model a \pseudo_random_number_generator. It generates random - * numbers distributed between [0, 2^w) by combining one or - * more invocations of the base engine. - * - * Requires: 0 < w <= std::numeric_limits::digits - */ -template -class independent_bits_engine -{ -public: - typedef Engine base_type; - typedef UIntType result_type; - typedef typename Engine::result_type base_result_type; - - // Required by old Boost.Random concept - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return 0; } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return max_imp(boost::is_integral()); } - - /** - * Constructs an @c independent_bits_engine using the - * default constructor of the base generator. - */ - independent_bits_engine() { } - - /** - * Constructs an @c independent_bits_engine, using seed as - * the constructor argument for both base generators. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(independent_bits_engine, - base_result_type, seed_arg) - { - _base.seed(seed_arg); - } - - /** - * Constructs an @c independent_bits_engine, using seq as - * the constructor argument for the base generator. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(independent_bits_engine, - SeedSeq, seq) - { _base.seed(seq); } - - /** Constructs an @c independent_bits_engine by copying @c base. */ - independent_bits_engine(const base_type& base_arg) : _base(base_arg) {} - - /** - * Contructs an @c independent_bits_engine with - * values from the range defined by the input iterators first - * and last. first will be modified to point to the element - * after the last one used. - * - * Throws: @c std::invalid_argument if the input range is too small. - * - * Exception Safety: Basic - */ - template - independent_bits_engine(It& first, It last) : _base(first, last) { } - - /** - * Seeds an @c independent_bits_engine using the default - * seed of the base generator. - */ - void seed() { _base.seed(); } - - /** - * Seeds an @c independent_bits_engine, using @c seed as the - * seed for the base generator. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(independent_bits_engine, - base_result_type, seed_arg) - { _base.seed(seed_arg); } - - /** - * Seeds an @c independent_bits_engine, using @c seq to - * seed the base generator. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(independent_bits_engine, - SeedSeq, seq) - { _base.seed(seq); } - - /** - * Seeds an @c independent_bits_engine with - * values from the range defined by the input iterators first - * and last. first will be modified to point to the element - * after the last one used. - * - * Throws: @c std::invalid_argument if the input range is too small. - * - * Exception Safety: Basic - */ - template void seed(It& first, It last) - { _base.seed(first, last); } - - /** Returns the next value of the generator. */ - result_type operator()() - { - // While it may seem wasteful to recalculate this - // every time, both msvc and gcc can propagate - // constants, resolving this at compile time. - base_unsigned range = - detail::subtract()((_base.max)(), (_base.min)()); - std::size_t m = - (range == (std::numeric_limits::max)()) ? - std::numeric_limits::digits : - detail::integer_log2(range + 1); - std::size_t n = (w + m - 1) / m; - std::size_t w0, n0; - base_unsigned y0, y1; - base_unsigned y0_mask, y1_mask; - calc_params(n, range, w0, n0, y0, y1, y0_mask, y1_mask); - if(base_unsigned(range - y0 + 1) > y0 / n) { - // increment n and try again. - ++n; - calc_params(n, range, w0, n0, y0, y1, y0_mask, y1_mask); - } - - BOOST_ASSERT(n0*w0 + (n - n0)*(w0 + 1) == w); - - result_type S = 0; - for(std::size_t k = 0; k < n0; ++k) { - base_unsigned u; - do { - u = detail::subtract()(_base(), (_base.min)()); - } while(u > base_unsigned(y0 - 1)); - S = (S << w0) + (u & y0_mask); - } - for(std::size_t k = 0; k < (n - n0); ++k) { - base_unsigned u; - do { - u = detail::subtract()(_base(), (_base.min)()); - } while(u > base_unsigned(y1 - 1)); - S = (S << (w0 + 1)) + (u & y1_mask); - } - return S; - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - for(boost::uintmax_t i = 0; i < z; ++i) { - (*this)(); - } - } - - const base_type& base() const { return _base; } - - /** - * Writes the textual representation if the generator to a @c std::ostream. - * The textual representation of the engine is the textual representation - * of the base engine. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, independent_bits_engine, r) - { - os << r._base; - return os; - } - - /** - * Reads the state of an @c independent_bits_engine from a - * @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, independent_bits_engine, r) - { - is >> r._base; - return is; - } - - /** - * Returns: true iff the two @c independent_bits_engines will - * produce the same sequence of values. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(independent_bits_engine, x, y) - { return x._base == y._base; } - /** - * Returns: true iff the two @c independent_bits_engines will - * produce different sequences of values. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(independent_bits_engine) - -private: - - /// \cond show_private - typedef typename boost::random::traits::make_unsigned::type base_unsigned; - - static UIntType max_imp(const boost::true_type&) - { - return boost::low_bits_mask_t::sig_bits; - } - static UIntType max_imp(const boost::false_type&) - { - // We have a multiprecision integer type: - BOOST_STATIC_ASSERT(std::numeric_limits::is_specialized); - return w < std::numeric_limits::digits ? UIntType((UIntType(1) << w) - 1) : UIntType((((UIntType(1) << (w - 1)) - 1) << 1) | 1u); - } - - void calc_params( - std::size_t n, base_unsigned range, - std::size_t& w0, std::size_t& n0, - base_unsigned& y0, base_unsigned& y1, - base_unsigned& y0_mask, base_unsigned& y1_mask) - { - BOOST_ASSERT(w >= n); - w0 = w/n; - n0 = n - w % n; - y0_mask = (base_unsigned(2) << (w0 - 1)) - 1; - y1_mask = (y0_mask << 1) | 1; - y0 = (range + 1) & ~y0_mask; - y1 = (range + 1) & ~y1_mask; - BOOST_ASSERT(y0 != 0 || base_unsigned(range + 1) == 0); - } - /// \endcond - - Engine _base; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -template -const bool independent_bits_engine::has_fixed_range; -#endif - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_INDEPENDENT_BITS_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/inversive_congruential.hpp b/contrib/libboost/boost_1_65_0/boost/random/inversive_congruential.hpp deleted file mode 100644 index 2329c0db3a9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/inversive_congruential.hpp +++ /dev/null @@ -1,267 +0,0 @@ -/* boost random/inversive_congruential.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_INVERSIVE_CONGRUENTIAL_HPP -#define BOOST_RANDOM_INVERSIVE_CONGRUENTIAL_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { - -// Eichenauer and Lehn 1986 -/** - * Instantiations of class template @c inversive_congruential_engine model a - * \pseudo_random_number_generator. It uses the inversive congruential - * algorithm (ICG) described in - * - * @blockquote - * "Inversive pseudorandom number generators: concepts, results and links", - * Peter Hellekalek, In: "Proceedings of the 1995 Winter Simulation - * Conference", C. Alexopoulos, K. Kang, W.R. Lilegdon, and D. Goldsman - * (editors), 1995, pp. 255-262. ftp://random.mat.sbg.ac.at/pub/data/wsc95.ps - * @endblockquote - * - * The output sequence is defined by x(n+1) = (a*inv(x(n)) - b) (mod p), - * where x(0), a, b, and the prime number p are parameters of the generator. - * The expression inv(k) denotes the multiplicative inverse of k in the - * field of integer numbers modulo p, with inv(0) := 0. - * - * The template parameter IntType shall denote a signed integral type large - * enough to hold p; a, b, and p are the parameters of the generators. The - * template parameter val is the validation value checked by validation. - * - * @xmlnote - * The implementation currently uses the Euclidian Algorithm to compute - * the multiplicative inverse. Therefore, the inversive generators are about - * 10-20 times slower than the others (see section"performance"). However, - * the paper talks of only 3x slowdown, so the Euclidian Algorithm is probably - * not optimal for calculating the multiplicative inverse. - * @endxmlnote - */ -template -class inversive_congruential_engine -{ -public: - typedef IntType result_type; - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - - BOOST_STATIC_CONSTANT(result_type, multiplier = a); - BOOST_STATIC_CONSTANT(result_type, increment = b); - BOOST_STATIC_CONSTANT(result_type, modulus = p); - BOOST_STATIC_CONSTANT(IntType, default_seed = 1); - - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () { return b == 0 ? 1 : 0; } - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () { return p-1; } - - /** - * Constructs an @c inversive_congruential_engine, seeding it with - * the default seed. - */ - inversive_congruential_engine() { seed(); } - - /** - * Constructs an @c inversive_congruential_engine, seeding it with @c x0. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(inversive_congruential_engine, - IntType, x0) - { seed(x0); } - - /** - * Constructs an @c inversive_congruential_engine, seeding it with values - * produced by a call to @c seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(inversive_congruential_engine, - SeedSeq, seq) - { seed(seq); } - - /** - * Constructs an @c inversive_congruential_engine, seeds it - * with values taken from the itrator range [first, last), - * and adjusts first to point to the element after the last one - * used. If there are not enough elements, throws @c std::invalid_argument. - * - * first and last must be input iterators. - */ - template inversive_congruential_engine(It& first, It last) - { seed(first, last); } - - /** - * Calls seed(default_seed) - */ - void seed() { seed(default_seed); } - - /** - * If c mod m is zero and x0 mod m is zero, changes the current value of - * the generator to 1. Otherwise, changes it to x0 mod m. If c is zero, - * distinct seeds in the range [1,m) will leave the generator in distinct - * states. If c is not zero, the range is [0,m). - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(inversive_congruential_engine, IntType, x0) - { - // wrap _x if it doesn't fit in the destination - if(modulus == 0) { - _value = x0; - } else { - _value = x0 % modulus; - } - // handle negative seeds - if(_value <= 0 && _value != 0) { - _value += modulus; - } - // adjust to the correct range - if(increment == 0 && _value == 0) { - _value = 1; - } - BOOST_ASSERT(_value >= (min)()); - BOOST_ASSERT(_value <= (max)()); - } - - /** - * Seeds an @c inversive_congruential_engine using values from a SeedSeq. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(inversive_congruential_engine, SeedSeq, seq) - { seed(detail::seed_one_int(seq)); } - - /** - * seeds an @c inversive_congruential_engine with values taken - * from the itrator range [first, last) and adjusts @c first to - * point to the element after the last one used. If there are - * not enough elements, throws @c std::invalid_argument. - * - * @c first and @c last must be input iterators. - */ - template void seed(It& first, It last) - { seed(detail::get_one_int(first, last)); } - - /** Returns the next output of the generator. */ - IntType operator()() - { - typedef const_mod do_mod; - _value = do_mod::mult_add(a, do_mod::invert(_value), b); - return _value; - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - for(boost::uintmax_t j = 0; j < z; ++j) { - (*this)(); - } - } - - /** - * Writes the textual representation of the generator to a @c std::ostream. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, inversive_congruential_engine, x) - { - os << x._value; - return os; - } - - /** - * Reads the textual representation of the generator from a @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, inversive_congruential_engine, x) - { - is >> x._value; - return is; - } - - /** - * Returns true if the two generators will produce identical - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(inversive_congruential_engine, x, y) - { return x._value == y._value; } - - /** - * Returns true if the two generators will produce different - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(inversive_congruential_engine) - -private: - IntType _value; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool inversive_congruential_engine::has_fixed_range; -template -const typename inversive_congruential_engine::result_type inversive_congruential_engine::multiplier; -template -const typename inversive_congruential_engine::result_type inversive_congruential_engine::increment; -template -const typename inversive_congruential_engine::result_type inversive_congruential_engine::modulus; -template -const typename inversive_congruential_engine::result_type inversive_congruential_engine::default_seed; -#endif - -/// \cond show_deprecated - -// provided for backwards compatibility -template -class inversive_congruential : public inversive_congruential_engine -{ - typedef inversive_congruential_engine base_type; -public: - inversive_congruential(IntType x0 = 1) : base_type(x0) {} - template - inversive_congruential(It& first, It last) : base_type(first, last) {} -}; - -/// \endcond - -/** - * The specialization hellekalek1995 was suggested in - * - * @blockquote - * "Inversive pseudorandom number generators: concepts, results and links", - * Peter Hellekalek, In: "Proceedings of the 1995 Winter Simulation - * Conference", C. Alexopoulos, K. Kang, W.R. Lilegdon, and D. Goldsman - * (editors), 1995, pp. 255-262. ftp://random.mat.sbg.ac.at/pub/data/wsc95.ps - * @endblockquote - */ -typedef inversive_congruential_engine hellekalek1995; - -} // namespace random - -using random::hellekalek1995; - -} // namespace boost - -#include - -#endif // BOOST_RANDOM_INVERSIVE_CONGRUENTIAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/lagged_fibonacci.hpp b/contrib/libboost/boost_1_65_0/boost/random/lagged_fibonacci.hpp deleted file mode 100644 index 5a809f3e494..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/lagged_fibonacci.hpp +++ /dev/null @@ -1,537 +0,0 @@ -/* boost random/lagged_fibonacci.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2013-10-14 fixed some warnings with Wshadow (mgaunard) - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_LAGGED_FIBONACCI_HPP -#define BOOST_RANDOM_LAGGED_FIBONACCI_HPP - -#include -#include -#include // std::max -#include -#include // std::pow -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * Instantiations of class template \lagged_fibonacci_engine model a - * \pseudo_random_number_generator. It uses a lagged Fibonacci - * algorithm with two lags @c p and @c q: - * x(i) = x(i-p) + x(i-q) (mod 2w) with p > q. - */ -template -class lagged_fibonacci_engine -{ -public: - typedef UIntType result_type; - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - BOOST_STATIC_CONSTANT(int, word_size = w); - BOOST_STATIC_CONSTANT(unsigned int, long_lag = p); - BOOST_STATIC_CONSTANT(unsigned int, short_lag = q); - - BOOST_STATIC_CONSTANT(UIntType, default_seed = 331u); - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () { return 0; } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return low_bits_mask_t::sig_bits; } - - /** Creates a new @c lagged_fibonacci_engine and calls @c seed(). */ - lagged_fibonacci_engine() { seed(); } - - /** Creates a new @c lagged_fibonacci_engine and calls @c seed(value). */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(lagged_fibonacci_engine, - UIntType, value) - { seed(value); } - - /** Creates a new @c lagged_fibonacci_engine and calls @c seed(seq). */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(lagged_fibonacci_engine, - SeedSeq, seq) - { seed(seq); } - - /** - * Creates a new @c lagged_fibonacci_engine and calls @c seed(first, last). - */ - template lagged_fibonacci_engine(It& first, It last) - { seed(first, last); } - - // compiler-generated copy ctor and assignment operator are fine - - /** Calls @c seed(default_seed). */ - void seed() { seed(default_seed); } - - /** - * Sets the state of the generator to values produced by - * a \minstd_rand0 generator. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(lagged_fibonacci_engine, - UIntType, value) - { - minstd_rand0 intgen(static_cast(value)); - detail::generator_seed_seq gen(intgen); - seed(gen); - } - - /** - * Sets the state of the generator using values produced by seq. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(lagged_fibonacci_engine, SeedSeq, seq) - { - detail::seed_array_int(seq, x); - i = long_lag; - } - - /** - * Sets the state of the generator to values from the iterator - * range [first, last). If there are not enough elements in the - * range [first, last) throws @c std::invalid_argument. - */ - template - void seed(It& first, It last) - { - detail::fill_array_int(first, last, x); - i = long_lag; - } - - /** Returns the next value of the generator. */ - result_type operator()() - { - if(i >= long_lag) - fill(); - return x[i++]; - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - for(boost::uintmax_t j = 0; j < z; ++j) { - (*this)(); - } - } - - /** - * Writes the textual representation of the generator to a @c std::ostream. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, lagged_fibonacci_engine, f) - { - os << f.i; - for(unsigned int j = 0; j < f.long_lag; ++j) - os << ' ' << f.x[j]; - return os; - } - - /** - * Reads the textual representation of the generator from a @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, lagged_fibonacci_engine, f) - { - is >> f.i >> std::ws; - for(unsigned int j = 0; j < f.long_lag; ++j) - is >> f.x[j] >> std::ws; - return is; - } - - /** - * Returns true if the two generators will produce identical - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(lagged_fibonacci_engine, x_, y_) - { return x_.i == y_.i && std::equal(x_.x, x_.x+long_lag, y_.x); } - - /** - * Returns true if the two generators will produce different - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(lagged_fibonacci_engine) - -private: - /// \cond show_private - void fill(); - /// \endcond - - unsigned int i; - UIntType x[long_lag]; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool lagged_fibonacci_engine::has_fixed_range; -template -const unsigned int lagged_fibonacci_engine::long_lag; -template -const unsigned int lagged_fibonacci_engine::short_lag; -template -const UIntType lagged_fibonacci_engine::default_seed; -#endif - -/// \cond show_private - -template -void lagged_fibonacci_engine::fill() -{ - // two loops to avoid costly modulo operations - { // extra scope for MSVC brokenness w.r.t. for scope - for(unsigned int j = 0; j < short_lag; ++j) - x[j] = (x[j] + x[j+(long_lag-short_lag)]) & low_bits_mask_t::sig_bits; - } - for(unsigned int j = short_lag; j < long_lag; ++j) - x[j] = (x[j] + x[j-short_lag]) & low_bits_mask_t::sig_bits; - i = 0; -} - -/// \endcond - -/// \cond show_deprecated - -// provided for backwards compatibility -template -class lagged_fibonacci : public lagged_fibonacci_engine -{ - typedef lagged_fibonacci_engine base_type; -public: - lagged_fibonacci() {} - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(lagged_fibonacci, UIntType, val) - { this->seed(val); } - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(lagged_fibonacci, SeedSeq, seq) - { this->seed(seq); } - template - lagged_fibonacci(It& first, It last) : base_type(first, last) {} -}; - -/// \endcond - -// lagged Fibonacci generator for the range [0..1) -// contributed by Matthias Troyer -// for p=55, q=24 originally by G. J. Mitchell and D. P. Moore 1958 - -/** - * Instantiations of class template @c lagged_fibonacci_01 model a - * \pseudo_random_number_generator. It uses a lagged Fibonacci - * algorithm with two lags @c p and @c q, evaluated in floating-point - * arithmetic: x(i) = x(i-p) + x(i-q) (mod 1) with p > q. See - * - * @blockquote - * "Uniform random number generators for supercomputers", Richard Brent, - * Proc. of Fifth Australian Supercomputer Conference, Melbourne, - * Dec. 1992, pp. 704-706. - * @endblockquote - * - * @xmlnote - * The quality of the generator crucially depends on the choice - * of the parameters. User code should employ one of the sensibly - * parameterized generators such as \lagged_fibonacci607 instead. - * @endxmlnote - * - * The generator requires considerable amounts of memory for the storage - * of its state array. For example, \lagged_fibonacci607 requires about - * 4856 bytes and \lagged_fibonacci44497 requires about 350 KBytes. - */ -template -class lagged_fibonacci_01_engine -{ -public: - typedef RealType result_type; - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - BOOST_STATIC_CONSTANT(int, word_size = w); - BOOST_STATIC_CONSTANT(unsigned int, long_lag = p); - BOOST_STATIC_CONSTANT(unsigned int, short_lag = q); - - BOOST_STATIC_CONSTANT(boost::uint32_t, default_seed = 331u); - - /** Constructs a @c lagged_fibonacci_01 generator and calls @c seed(). */ - lagged_fibonacci_01_engine() { seed(); } - /** Constructs a @c lagged_fibonacci_01 generator and calls @c seed(value). */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(lagged_fibonacci_01_engine, uint32_t, value) - { seed(value); } - /** Constructs a @c lagged_fibonacci_01 generator and calls @c seed(gen). */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(lagged_fibonacci_01_engine, SeedSeq, seq) - { seed(seq); } - template lagged_fibonacci_01_engine(It& first, It last) - { seed(first, last); } - - // compiler-generated copy ctor and assignment operator are fine - - /** Calls seed(default_seed). */ - void seed() { seed(default_seed); } - - /** - * Constructs a \minstd_rand0 generator with the constructor parameter - * value and calls seed with it. Distinct seeds in the range - * [1, 2147483647) will produce generators with different states. Other - * seeds will be equivalent to some seed within this range. See - * \linear_congruential_engine for details. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(lagged_fibonacci_01_engine, boost::uint32_t, value) - { - minstd_rand0 intgen(value); - detail::generator_seed_seq gen(intgen); - seed(gen); - } - - /** - * Seeds this @c lagged_fibonacci_01_engine using values produced by - * @c seq.generate. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(lagged_fibonacci_01_engine, SeedSeq, seq) - { - detail::seed_array_real(seq, x); - i = long_lag; - } - - /** - * Seeds this @c lagged_fibonacci_01_engine using values from the - * iterator range [first, last). If there are not enough elements - * in the range, throws @c std::invalid_argument. - */ - template - void seed(It& first, It last) - { - detail::fill_array_real(first, last, x); - i = long_lag; - } - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () { return result_type(0); } - /** Returns the upper bound of the generators outputs. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () { return result_type(1); } - - /** Returns the next value of the generator. */ - result_type operator()() - { - if(i >= long_lag) - fill(); - return x[i++]; - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { return detail::generate_from_real(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - for(boost::uintmax_t j = 0; j < z; ++j) { - (*this)(); - } - } - - /** - * Writes the textual representation of the generator to a @c std::ostream. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, lagged_fibonacci_01_engine, f) - { - // allow for Koenig lookup - using std::pow; - os << f.i; - std::ios_base::fmtflags oldflags = os.flags(os.dec | os.fixed | os.left); - for(unsigned int j = 0; j < f.long_lag; ++j) - os << ' ' << f.x[j] * f.modulus(); - os.flags(oldflags); - return os; - } - - /** - * Reads the textual representation of the generator from a @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, lagged_fibonacci_01_engine, f) - { - is >> f.i; - for(unsigned int j = 0; j < f.long_lag; ++j) { - typename lagged_fibonacci_01_engine::result_type value; - is >> std::ws >> value; - f.x[j] = value / f.modulus(); - } - return is; - } - - /** - * Returns true if the two generators will produce identical - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(lagged_fibonacci_01_engine, x_, y_) - { return x_.i == y_.i && std::equal(x_.x, x_.x+long_lag, y_.x); } - - /** - * Returns true if the two generators will produce different - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(lagged_fibonacci_01_engine) - -private: - /// \cond show_private - void fill(); - static RealType modulus() - { - using std::pow; - return pow(RealType(2), word_size); - } - /// \endcond - unsigned int i; - RealType x[long_lag]; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool lagged_fibonacci_01_engine::has_fixed_range; -template -const unsigned int lagged_fibonacci_01_engine::long_lag; -template -const unsigned int lagged_fibonacci_01_engine::short_lag; -template -const int lagged_fibonacci_01_engine::word_size; -template -const boost::uint32_t lagged_fibonacci_01_engine::default_seed; -#endif - -/// \cond show_private -template -void lagged_fibonacci_01_engine::fill() -{ - // two loops to avoid costly modulo operations - { // extra scope for MSVC brokenness w.r.t. for scope - for(unsigned int j = 0; j < short_lag; ++j) { - RealType t = x[j] + x[j+(long_lag-short_lag)]; - if(t >= RealType(1)) - t -= RealType(1); - x[j] = t; - } - } - for(unsigned int j = short_lag; j < long_lag; ++j) { - RealType t = x[j] + x[j-short_lag]; - if(t >= RealType(1)) - t -= RealType(1); - x[j] = t; - } - i = 0; -} -/// \endcond - -/// \cond show_deprecated - -// provided for backwards compatibility -template -class lagged_fibonacci_01 : public lagged_fibonacci_01_engine -{ - typedef lagged_fibonacci_01_engine base_type; -public: - lagged_fibonacci_01() {} - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(lagged_fibonacci_01, boost::uint32_t, val) - { this->seed(val); } - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(lagged_fibonacci_01, SeedSeq, seq) - { this->seed(seq); } - template - lagged_fibonacci_01(It& first, It last) : base_type(first, last) {} -}; - -/// \endcond - -namespace detail { - -template -struct generator_bits; - -template -struct generator_bits > -{ - static std::size_t value() { return w; } -}; - -template -struct generator_bits > -{ - static std::size_t value() { return w; } -}; - -} - -#ifdef BOOST_RANDOM_DOXYGEN -namespace detail { -/** - * The specializations lagged_fibonacci607 ... lagged_fibonacci44497 - * use well tested lags. - * - * See - * - * @blockquote - * "On the Periods of Generalized Fibonacci Recurrences", Richard P. Brent - * Computer Sciences Laboratory Australian National University, December 1992 - * @endblockquote - * - * The lags used here can be found in - * - * @blockquote - * "Uniform random number generators for supercomputers", Richard Brent, - * Proc. of Fifth Australian Supercomputer Conference, Melbourne, - * Dec. 1992, pp. 704-706. - * @endblockquote - */ -struct lagged_fibonacci_doc {}; -} -#endif - -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci607; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci1279; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci2281; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci3217; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci4423; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci9689; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci19937; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci23209; -/** @copydoc boost::random::detail::lagged_fibonacci_doc */ -typedef lagged_fibonacci_01_engine lagged_fibonacci44497; - -} // namespace random - -using random::lagged_fibonacci607; -using random::lagged_fibonacci1279; -using random::lagged_fibonacci2281; -using random::lagged_fibonacci3217; -using random::lagged_fibonacci4423; -using random::lagged_fibonacci9689; -using random::lagged_fibonacci19937; -using random::lagged_fibonacci23209; -using random::lagged_fibonacci44497; - -} // namespace boost - -#endif // BOOST_RANDOM_LAGGED_FIBONACCI_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/laplace_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/laplace_distribution.hpp deleted file mode 100644 index ec176a59c1f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/laplace_distribution.hpp +++ /dev/null @@ -1,175 +0,0 @@ -/* boost random/laplace_distribution.hpp header file - * - * Copyright Steven Watanabe 2014 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_LAPLACE_DISTRIBUTION_HPP -#define BOOST_RANDOM_LAPLACE_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The laplace distribution is a real-valued distribution with - * two parameters, mean and beta. - * - * It has \f$\displaystyle p(x) = \frac{e^-{\frac{|x-\mu|}{\beta}}}{2\beta}\f$. - */ -template -class laplace_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef laplace_distribution distribution_type; - - /** - * Constructs a @c param_type from the "mean" and "beta" parameters - * of the distribution. - */ - explicit param_type(RealType mean_arg = RealType(0.0), - RealType beta_arg = RealType(1.0)) - : _mean(mean_arg), _beta(beta_arg) - {} - - /** Returns the "mean" parameter of the distribtuion. */ - RealType mean() const { return _mean; } - /** Returns the "beta" parameter of the distribution. */ - RealType beta() const { return _beta; } - - /** Writes a @c param_type to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { os << parm._mean << ' ' << parm._beta; return os; } - - /** Reads a @c param_type from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { is >> parm._mean >> std::ws >> parm._beta; return is; } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._mean == rhs._mean && lhs._beta == rhs._beta; } - - /** Returns true if the two sets of parameters are the different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _mean; - RealType _beta; - }; - - /** - * Constructs an @c laplace_distribution from its "mean" and "beta" parameters. - */ - explicit laplace_distribution(RealType mean_arg = RealType(0.0), - RealType beta_arg = RealType(1.0)) - : _mean(mean_arg), _beta(beta_arg) - {} - /** Constructs an @c laplace_distribution from its parameters. */ - explicit laplace_distribution(const param_type& parm) - : _mean(parm.mean()), _beta(parm.beta()) - {} - - /** - * Returns a random variate distributed according to the - * laplace distribution. - */ - template - RealType operator()(URNG& urng) const - { - RealType exponential = exponential_distribution()(urng); - if(uniform_01()(urng) < 0.5) - exponential = -exponential; - return _mean + _beta * exponential; - } - - /** - * Returns a random variate distributed accordint to the laplace - * distribution with parameters specified by @c param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return laplace_distribution(parm)(urng); - } - - /** Returns the "mean" parameter of the distribution. */ - RealType mean() const { return _mean; } - /** Returns the "beta" parameter of the distribution. */ - RealType beta() const { return _beta; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return RealType(-std::numeric_limits::infinity()); } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return RealType(std::numeric_limits::infinity()); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_mean, _beta); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _mean = parm.mean(); - _beta = parm.beta(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Writes an @c laplace_distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, laplace_distribution, wd) - { - os << wd.param(); - return os; - } - - /** Reads an @c laplace_distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, laplace_distribution, wd) - { - param_type parm; - if(is >> parm) { - wd.param(parm); - } - return is; - } - - /** - * Returns true if the two instances of @c laplace_distribution will - * return identical sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(laplace_distribution, lhs, rhs) - { return lhs._mean == rhs._mean && lhs._beta == rhs._beta; } - - /** - * Returns true if the two instances of @c laplace_distribution will - * return different sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(laplace_distribution) - -private: - RealType _mean; - RealType _beta; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_LAPLACE_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/linear_congruential.hpp b/contrib/libboost/boost_1_65_0/boost/random/linear_congruential.hpp deleted file mode 100644 index de3a1d07491..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/linear_congruential.hpp +++ /dev/null @@ -1,466 +0,0 @@ -/* boost random/linear_congruential.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_LINEAR_CONGRUENTIAL_HPP -#define BOOST_RANDOM_LINEAR_CONGRUENTIAL_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { - -/** - * Instantiations of class template linear_congruential_engine model a - * \pseudo_random_number_generator. Linear congruential pseudo-random - * number generators are described in: - * - * @blockquote - * "Mathematical methods in large-scale computing units", D. H. Lehmer, - * Proc. 2nd Symposium on Large-Scale Digital Calculating Machines, - * Harvard University Press, 1951, pp. 141-146 - * @endblockquote - * - * Let x(n) denote the sequence of numbers returned by some pseudo-random - * number generator. Then for the linear congruential generator, - * x(n+1) := (a * x(n) + c) mod m. Parameters for the generator are - * x(0), a, c, m. The template parameter IntType shall denote an integral - * type. It must be large enough to hold values a, c, and m. The template - * parameters a and c must be smaller than m. - * - * Note: The quality of the generator crucially depends on the choice of - * the parameters. User code should use one of the sensibly parameterized - * generators such as minstd_rand instead. - */ -template -class linear_congruential_engine -{ -public: - typedef IntType result_type; - - // Required for old Boost.Random concept - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - - BOOST_STATIC_CONSTANT(IntType, multiplier = a); - BOOST_STATIC_CONSTANT(IntType, increment = c); - BOOST_STATIC_CONSTANT(IntType, modulus = m); - BOOST_STATIC_CONSTANT(IntType, default_seed = 1); - - BOOST_STATIC_ASSERT(std::numeric_limits::is_integer); - BOOST_STATIC_ASSERT(m == 0 || a < m); - BOOST_STATIC_ASSERT(m == 0 || c < m); - - /** - * Constructs a @c linear_congruential_engine, using the default seed - */ - linear_congruential_engine() { seed(); } - - /** - * Constructs a @c linear_congruential_engine, seeding it with @c x0. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(linear_congruential_engine, - IntType, x0) - { seed(x0); } - - /** - * Constructs a @c linear_congruential_engine, seeding it with values - * produced by a call to @c seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(linear_congruential_engine, - SeedSeq, seq) - { seed(seq); } - - /** - * Constructs a @c linear_congruential_engine and seeds it - * with values taken from the itrator range [first, last) - * and adjusts first to point to the element after the last one - * used. If there are not enough elements, throws @c std::invalid_argument. - * - * first and last must be input iterators. - */ - template - linear_congruential_engine(It& first, It last) - { - seed(first, last); - } - - // compiler-generated copy constructor and assignment operator are fine - - /** - * Calls seed(default_seed) - */ - void seed() { seed(default_seed); } - - /** - * If c mod m is zero and x0 mod m is zero, changes the current value of - * the generator to 1. Otherwise, changes it to x0 mod m. If c is zero, - * distinct seeds in the range [1,m) will leave the generator in distinct - * states. If c is not zero, the range is [0,m). - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(linear_congruential_engine, IntType, x0) - { - // wrap _x if it doesn't fit in the destination - if(modulus == 0) { - _x = x0; - } else { - _x = x0 % modulus; - } - // handle negative seeds - if(_x <= 0 && _x != 0) { - _x += modulus; - } - // adjust to the correct range - if(increment == 0 && _x == 0) { - _x = 1; - } - BOOST_ASSERT(_x >= (min)()); - BOOST_ASSERT(_x <= (max)()); - } - - /** - * Seeds a @c linear_congruential_engine using values from a SeedSeq. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(linear_congruential_engine, SeedSeq, seq) - { seed(detail::seed_one_int(seq)); } - - /** - * seeds a @c linear_congruential_engine with values taken - * from the itrator range [first, last) and adjusts @c first to - * point to the element after the last one used. If there are - * not enough elements, throws @c std::invalid_argument. - * - * @c first and @c last must be input iterators. - */ - template - void seed(It& first, It last) - { seed(detail::get_one_int(first, last)); } - - /** - * Returns the smallest value that the @c linear_congruential_engine - * can produce. - */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return c == 0 ? 1 : 0; } - /** - * Returns the largest value that the @c linear_congruential_engine - * can produce. - */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return modulus-1; } - - /** Returns the next value of the @c linear_congruential_engine. */ - IntType operator()() - { - _x = const_mod::mult_add(a, _x, c); - return _x; - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - typedef const_mod mod_type; - IntType b_inv = mod_type::invert(a-1); - IntType b_gcd = mod_type::mult(a-1, b_inv); - if(b_gcd == 1) { - IntType a_z = mod_type::pow(a, z); - _x = mod_type::mult_add(a_z, _x, - mod_type::mult(mod_type::mult(c, b_inv), a_z - 1)); - } else { - // compute (a^z - 1)*c % (b_gcd * m) / (b / b_gcd) * inv(b / b_gcd) - // we're storing the intermediate result / b_gcd - IntType a_zm1_over_gcd = 0; - IntType a_km1_over_gcd = (a - 1) / b_gcd; - boost::uintmax_t exponent = z; - while(exponent != 0) { - if(exponent % 2 == 1) { - a_zm1_over_gcd = - mod_type::mult_add( - b_gcd, - mod_type::mult(a_zm1_over_gcd, a_km1_over_gcd), - mod_type::add(a_zm1_over_gcd, a_km1_over_gcd)); - } - a_km1_over_gcd = mod_type::mult_add( - b_gcd, - mod_type::mult(a_km1_over_gcd, a_km1_over_gcd), - mod_type::add(a_km1_over_gcd, a_km1_over_gcd)); - exponent /= 2; - } - - IntType a_z = mod_type::mult_add(b_gcd, a_zm1_over_gcd, 1); - IntType num = mod_type::mult(c, a_zm1_over_gcd); - b_inv = mod_type::invert((a-1)/b_gcd); - _x = mod_type::mult_add(a_z, _x, mod_type::mult(b_inv, num)); - } - } - - friend bool operator==(const linear_congruential_engine& x, - const linear_congruential_engine& y) - { return x._x == y._x; } - friend bool operator!=(const linear_congruential_engine& x, - const linear_congruential_engine& y) - { return !(x == y); } - -#if !defined(BOOST_RANDOM_NO_STREAM_OPERATORS) - /** Writes a @c linear_congruential_engine to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const linear_congruential_engine& lcg) - { - return os << lcg._x; - } - - /** Reads a @c linear_congruential_engine from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, - linear_congruential_engine& lcg) - { - lcg.read(is); - return is; - } -#endif - -private: - - /// \cond show_private - - template - void read(std::basic_istream& is) { - IntType x; - if(is >> x) { - if(x >= (min)() && x <= (max)()) { - _x = x; - } else { - is.setstate(std::ios_base::failbit); - } - } - } - - /// \endcond - - IntType _x; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool linear_congruential_engine::has_fixed_range; -template -const IntType linear_congruential_engine::multiplier; -template -const IntType linear_congruential_engine::increment; -template -const IntType linear_congruential_engine::modulus; -template -const IntType linear_congruential_engine::default_seed; -#endif - -/// \cond show_deprecated - -// provided for backwards compatibility -template -class linear_congruential : public linear_congruential_engine -{ - typedef linear_congruential_engine base_type; -public: - linear_congruential(IntType x0 = 1) : base_type(x0) {} - template - linear_congruential(It& first, It last) : base_type(first, last) {} -}; - -/// \endcond - -/** - * The specialization \minstd_rand0 was originally suggested in - * - * @blockquote - * A pseudo-random number generator for the System/360, P.A. Lewis, - * A.S. Goodman, J.M. Miller, IBM Systems Journal, Vol. 8, No. 2, - * 1969, pp. 136-146 - * @endblockquote - * - * It is examined more closely together with \minstd_rand in - * - * @blockquote - * "Random Number Generators: Good ones are hard to find", - * Stephen K. Park and Keith W. Miller, Communications of - * the ACM, Vol. 31, No. 10, October 1988, pp. 1192-1201 - * @endblockquote - */ -typedef linear_congruential_engine minstd_rand0; - -/** The specialization \minstd_rand was suggested in - * - * @blockquote - * "Random Number Generators: Good ones are hard to find", - * Stephen K. Park and Keith W. Miller, Communications of - * the ACM, Vol. 31, No. 10, October 1988, pp. 1192-1201 - * @endblockquote - */ -typedef linear_congruential_engine minstd_rand; - - -#if !defined(BOOST_NO_INT64_T) && !defined(BOOST_NO_INTEGRAL_INT64_T) -/** - * Class @c rand48 models a \pseudo_random_number_generator. It uses - * the linear congruential algorithm with the parameters a = 0x5DEECE66D, - * c = 0xB, m = 2**48. It delivers identical results to the @c lrand48() - * function available on some systems (assuming lcong48 has not been called). - * - * It is only available on systems where @c uint64_t is provided as an - * integral type, so that for example static in-class constants and/or - * enum definitions with large @c uint64_t numbers work. - */ -class rand48 -{ -public: - typedef boost::uint32_t result_type; - - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - /** - * Returns the smallest value that the generator can produce - */ - static uint32_t min BOOST_PREVENT_MACRO_SUBSTITUTION () { return 0; } - /** - * Returns the largest value that the generator can produce - */ - static uint32_t max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return 0x7FFFFFFF; } - - /** Seeds the generator with the default seed. */ - rand48() : lcf(cnv(static_cast(1))) {} - /** - * Constructs a \rand48 generator with x(0) := (x0 << 16) | 0x330e. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(rand48, result_type, x0) - { seed(x0); } - /** - * Seeds the generator with values produced by @c seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(rand48, SeedSeq, seq) - { seed(seq); } - /** - * Seeds the generator using values from an iterator range, - * and updates first to point one past the last value consumed. - */ - template rand48(It& first, It last) : lcf(first, last) { } - - // compiler-generated copy ctor and assignment operator are fine - - /** Seeds the generator with the default seed. */ - void seed() { seed(static_cast(1)); } - /** - * Changes the current value x(n) of the generator to (x0 << 16) | 0x330e. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(rand48, result_type, x0) - { lcf.seed(cnv(x0)); } - /** - * Seeds the generator using values from an iterator range, - * and updates first to point one past the last value consumed. - */ - template void seed(It& first, It last) { lcf.seed(first,last); } - /** - * Seeds the generator with values produced by @c seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(rand48, SeedSeq, seq) - { lcf.seed(seq); } - - /** Returns the next value of the generator. */ - uint32_t operator()() { return static_cast(lcf() >> 17); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) { lcf.discard(z); } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { - for(; first != last; ++first) { - *first = (*this)(); - } - } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes a @c rand48 to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, const rand48& r) - { os << r.lcf; return os; } - - /** Reads a @c rand48 from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, rand48& r) - { is >> r.lcf; return is; } -#endif - - /** - * Returns true if the two generators will produce identical - * sequences of values. - */ - friend bool operator==(const rand48& x, const rand48& y) - { return x.lcf == y.lcf; } - /** - * Returns true if the two generators will produce different - * sequences of values. - */ - friend bool operator!=(const rand48& x, const rand48& y) - { return !(x == y); } -private: - /// \cond show_private - typedef random::linear_congruential_engine lcf_t; - lcf_t lcf; - - static boost::uint64_t cnv(boost::uint32_t x) - { return (static_cast(x) << 16) | 0x330e; } - /// \endcond -}; -#endif /* !BOOST_NO_INT64_T && !BOOST_NO_INTEGRAL_INT64_T */ - -} // namespace random - -using random::minstd_rand0; -using random::minstd_rand; -using random::rand48; - -} // namespace boost - -#include - -#endif // BOOST_RANDOM_LINEAR_CONGRUENTIAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/linear_feedback_shift.hpp b/contrib/libboost/boost_1_65_0/boost/random/linear_feedback_shift.hpp deleted file mode 100644 index a695dfde97f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/linear_feedback_shift.hpp +++ /dev/null @@ -1,217 +0,0 @@ -/* boost random/linear_feedback_shift.hpp header file - * - * Copyright Jens Maurer 2002 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_LINEAR_FEEDBACK_SHIFT_HPP -#define BOOST_RANDOM_LINEAR_FEEDBACK_SHIFT_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * Instatiations of @c linear_feedback_shift model a - * \pseudo_random_number_generator. It was originally - * proposed in - * - * @blockquote - * "Random numbers generated by linear recurrence modulo two.", - * Tausworthe, R. C.(1965), Mathematics of Computation 19, 201-209. - * @endblockquote - */ -template -class linear_feedback_shift_engine -{ -public: - typedef UIntType result_type; - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - BOOST_STATIC_CONSTANT(int, word_size = w); - BOOST_STATIC_CONSTANT(int, exponent1 = k); - BOOST_STATIC_CONSTANT(int, exponent2 = q); - BOOST_STATIC_CONSTANT(int, step_size = s); - BOOST_STATIC_CONSTANT(UIntType, default_seed = 341); - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () { return 0; } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return wordmask(); } - - BOOST_STATIC_ASSERT(w > 0); - BOOST_STATIC_ASSERT(q > 0); - BOOST_STATIC_ASSERT(k < w); - BOOST_STATIC_ASSERT(0 < 2*q && 2*q < k); - BOOST_STATIC_ASSERT(0 < s && s <= k-q); - - /** Constructs a @c linear_feedback_shift_engine, using the default seed. */ - linear_feedback_shift_engine() { seed(); } - - /** Constructs a @c linear_feedback_shift_engine, seeding it with s0. */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(linear_feedback_shift_engine, - UIntType, s0) - { seed(s0); } - - /** Constructs a @c linear_feedback_shift_engine, seeding it with seq. */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(linear_feedback_shift_engine, - SeedSeq, seq) - { seed(seq); } - - /** - * Constructs a @c linear_feedback_shift_engine, seeding it with - * values from the range [first, last). - */ - template linear_feedback_shift_engine(It& first, It last) - { seed(first, last); } - - /** Seeds a @c linear_feedback_shift_engine with the default seed. */ - void seed() { seed(default_seed); } - - /** Seeds a @c linear_feedback_shift_engine with @c s0. */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(linear_feedback_shift_engine, - UIntType, s0) - { - value = s0 & wordmask(); - if(value < (1 << (w-k))) { - value += 1 << (w-k); - } - } - - /** - * Seeds a @c linear_feedback_shift_engine with values - * produced by @c seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(linear_feedback_shift_engine, - SeedSeq, seq) - { seed(detail::seed_one_int(seq)); } - - /** - * Seeds a @c linear_feedback_shift_engine with values - * from the range [first, last). - */ - template void seed(It& first, It last) - { - seed(detail::get_one_int(first, last)); - } - - /** Returns the next value of the generator. */ - result_type operator()() - { - const UIntType b = (((value << q) ^ value) & wordmask()) >> (k-s); - const UIntType mask = (wordmask() << (w-k)) & wordmask(); - value = ((value & mask) << s) ^ b; - return value; - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - for(boost::uintmax_t j = 0; j < z; ++j) { - (*this)(); - } - } - - /** - * Writes the textual representation of the generator to a @c std::ostream. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, linear_feedback_shift_engine, x) - { - os << x.value; - return os; - } - - /** - * Reads the textual representation of the generator from a @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, linear_feedback_shift_engine, x) - { - is >> x.value; - return is; - } - - /** - * Returns true if the two generators will produce identical - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(linear_feedback_shift_engine, x, y) - { return x.value == y.value; } - - /** - * Returns true if the two generators will produce different - * sequences of outputs. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(linear_feedback_shift_engine) - -private: - /// \cond show_private - static UIntType wordmask() { return boost::low_bits_mask_t::sig_bits; } - /// \endcond - UIntType value; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool linear_feedback_shift_engine::has_fixed_range; -template -const int linear_feedback_shift_engine::word_size; -template -const int linear_feedback_shift_engine::exponent1; -template -const int linear_feedback_shift_engine::exponent2; -template -const int linear_feedback_shift_engine::step_size; -template -const UIntType linear_feedback_shift_engine::default_seed; -#endif - -/// \cond show_deprecated - -/** Provided for backwards compatibility. */ -template -class linear_feedback_shift : - public linear_feedback_shift_engine -{ - typedef linear_feedback_shift_engine base_type; -public: - linear_feedback_shift() {} - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(linear_feedback_shift, - SeedSeq, seq) - { seed(seq); } - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(linear_feedback_shift, - UIntType, val) - { seed(val); } - template - linear_feedback_shift(It& first, It last) : base_type(first, last) {} -}; - -/// \endcond - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_LINEAR_FEEDBACK_SHIFT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/lognormal_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/lognormal_distribution.hpp deleted file mode 100644 index 7ed1e13abf4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/lognormal_distribution.hpp +++ /dev/null @@ -1,254 +0,0 @@ -/* boost random/lognormal_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_LOGNORMAL_DISTRIBUTION_HPP -#define BOOST_RANDOM_LOGNORMAL_DISTRIBUTION_HPP - -#include // std::exp, std::sqrt -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * Instantiations of class template lognormal_distribution model a - * \random_distribution. Such a distribution produces random numbers - * with \f$\displaystyle p(x) = \frac{1}{x s \sqrt{2\pi}} e^{\frac{-\left(\log(x)-m\right)^2}{2s^2}}\f$ - * for x > 0. - * - * @xmlwarning - * This distribution has been updated to match the C++ standard. - * Its behavior has changed from the original - * boost::lognormal_distribution. A backwards compatible - * version is provided in namespace boost. - * @endxmlwarning - */ -template -class lognormal_distribution -{ -public: - typedef typename normal_distribution::input_type input_type; - typedef RealType result_type; - - class param_type - { - public: - - typedef lognormal_distribution distribution_type; - - /** Constructs the parameters of a lognormal_distribution. */ - explicit param_type(RealType m_arg = RealType(0.0), - RealType s_arg = RealType(1.0)) - : _m(m_arg), _s(s_arg) {} - - /** Returns the "m" parameter of the distribution. */ - RealType m() const { return _m; } - - /** Returns the "s" parameter of the distribution. */ - RealType s() const { return _s; } - - /** Writes the parameters to a std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._m << " " << parm._s; - return os; - } - - /** Reads the parameters from a std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - is >> parm._m >> std::ws >> parm._s; - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._m == rhs._m && lhs._s == rhs._s; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _m; - RealType _s; - }; - - /** - * Constructs a lognormal_distribution. @c m and @c s are the - * parameters of the distribution. - */ - explicit lognormal_distribution(RealType m_arg = RealType(0.0), - RealType s_arg = RealType(1.0)) - : _normal(m_arg, s_arg) {} - - /** - * Constructs a lognormal_distribution from its parameters. - */ - explicit lognormal_distribution(const param_type& parm) - : _normal(parm.m(), parm.s()) {} - - // compiler-generated copy ctor and assignment operator are fine - - /** Returns the m parameter of the distribution. */ - RealType m() const { return _normal.mean(); } - /** Returns the s parameter of the distribution. */ - RealType s() const { return _normal.sigma(); } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return RealType(0); } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return (std::numeric_limits::infinity)(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(m(), s()); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - typedef normal_distribution normal_type; - typename normal_type::param_type normal_param(parm.m(), parm.s()); - _normal.param(normal_param); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { _normal.reset(); } - - /** - * Returns a random variate distributed according to the - * lognormal distribution. - */ - template - result_type operator()(Engine& eng) - { - using std::exp; - return exp(_normal(eng)); - } - - /** - * Returns a random variate distributed according to the - * lognormal distribution with parameters specified by param. - */ - template - result_type operator()(Engine& eng, const param_type& parm) - { return lognormal_distribution(parm)(eng); } - - /** Writes the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, lognormal_distribution, ld) - { - os << ld._normal; - return os; - } - - /** Reads the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, lognormal_distribution, ld) - { - is >> ld._normal; - return is; - } - - /** - * Returns true if the two distributions will produce identical - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(lognormal_distribution, lhs, rhs) - { return lhs._normal == rhs._normal; } - - /** - * Returns true if the two distributions may produce different - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(lognormal_distribution) - -private: - normal_distribution _normal; -}; - -} // namespace random - -/// \cond show_deprecated - -/** - * Provided for backwards compatibility. This class is - * deprecated. It provides the old behavior of lognormal_distribution with - * \f$\displaystyle p(x) = \frac{1}{x \sigma_N \sqrt{2\pi}} e^{\frac{-\left(\log(x)-\mu_N\right)^2}{2\sigma_N^2}}\f$ - * for x > 0, where \f$\displaystyle \mu_N = \log\left(\frac{\mu^2}{\sqrt{\sigma^2 + \mu^2}}\right)\f$ and - * \f$\displaystyle \sigma_N = \sqrt{\log\left(1 + \frac{\sigma^2}{\mu^2}\right)}\f$. - */ -template -class lognormal_distribution -{ -public: - typedef typename normal_distribution::input_type input_type; - typedef RealType result_type; - - lognormal_distribution(RealType mean_arg = RealType(1.0), - RealType sigma_arg = RealType(1.0)) - : _mean(mean_arg), _sigma(sigma_arg) - { - init(); - } - RealType mean() const { return _mean; } - RealType sigma() const { return _sigma; } - void reset() { _normal.reset(); } - template - RealType operator()(Engine& eng) - { - using std::exp; - return exp(_normal(eng) * _nsigma + _nmean); - } - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, lognormal_distribution, ld) - { - os << ld._normal << " " << ld._mean << " " << ld._sigma; - return os; - } - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, lognormal_distribution, ld) - { - is >> ld._normal >> std::ws >> ld._mean >> std::ws >> ld._sigma; - ld.init(); - return is; - } -private: - /// \cond show_private - void init() - { - using std::log; - using std::sqrt; - _nmean = log(_mean*_mean/sqrt(_sigma*_sigma + _mean*_mean)); - _nsigma = sqrt(log(_sigma*_sigma/_mean/_mean+result_type(1))); - } - RealType _mean; - RealType _sigma; - RealType _nmean; - RealType _nsigma; - normal_distribution _normal; - /// \endcond -}; - -/// \endcond - -} // namespace boost - -#endif // BOOST_RANDOM_LOGNORMAL_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/mersenne_twister.hpp b/contrib/libboost/boost_1_65_0/boost/random/mersenne_twister.hpp deleted file mode 100644 index ce73e6825f3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/mersenne_twister.hpp +++ /dev/null @@ -1,682 +0,0 @@ -/* boost random/mersenne_twister.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2013-10-14 fixed some warnings with Wshadow (mgaunard) - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_MERSENNE_TWISTER_HPP -#define BOOST_RANDOM_MERSENNE_TWISTER_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { - -/** - * Instantiations of class template mersenne_twister_engine model a - * \pseudo_random_number_generator. It uses the algorithm described in - * - * @blockquote - * "Mersenne Twister: A 623-dimensionally equidistributed uniform - * pseudo-random number generator", Makoto Matsumoto and Takuji Nishimura, - * ACM Transactions on Modeling and Computer Simulation: Special Issue on - * Uniform Random Number Generation, Vol. 8, No. 1, January 1998, pp. 3-30. - * @endblockquote - * - * @xmlnote - * The boost variant has been implemented from scratch and does not - * derive from or use mt19937.c provided on the above WWW site. However, it - * was verified that both produce identical output. - * @endxmlnote - * - * The seeding from an integer was changed in April 2005 to address a - * weakness. - * - * The quality of the generator crucially depends on the choice of the - * parameters. User code should employ one of the sensibly parameterized - * generators such as \mt19937 instead. - * - * The generator requires considerable amounts of memory for the storage of - * its state array. For example, \mt11213b requires about 1408 bytes and - * \mt19937 requires about 2496 bytes. - */ -template -class mersenne_twister_engine -{ -public: - typedef UIntType result_type; - BOOST_STATIC_CONSTANT(std::size_t, word_size = w); - BOOST_STATIC_CONSTANT(std::size_t, state_size = n); - BOOST_STATIC_CONSTANT(std::size_t, shift_size = m); - BOOST_STATIC_CONSTANT(std::size_t, mask_bits = r); - BOOST_STATIC_CONSTANT(UIntType, xor_mask = a); - BOOST_STATIC_CONSTANT(std::size_t, tempering_u = u); - BOOST_STATIC_CONSTANT(UIntType, tempering_d = d); - BOOST_STATIC_CONSTANT(std::size_t, tempering_s = s); - BOOST_STATIC_CONSTANT(UIntType, tempering_b = b); - BOOST_STATIC_CONSTANT(std::size_t, tempering_t = t); - BOOST_STATIC_CONSTANT(UIntType, tempering_c = c); - BOOST_STATIC_CONSTANT(std::size_t, tempering_l = l); - BOOST_STATIC_CONSTANT(UIntType, initialization_multiplier = f); - BOOST_STATIC_CONSTANT(UIntType, default_seed = 5489u); - - // backwards compatibility - BOOST_STATIC_CONSTANT(UIntType, parameter_a = a); - BOOST_STATIC_CONSTANT(std::size_t, output_u = u); - BOOST_STATIC_CONSTANT(std::size_t, output_s = s); - BOOST_STATIC_CONSTANT(UIntType, output_b = b); - BOOST_STATIC_CONSTANT(std::size_t, output_t = t); - BOOST_STATIC_CONSTANT(UIntType, output_c = c); - BOOST_STATIC_CONSTANT(std::size_t, output_l = l); - - // old Boost.Random concept requirements - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - - - /** - * Constructs a @c mersenne_twister_engine and calls @c seed(). - */ - mersenne_twister_engine() { seed(); } - - /** - * Constructs a @c mersenne_twister_engine and calls @c seed(value). - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(mersenne_twister_engine, - UIntType, value) - { seed(value); } - template mersenne_twister_engine(It& first, It last) - { seed(first,last); } - - /** - * Constructs a mersenne_twister_engine and calls @c seed(gen). - * - * @xmlnote - * The copy constructor will always be preferred over - * the templated constructor. - * @endxmlnote - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(mersenne_twister_engine, - SeedSeq, seq) - { seed(seq); } - - // compiler-generated copy ctor and assignment operator are fine - - /** Calls @c seed(default_seed). */ - void seed() { seed(default_seed); } - - /** - * Sets the state x(0) to v mod 2w. Then, iteratively, - * sets x(i) to - * (i + f * (x(i-1) xor (x(i-1) rshift w-2))) mod 2w - * for i = 1 .. n-1. x(n) is the first value to be returned by operator(). - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(mersenne_twister_engine, UIntType, value) - { - // New seeding algorithm from - // http://www.math.sci.hiroshima-u.ac.jp/~m-mat/MT/MT2002/emt19937ar.html - // In the previous versions, MSBs of the seed affected only MSBs of the - // state x[]. - const UIntType mask = (max)(); - x[0] = value & mask; - for (i = 1; i < n; i++) { - // See Knuth "The Art of Computer Programming" - // Vol. 2, 3rd ed., page 106 - x[i] = (f * (x[i-1] ^ (x[i-1] >> (w-2))) + i) & mask; - } - - normalize_state(); - } - - /** - * Seeds a mersenne_twister_engine using values produced by seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(mersenne_twister_engine, SeeqSeq, seq) - { - detail::seed_array_int(seq, x); - i = n; - - normalize_state(); - } - - /** Sets the state of the generator using values from an iterator range. */ - template - void seed(It& first, It last) - { - detail::fill_array_int(first, last, x); - i = n; - - normalize_state(); - } - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return 0; } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return boost::low_bits_mask_t::sig_bits; } - - /** Produces the next value of the generator. */ - result_type operator()(); - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** - * Advances the state of the generator by @c z steps. Equivalent to - * - * @code - * for(unsigned long long i = 0; i < z; ++i) { - * gen(); - * } - * @endcode - */ - void discard(boost::uintmax_t z) - { -#ifndef BOOST_RANDOM_MERSENNE_TWISTER_DISCARD_THRESHOLD -#define BOOST_RANDOM_MERSENNE_TWISTER_DISCARD_THRESHOLD 10000000 -#endif - if(z > BOOST_RANDOM_MERSENNE_TWISTER_DISCARD_THRESHOLD) { - discard_many(z); - } else { - for(boost::uintmax_t j = 0; j < z; ++j) { - (*this)(); - } - } - } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes a mersenne_twister_engine to a @c std::ostream */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const mersenne_twister_engine& mt) - { - mt.print(os); - return os; - } - - /** Reads a mersenne_twister_engine from a @c std::istream */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, - mersenne_twister_engine& mt) - { - for(std::size_t j = 0; j < mt.state_size; ++j) - is >> mt.x[j] >> std::ws; - // MSVC (up to 7.1) and Borland (up to 5.64) don't handle the template - // value parameter "n" available from the class template scope, so use - // the static constant with the same value - mt.i = mt.state_size; - return is; - } -#endif - - /** - * Returns true if the two generators are in the same state, - * and will thus produce identical sequences. - */ - friend bool operator==(const mersenne_twister_engine& x_, - const mersenne_twister_engine& y_) - { - if(x_.i < y_.i) return x_.equal_imp(y_); - else return y_.equal_imp(x_); - } - - /** - * Returns true if the two generators are in different states. - */ - friend bool operator!=(const mersenne_twister_engine& x_, - const mersenne_twister_engine& y_) - { return !(x_ == y_); } - -private: - /// \cond show_private - - void twist(); - - /** - * Does the work of operator==. This is in a member function - * for portability. Some compilers, such as msvc 7.1 and - * Sun CC 5.10 can't access template parameters or static - * members of the class from inline friend functions. - * - * requires i <= other.i - */ - bool equal_imp(const mersenne_twister_engine& other) const - { - UIntType back[n]; - std::size_t offset = other.i - i; - for(std::size_t j = 0; j + offset < n; ++j) - if(x[j] != other.x[j+offset]) - return false; - rewind(&back[n-1], offset); - for(std::size_t j = 0; j < offset; ++j) - if(back[j + n - offset] != other.x[j]) - return false; - return true; - } - - /** - * Does the work of operator<<. This is in a member function - * for portability. - */ - template - void print(std::basic_ostream& os) const - { - UIntType data[n]; - for(std::size_t j = 0; j < i; ++j) { - data[j + n - i] = x[j]; - } - if(i != n) { - rewind(&data[n - i - 1], n - i); - } - os << data[0]; - for(std::size_t j = 1; j < n; ++j) { - os << ' ' << data[j]; - } - } - - /** - * Copies z elements of the state preceding x[0] into - * the array whose last element is last. - */ - void rewind(UIntType* last, std::size_t z) const - { - const UIntType upper_mask = (~static_cast(0)) << r; - const UIntType lower_mask = ~upper_mask; - UIntType y0 = x[m-1] ^ x[n-1]; - if(y0 & (static_cast(1) << (w-1))) { - y0 = ((y0 ^ a) << 1) | 1; - } else { - y0 = y0 << 1; - } - for(std::size_t sz = 0; sz < z; ++sz) { - UIntType y1 = - rewind_find(last, sz, m-1) ^ rewind_find(last, sz, n-1); - if(y1 & (static_cast(1) << (w-1))) { - y1 = ((y1 ^ a) << 1) | 1; - } else { - y1 = y1 << 1; - } - *(last - sz) = (y0 & upper_mask) | (y1 & lower_mask); - y0 = y1; - } - } - - /** - * Converts an arbitrary array into a valid generator state. - * First we normalize x[0], so that it contains the same - * value we would get by running the generator forwards - * and then in reverse. (The low order r bits are redundant). - * Then, if the state consists of all zeros, we set the - * high order bit of x[0] to 1. This function only needs to - * be called by seed, since the state transform preserves - * this relationship. - */ - void normalize_state() - { - const UIntType upper_mask = (~static_cast(0)) << r; - const UIntType lower_mask = ~upper_mask; - UIntType y0 = x[m-1] ^ x[n-1]; - if(y0 & (static_cast(1) << (w-1))) { - y0 = ((y0 ^ a) << 1) | 1; - } else { - y0 = y0 << 1; - } - x[0] = (x[0] & upper_mask) | (y0 & lower_mask); - - // fix up the state if it's all zeroes. - for(std::size_t j = 0; j < n; ++j) { - if(x[j] != 0) return; - } - x[0] = static_cast(1) << (w-1); - } - - /** - * Given a pointer to the last element of the rewind array, - * and the current size of the rewind array, finds an element - * relative to the next available slot in the rewind array. - */ - UIntType - rewind_find(UIntType* last, std::size_t size, std::size_t j) const - { - std::size_t index = (j + n - size + n - 1) % n; - if(index < n - size) { - return x[index]; - } else { - return *(last - (n - 1 - index)); - } - } - - /** - * Optimized algorithm for large jumps. - * - * Hiroshi Haramoto, Makoto Matsumoto, and Pierre L'Ecuyer. 2008. - * A Fast Jump Ahead Algorithm for Linear Recurrences in a Polynomial - * Space. In Proceedings of the 5th international conference on - * Sequences and Their Applications (SETA '08). - * DOI=10.1007/978-3-540-85912-3_26 - */ - void discard_many(boost::uintmax_t z) - { - // Compute the minimal polynomial, phi(t) - // This depends only on the transition function, - // which is constant. The characteristic - // polynomial is the same as the minimal - // polynomial for a maximum period generator - // (which should be all specializations of - // mersenne_twister.) Even if it weren't, - // the characteristic polynomial is guaranteed - // to be a multiple of the minimal polynomial, - // which is good enough. - detail::polynomial phi = get_characteristic_polynomial(); - - // calculate g(t) = t^z % phi(t) - detail::polynomial g = mod_pow_x(z, phi); - - // h(s_0, t) = \sum_{i=0}^{2k-1}o(s_i)t^{2k-i-1} - detail::polynomial h; - const std::size_t num_bits = w*n - r; - for(std::size_t j = 0; j < num_bits * 2; ++j) { - // Yes, we're advancing the generator state - // here, but it doesn't matter because - // we're going to overwrite it completely - // in reconstruct_state. - if(i >= n) twist(); - h[2*num_bits - j - 1] = x[i++] & UIntType(1); - } - // g(t)h(s_0, t) - detail::polynomial gh = g * h; - detail::polynomial result; - for(std::size_t j = 0; j <= num_bits; ++j) { - result[j] = gh[2*num_bits - j - 1]; - } - reconstruct_state(result); - } - static detail::polynomial get_characteristic_polynomial() - { - const std::size_t num_bits = w*n - r; - detail::polynomial helper; - helper[num_bits - 1] = 1; - mersenne_twister_engine tmp; - tmp.reconstruct_state(helper); - // Skip the first num_bits elements, since we - // already know what they are. - for(std::size_t j = 0; j < num_bits; ++j) { - if(tmp.i >= n) tmp.twist(); - if(j == num_bits - 1) - assert((tmp.x[tmp.i] & 1) == 1); - else - assert((tmp.x[tmp.i] & 1) == 0); - ++tmp.i; - } - detail::polynomial phi; - phi[num_bits] = 1; - detail::polynomial next_bits = tmp.as_polynomial(num_bits); - for(std::size_t j = 0; j < num_bits; ++j) { - int val = next_bits[j] ^ phi[num_bits-j-1]; - phi[num_bits-j-1] = val; - if(val) { - for(std::size_t k = j + 1; k < num_bits; ++k) { - phi[num_bits-k-1] ^= next_bits[k-j-1]; - } - } - } - return phi; - } - detail::polynomial as_polynomial(std::size_t size) { - detail::polynomial result; - for(std::size_t j = 0; j < size; ++j) { - if(i >= n) twist(); - result[j] = x[i++] & UIntType(1); - } - return result; - } - void reconstruct_state(const detail::polynomial& p) - { - const UIntType upper_mask = (~static_cast(0)) << r; - const UIntType lower_mask = ~upper_mask; - const std::size_t num_bits = w*n - r; - for(std::size_t j = num_bits - n + 1; j <= num_bits; ++j) - x[j % n] = p[j]; - - UIntType y0 = 0; - for(std::size_t j = num_bits + 1; j >= n - 1; --j) { - UIntType y1 = x[j % n] ^ x[(j + m) % n]; - if(p[j - n + 1]) - y1 = (y1 ^ a) << UIntType(1) | UIntType(1); - else - y1 = y1 << UIntType(1); - x[(j + 1) % n] = (y0 & upper_mask) | (y1 & lower_mask); - y0 = y1; - } - i = 0; - } - - /// \endcond - - // state representation: next output is o(x(i)) - // x[0] ... x[k] x[k+1] ... x[n-1] represents - // x(i-k) ... x(i) x(i+1) ... x(i-k+n-1) - - UIntType x[n]; - std::size_t i; -}; - -/// \cond show_private - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -#define BOOST_RANDOM_MT_DEFINE_CONSTANT(type, name) \ -template \ -const type mersenne_twister_engine::name -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, word_size); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, state_size); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, shift_size); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, mask_bits); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, xor_mask); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, tempering_u); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, tempering_d); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, tempering_s); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, tempering_b); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, tempering_t); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, tempering_c); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, tempering_l); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, initialization_multiplier); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, default_seed); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, parameter_a); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, output_u ); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, output_s); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, output_b); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, output_t); -BOOST_RANDOM_MT_DEFINE_CONSTANT(UIntType, output_c); -BOOST_RANDOM_MT_DEFINE_CONSTANT(std::size_t, output_l); -BOOST_RANDOM_MT_DEFINE_CONSTANT(bool, has_fixed_range); -#undef BOOST_RANDOM_MT_DEFINE_CONSTANT -#endif - -template -void -mersenne_twister_engine::twist() -{ - const UIntType upper_mask = (~static_cast(0)) << r; - const UIntType lower_mask = ~upper_mask; - - const std::size_t unroll_factor = 6; - const std::size_t unroll_extra1 = (n-m) % unroll_factor; - const std::size_t unroll_extra2 = (m-1) % unroll_factor; - - // split loop to avoid costly modulo operations - { // extra scope for MSVC brokenness w.r.t. for scope - for(std::size_t j = 0; j < n-m-unroll_extra1; j++) { - UIntType y = (x[j] & upper_mask) | (x[j+1] & lower_mask); - x[j] = x[j+m] ^ (y >> 1) ^ ((x[j+1]&1) * a); - } - } - { - for(std::size_t j = n-m-unroll_extra1; j < n-m; j++) { - UIntType y = (x[j] & upper_mask) | (x[j+1] & lower_mask); - x[j] = x[j+m] ^ (y >> 1) ^ ((x[j+1]&1) * a); - } - } - { - for(std::size_t j = n-m; j < n-1-unroll_extra2; j++) { - UIntType y = (x[j] & upper_mask) | (x[j+1] & lower_mask); - x[j] = x[j-(n-m)] ^ (y >> 1) ^ ((x[j+1]&1) * a); - } - } - { - for(std::size_t j = n-1-unroll_extra2; j < n-1; j++) { - UIntType y = (x[j] & upper_mask) | (x[j+1] & lower_mask); - x[j] = x[j-(n-m)] ^ (y >> 1) ^ ((x[j+1]&1) * a); - } - } - // last iteration - UIntType y = (x[n-1] & upper_mask) | (x[0] & lower_mask); - x[n-1] = x[m-1] ^ (y >> 1) ^ ((x[0]&1) * a); - i = 0; -} -/// \endcond - -template -inline typename -mersenne_twister_engine::result_type -mersenne_twister_engine::operator()() -{ - if(i == n) - twist(); - // Step 4 - UIntType z = x[i]; - ++i; - z ^= ((z >> u) & d); - z ^= ((z << s) & b); - z ^= ((z << t) & c); - z ^= (z >> l); - return z; -} - -/** - * The specializations \mt11213b and \mt19937 are from - * - * @blockquote - * "Mersenne Twister: A 623-dimensionally equidistributed - * uniform pseudo-random number generator", Makoto Matsumoto - * and Takuji Nishimura, ACM Transactions on Modeling and - * Computer Simulation: Special Issue on Uniform Random Number - * Generation, Vol. 8, No. 1, January 1998, pp. 3-30. - * @endblockquote - */ -typedef mersenne_twister_engine mt11213b; - -/** - * The specializations \mt11213b and \mt19937 are from - * - * @blockquote - * "Mersenne Twister: A 623-dimensionally equidistributed - * uniform pseudo-random number generator", Makoto Matsumoto - * and Takuji Nishimura, ACM Transactions on Modeling and - * Computer Simulation: Special Issue on Uniform Random Number - * Generation, Vol. 8, No. 1, January 1998, pp. 3-30. - * @endblockquote - */ -typedef mersenne_twister_engine mt19937; - -#if !defined(BOOST_NO_INT64_T) && !defined(BOOST_NO_INTEGRAL_INT64_T) -typedef mersenne_twister_engine mt19937_64; -#endif - -/// \cond show_deprecated - -template -class mersenne_twister : - public mersenne_twister_engine -{ - typedef mersenne_twister_engine base_type; -public: - mersenne_twister() {} - BOOST_RANDOM_DETAIL_GENERATOR_CONSTRUCTOR(mersenne_twister, Gen, gen) - { seed(gen); } - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(mersenne_twister, UIntType, val) - { seed(val); } - template - mersenne_twister(It& first, It last) : base_type(first, last) {} - void seed() { base_type::seed(); } - BOOST_RANDOM_DETAIL_GENERATOR_SEED(mersenne_twister, Gen, gen) - { - detail::generator_seed_seq seq(gen); - base_type::seed(seq); - } - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(mersenne_twister, UIntType, val) - { base_type::seed(val); } - template - void seed(It& first, It last) { base_type::seed(first, last); } -}; - -/// \endcond - -} // namespace random - -using random::mt11213b; -using random::mt19937; -using random::mt19937_64; - -} // namespace boost - -BOOST_RANDOM_PTR_HELPER_SPEC(boost::mt11213b) -BOOST_RANDOM_PTR_HELPER_SPEC(boost::mt19937) -BOOST_RANDOM_PTR_HELPER_SPEC(boost::mt19937_64) - -#include - -#endif // BOOST_RANDOM_MERSENNE_TWISTER_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/negative_binomial_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/negative_binomial_distribution.hpp deleted file mode 100644 index cbe26fad25c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/negative_binomial_distribution.hpp +++ /dev/null @@ -1,220 +0,0 @@ -/* boost random/negative_binomial_distribution.hpp header file - * - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_NEGATIVE_BINOMIAL_DISTRIBUTION_HPP_INCLUDED -#define BOOST_RANDOM_NEGATIVE_BINOMIAL_DISTRIBUTION_HPP_INCLUDED - -#include - -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The negative binomial distribution is an integer valued - * distribution with two parameters, @c k and @c p. The - * distribution produces non-negative values. - * - * The distribution function is - * \f$\displaystyle P(i) = {k+i-1\choose i}p^k(1-p)^i\f$. - * - * This implementation uses a gamma-poisson mixture. - */ -template -class negative_binomial_distribution { -public: - typedef IntType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef negative_binomial_distribution distribution_type; - /** - * Construct a param_type object. @c k and @c p - * are the parameters of the distribution. - * - * Requires: k >=0 && 0 <= p <= 1 - */ - explicit param_type(IntType k_arg = 1, RealType p_arg = RealType (0.5)) - : _k(k_arg), _p(p_arg) - {} - /** Returns the @c k parameter of the distribution. */ - IntType k() const { return _k; } - /** Returns the @c p parameter of the distribution. */ - RealType p() const { return _p; } -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const param_type& parm) - { - os << parm._p << " " << parm._k; - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, param_type& parm) - { - is >> parm._p >> std::ws >> parm._k; - return is; - } -#endif - /** Returns true if the parameters have the same values. */ - friend bool operator==(const param_type& lhs, const param_type& rhs) - { - return lhs._k == rhs._k && lhs._p == rhs._p; - } - /** Returns true if the parameters have different values. */ - friend bool operator!=(const param_type& lhs, const param_type& rhs) - { - return !(lhs == rhs); - } - private: - IntType _k; - RealType _p; - }; - - /** - * Construct a @c negative_binomial_distribution object. @c k and @c p - * are the parameters of the distribution. - * - * Requires: k >=0 && 0 <= p <= 1 - */ - explicit negative_binomial_distribution(IntType k_arg = 1, - RealType p_arg = RealType(0.5)) - : _k(k_arg), _p(p_arg) - {} - - /** - * Construct an @c negative_binomial_distribution object from the - * parameters. - */ - explicit negative_binomial_distribution(const param_type& parm) - : _k(parm.k()), _p(parm.p()) - {} - - /** - * Returns a random variate distributed according to the - * negative binomial distribution. - */ - template - IntType operator()(URNG& urng) const - { - gamma_distribution gamma(_k, (1-_p)/_p); - poisson_distribution poisson(gamma(urng)); - return poisson(urng); - } - - /** - * Returns a random variate distributed according to the negative - * binomial distribution with parameters specified by @c param. - */ - template - IntType operator()(URNG& urng, const param_type& parm) const - { - return negative_binomial_distribution(parm)(urng); - } - - /** Returns the @c k parameter of the distribution. */ - IntType k() const { return _k; } - /** Returns the @c p parameter of the distribution. */ - RealType p() const { return _p; } - - /** Returns the smallest value that the distribution can produce. */ - IntType min BOOST_PREVENT_MACRO_SUBSTITUTION() const { return 0; } - /** Returns the largest value that the distribution can produce. */ - IntType max BOOST_PREVENT_MACRO_SUBSTITUTION() const - { return (std::numeric_limits::max)(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_k, _p); } - /** Sets parameters of the distribution. */ - void param(const param_type& parm) - { - _k = parm.k(); - _p = parm.p(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const negative_binomial_distribution& bd) - { - os << bd.param(); - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, - negative_binomial_distribution& bd) - { - bd.read(is); - return is; - } -#endif - - /** Returns true if the two distributions will produce the same - sequence of values, given equal generators. */ - friend bool operator==(const negative_binomial_distribution& lhs, - const negative_binomial_distribution& rhs) - { - return lhs._k == rhs._k && lhs._p == rhs._p; - } - /** Returns true if the two distributions could produce different - sequences of values, given equal generators. */ - friend bool operator!=(const negative_binomial_distribution& lhs, - const negative_binomial_distribution& rhs) - { - return !(lhs == rhs); - } - -private: - - /// @cond \show_private - - template - void read(std::basic_istream& is) { - param_type parm; - if(is >> parm) { - param(parm); - } - } - - // parameters - IntType _k; - RealType _p; - - /// @endcond -}; - -} - -} - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/non_central_chi_squared_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/non_central_chi_squared_distribution.hpp deleted file mode 100644 index 28c9ff6d9a4..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/non_central_chi_squared_distribution.hpp +++ /dev/null @@ -1,221 +0,0 @@ -/* boost random/non_central_chi_squared_distribution.hpp header file - * - * Copyright Thijs van den Berg 2014 - * - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_NON_CENTRAL_CHI_SQUARED_DISTRIBUTION_HPP -#define BOOST_RANDOM_NON_CENTRAL_CHI_SQUARED_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The noncentral chi-squared distribution is a real valued distribution with - * two parameter, @c k and @c lambda. The distribution produces values > 0. - * - * This is the distribution of the sum of squares of k Normal distributed - * variates each with variance one and \f$\lambda\f$ the sum of squares of the - * normal means. - * - * The distribution function is - * \f$\displaystyle P(x) = \frac{1}{2} e^{-(x+\lambda)/2} \left( \frac{x}{\lambda} \right)^{k/4-1/2} I_{k/2-1}( \sqrt{\lambda x} )\f$. - * where \f$\displaystyle I_\nu(z)\f$ is a modified Bessel function of the - * first kind. - * - * The algorithm is taken from - * - * @blockquote - * "Monte Carlo Methods in Financial Engineering", Paul Glasserman, - * 2003, XIII, 596 p, Stochastic Modelling and Applied Probability, Vol. 53, - * ISBN 978-0-387-21617-1, p 124, Fig. 3.5. - * @endblockquote - */ -template -class non_central_chi_squared_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef non_central_chi_squared_distribution distribution_type; - - /** - * Constructs the parameters of a non_central_chi_squared_distribution. - * @c k and @c lambda are the parameter of the distribution. - * - * Requires: k > 0 && lambda > 0 - */ - explicit - param_type(RealType k_arg = RealType(1), RealType lambda_arg = RealType(1)) - : _k(k_arg), _lambda(lambda_arg) - { - BOOST_ASSERT(k_arg > RealType(0)); - BOOST_ASSERT(lambda_arg > RealType(0)); - } - - /** Returns the @c k parameter of the distribution */ - RealType k() const { return _k; } - - /** Returns the @c lambda parameter of the distribution */ - RealType lambda() const { return _lambda; } - - /** Writes the parameters of the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._k << ' ' << parm._lambda; - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - is >> parm._k >> std::ws >> parm._lambda; - return is; - } - - /** Returns true if the parameters have the same values. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._k == rhs._k && lhs._lambda == rhs._lambda; } - - /** Returns true if the parameters have different values. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _k; - RealType _lambda; - }; - - /** - * Construct a @c non_central_chi_squared_distribution object. @c k and - * @c lambda are the parameter of the distribution. - * - * Requires: k > 0 && lambda > 0 - */ - explicit - non_central_chi_squared_distribution(RealType k_arg = RealType(1), RealType lambda_arg = RealType(1)) - : _param(k_arg, lambda_arg) - { - BOOST_ASSERT(k_arg > RealType(0)); - BOOST_ASSERT(lambda_arg > RealType(0)); - } - - /** - * Construct a @c non_central_chi_squared_distribution object from the parameter. - */ - explicit - non_central_chi_squared_distribution(const param_type& parm) - : _param( parm ) - { } - - /** - * Returns a random variate distributed according to the - * non central chi squared distribution specified by @c param. - */ - template - RealType operator()(URNG& eng, const param_type& parm) const - { return non_central_chi_squared_distribution(parm)(eng); } - - /** - * Returns a random variate distributed according to the - * non central chi squared distribution. - */ - template - RealType operator()(URNG& eng) - { - using std::sqrt; - if (_param.k() > 1) { - boost::random::normal_distribution n_dist; - boost::random::chi_squared_distribution c_dist(_param.k() - RealType(1)); - RealType _z = n_dist(eng); - RealType _x = c_dist(eng); - RealType term1 = _z + sqrt(_param.lambda()); - return term1*term1 + _x; - } - else { - boost::random::poisson_distribution<> p_dist(_param.lambda()/RealType(2)); - boost::random::poisson_distribution<>::result_type _p = p_dist(eng); - boost::random::chi_squared_distribution c_dist(_param.k() + RealType(2)*_p); - return c_dist(eng); - } - } - - /** Returns the @c k parameter of the distribution. */ - RealType k() const { return _param.k(); } - - /** Returns the @c lambda parameter of the distribution. */ - RealType lambda() const { return _param.lambda(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return _param; } - - /** Sets parameters of the distribution. */ - void param(const param_type& parm) { _param = parm; } - - /** Resets the distribution, so that subsequent uses does not depend on values already produced by it.*/ - void reset() {} - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION() const - { return RealType(0); } - - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION() const - { return (std::numeric_limits::infinity)(); } - - /** Writes the parameters of the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, non_central_chi_squared_distribution, dist) - { - os << dist.param(); - return os; - } - - /** reads the parameters of the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, non_central_chi_squared_distribution, dist) - { - param_type parm; - if(is >> parm) { - dist.param(parm); - } - return is; - } - - /** Returns true if two distributions have the same parameters and produce - the same sequence of random numbers given equal generators.*/ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(non_central_chi_squared_distribution, lhs, rhs) - { return lhs.param() == rhs.param(); } - - /** Returns true if two distributions have different parameters and/or can produce - different sequences of random numbers given equal generators.*/ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(non_central_chi_squared_distribution) - -private: - - /// @cond show_private - param_type _param; - /// @endcond -}; - -} // namespace random -} // namespace boost - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/normal_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/normal_distribution.hpp deleted file mode 100644 index b7ff3eba9b6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/normal_distribution.hpp +++ /dev/null @@ -1,374 +0,0 @@ -/* boost random/normal_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2010-2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_NORMAL_DISTRIBUTION_HPP -#define BOOST_RANDOM_NORMAL_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -namespace detail { - -// tables for the ziggurat algorithm -template -struct normal_table { - static const RealType table_x[129]; - static const RealType table_y[129]; -}; - -template -const RealType normal_table::table_x[129] = { - 3.7130862467403632609, 3.4426198558966521214, 3.2230849845786185446, 3.0832288582142137009, - 2.9786962526450169606, 2.8943440070186706210, 2.8231253505459664379, 2.7611693723841538514, - 2.7061135731187223371, 2.6564064112581924999, 2.6109722484286132035, 2.5690336259216391328, - 2.5300096723854666170, 2.4934545220919507609, 2.4590181774083500943, 2.4264206455302115930, - 2.3954342780074673425, 2.3658713701139875435, 2.3375752413355307354, 2.3104136836950021558, - 2.2842740596736568056, 2.2590595738653295251, 2.2346863955870569803, 2.2110814088747278106, - 2.1881804320720206093, 2.1659267937448407377, 2.1442701823562613518, 2.1231657086697899595, - 2.1025731351849988838, 2.0824562379877246441, 2.0627822745039633575, 2.0435215366506694976, - 2.0246469733729338782, 2.0061338699589668403, 1.9879595741230607243, 1.9701032608497132242, - 1.9525457295488889058, 1.9352692282919002011, 1.9182573008597320303, 1.9014946531003176140, - 1.8849670357028692380, 1.8686611409895420085, 1.8525645117230870617, 1.8366654602533840447, - 1.8209529965910050740, 1.8054167642140487420, 1.7900469825946189862, 1.7748343955807692457, - 1.7597702248942318749, 1.7448461281083765085, 1.7300541605582435350, 1.7153867407081165482, - 1.7008366185643009437, 1.6863968467734863258, 1.6720607540918522072, 1.6578219209482075462, - 1.6436741568569826489, 1.6296114794646783962, 1.6156280950371329644, 1.6017183802152770587, - 1.5878768648844007019, 1.5740982160167497219, 1.5603772223598406870, 1.5467087798535034608, - 1.5330878776675560787, 1.5195095847593707806, 1.5059690368565502602, 1.4924614237746154081, - 1.4789819769830978546, 1.4655259573357946276, 1.4520886428822164926, 1.4386653166774613138, - 1.4252512545068615734, 1.4118417124397602509, 1.3984319141236063517, 1.3850170377251486449, - 1.3715922024197322698, 1.3581524543224228739, 1.3446927517457130432, 1.3312079496576765017, - 1.3176927832013429910, 1.3041418501204215390, 1.2905495919178731508, 1.2769102735516997175, - 1.2632179614460282310, 1.2494664995643337480, 1.2356494832544811749, 1.2217602305309625678, - 1.2077917504067576028, 1.1937367078237721994, 1.1795873846544607035, 1.1653356361550469083, - 1.1509728421389760651, 1.1364898520030755352, 1.1218769225722540661, 1.1071236475235353980, - 1.0922188768965537614, 1.0771506248819376573, 1.0619059636836193998, 1.0464709007525802629, - 1.0308302360564555907, 1.0149673952392994716, 0.99886423348064351303, 0.98250080350276038481, - 0.96585507938813059489, 0.94890262549791195381, 0.93161619660135381056, 0.91396525100880177644, - 0.89591535256623852894, 0.87742742909771569142, 0.85845684317805086354, 0.83895221428120745572, - 0.81885390668331772331, 0.79809206062627480454, 0.77658398787614838598, 0.75423066443451007146, - 0.73091191062188128150, 0.70647961131360803456, 0.68074791864590421664, 0.65347863871504238702, - 0.62435859730908822111, 0.59296294244197797913, 0.55869217837551797140, 0.52065603872514491759, - 0.47743783725378787681, 0.42654798630330512490, 0.36287143102841830424, 0.27232086470466385065, - 0 -}; - -template -const RealType normal_table::table_y[129] = { - 0, 0.0026696290839025035092, 0.0055489952208164705392, 0.0086244844129304709682, - 0.011839478657982313715, 0.015167298010672042468, 0.018592102737165812650, 0.022103304616111592615, - 0.025693291936149616572, 0.029356317440253829618, 0.033087886146505155566, 0.036884388786968774128, - 0.040742868074790604632, 0.044660862200872429800, 0.048636295860284051878, 0.052667401903503169793, - 0.056752663481538584188, 0.060890770348566375972, 0.065080585213631873753, 0.069321117394180252601, - 0.073611501884754893389, 0.077950982514654714188, 0.082338898242957408243, 0.086774671895542968998, - 0.091257800827634710201, 0.09578784912257815216, 0.10036444102954554013, 0.10498725541035453978, - 0.10965602101581776100, 0.11437051244988827452, 0.11913054670871858767, 0.12393598020398174246, - 0.12878670619710396109, 0.13368265258464764118, 0.13862377998585103702, 0.14361008009193299469, - 0.14864157424369696566, 0.15371831220958657066, 0.15884037114093507813, 0.16400785468492774791, - 0.16922089223892475176, 0.17447963833240232295, 0.17978427212496211424, 0.18513499701071343216, - 0.19053204032091372112, 0.19597565311811041399, 0.20146611007620324118, 0.20700370944187380064, - 0.21258877307373610060, 0.21822164655637059599, 0.22390269938713388747, 0.22963232523430270355, - 0.23541094226572765600, 0.24123899354775131610, 0.24711694751469673582, 0.25304529850976585934, - 0.25902456739871074263, 0.26505530225816194029, 0.27113807914102527343, 0.27727350292189771153, - 0.28346220822601251779, 0.28970486044581049771, 0.29600215684985583659, 0.30235482778947976274, - 0.30876363800925192282, 0.31522938806815752222, 0.32175291587920862031, 0.32833509837615239609, - 0.33497685331697116147, 0.34167914123501368412, 0.34844296754987246935, 0.35526938485154714435, - 0.36215949537303321162, 0.36911445366827513952, 0.37613546951445442947, 0.38322381105988364587, - 0.39038080824138948916, 0.39760785649804255208, 0.40490642081148835099, 0.41227804010702462062, - 0.41972433205403823467, 0.42724699830956239880, 0.43484783025466189638, 0.44252871528024661483, - 0.45029164368692696086, 0.45813871627287196483, 0.46607215269457097924, 0.47409430069824960453, - 0.48220764633483869062, 0.49041482528932163741, 0.49871863547658432422, 0.50712205108130458951, - 0.51562823824987205196, 0.52424057267899279809, 0.53296265938998758838, 0.54179835503172412311, - 0.55075179312105527738, 0.55982741271069481791, 0.56902999107472161225, 0.57836468112670231279, - 0.58783705444182052571, 0.59745315095181228217, 0.60721953663260488551, 0.61714337082656248870, - 0.62723248525781456578, 0.63749547734314487428, 0.64794182111855080873, 0.65858200005865368016, - 0.66942766735770616891, 0.68049184100641433355, 0.69178914344603585279, 0.70333609902581741633, - 0.71515150742047704368, 0.72725691835450587793, 0.73967724368333814856, 0.75244155918570380145, - 0.76558417390923599480, 0.77914608594170316563, 0.79317701178385921053, 0.80773829469612111340, - 0.82290721139526200050, 0.83878360531064722379, 0.85550060788506428418, 0.87324304892685358879, - 0.89228165080230272301, 0.91304364799203805999, 0.93628268170837107547, 0.96359969315576759960, - 1 -}; - - -template -struct unit_normal_distribution -{ - template - RealType operator()(Engine& eng) { - const double * const table_x = normal_table::table_x; - const double * const table_y = normal_table::table_y; - for(;;) { - std::pair vals = generate_int_float_pair(eng); - int i = vals.second; - int sign = (i & 1) * 2 - 1; - i = i >> 1; - RealType x = vals.first * RealType(table_x[i]); - if(x < table_x[i + 1]) return x * sign; - if(i == 0) return generate_tail(eng) * sign; - - RealType y01 = uniform_01()(eng); - RealType y = RealType(table_y[i]) + y01 * RealType(table_y[i + 1] - table_y[i]); - - // These store the value y - bound, or something proportional to that difference: - RealType y_above_ubound, y_above_lbound; - - // There are three cases to consider: - // - convex regions (where x[i] > x[j] >= 1) - // - concave regions (where 1 <= x[i] < x[j]) - // - region containing the inflection point (where x[i] > 1 > x[j]) - // For convex (concave), exp^(-x^2/2) is bounded below (above) by the tangent at - // (x[i],y[i]) and is bounded above (below) by the diagonal line from (x[i+1],y[i+1]) to - // (x[i],y[i]). - // - // *If* the inflection point region satisfies slope(x[i+1]) < slope(diagonal), then we - // can treat the inflection region as a convex region: this condition is necessary and - // sufficient to ensure that the curve lies entirely below the diagonal (that, in turn, - // also implies that it will be above the tangent at x[i]). - // - // For the current table size (128), this is satisfied: slope(x[i+1]) = -0.60653 < - // slope(diag) = -0.60649, and so we have only two cases below instead of three. - - if (table_x[i] >= 1) { // convex (incl. inflection) - y_above_ubound = RealType(table_x[i] - table_x[i+1]) * y01 - (RealType(table_x[i]) - x); - y_above_lbound = y - (RealType(table_y[i]) + (RealType(table_x[i]) - x) * RealType(table_y[i]) * RealType(table_x[i])); - } - else { // concave - y_above_lbound = RealType(table_x[i] - table_x[i+1]) * y01 - (RealType(table_x[i]) - x); - y_above_ubound = y - (RealType(table_y[i]) + (RealType(table_x[i]) - x) * RealType(table_y[i]) * RealType(table_x[i])); - } - - if (y_above_ubound < 0 // if above the upper bound reject immediately - && - ( - y_above_lbound < 0 // If below the lower bound accept immediately - || - y < f(x) // Otherwise it's between the bounds and we need a full check - ) - ) { - return x * sign; - } - } - } - static RealType f(RealType x) { - using std::exp; - return exp(-(x*x/2)); - } - // Generate from the tail using rejection sampling from the exponential(x_1) distribution, - // shifted by x_1. This looks a little different from the usual rejection sampling because it - // transforms the condition by taking the log of both sides, thus avoiding the costly exp() call - // on the RHS, then takes advantage of the fact that -log(unif01) is simply generating an - // exponential (by inverse cdf sampling) by replacing the log(unif01) on the LHS with a - // exponential(1) draw, y. - template - RealType generate_tail(Engine& eng) { - const RealType tail_start = RealType(normal_table::table_x[1]); - boost::random::exponential_distribution exp_x(tail_start); - boost::random::exponential_distribution exp_y; - for(;;) { - RealType x = exp_x(eng); - RealType y = exp_y(eng); - // If we were doing non-transformed rejection sampling, this condition would be: - // if (unif01 < exp(-.5*x*x)) return x + tail_start; - if(2*y > x*x) return x + tail_start; - } - } -}; - -} // namespace detail - - -/** - * Instantiations of class template normal_distribution model a - * \random_distribution. Such a distribution produces random numbers - * @c x distributed with probability density function - * \f$\displaystyle p(x) = - * \frac{1}{\sqrt{2\pi}\sigma} e^{-\frac{(x-\mu)^2}{2\sigma^2}} - * \f$, - * where mean and sigma are the parameters of the distribution. - * - * The implementation uses the "ziggurat" algorithm, as described in - * - * @blockquote - * "The Ziggurat Method for Generating Random Variables", - * George Marsaglia and Wai Wan Tsang, Journal of Statistical Software, - * Volume 5, Number 8 (2000), 1-7. - * @endblockquote - */ -template -class normal_distribution -{ -public: - typedef RealType input_type; - typedef RealType result_type; - - class param_type { - public: - typedef normal_distribution distribution_type; - - /** - * Constructs a @c param_type with a given mean and - * standard deviation. - * - * Requires: sigma >= 0 - */ - explicit param_type(RealType mean_arg = RealType(0.0), - RealType sigma_arg = RealType(1.0)) - : _mean(mean_arg), - _sigma(sigma_arg) - {} - - /** Returns the mean of the distribution. */ - RealType mean() const { return _mean; } - - /** Returns the standand deviation of the distribution. */ - RealType sigma() const { return _sigma; } - - /** Writes a @c param_type to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { os << parm._mean << " " << parm._sigma ; return os; } - - /** Reads a @c param_type from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { is >> parm._mean >> std::ws >> parm._sigma; return is; } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._mean == rhs._mean && lhs._sigma == rhs._sigma; } - - /** Returns true if the two sets of parameters are the different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _mean; - RealType _sigma; - }; - - /** - * Constructs a @c normal_distribution object. @c mean and @c sigma are - * the parameters for the distribution. - * - * Requires: sigma >= 0 - */ - explicit normal_distribution(const RealType& mean_arg = RealType(0.0), - const RealType& sigma_arg = RealType(1.0)) - : _mean(mean_arg), _sigma(sigma_arg) - { - BOOST_ASSERT(_sigma >= RealType(0)); - } - - /** - * Constructs a @c normal_distribution object from its parameters. - */ - explicit normal_distribution(const param_type& parm) - : _mean(parm.mean()), _sigma(parm.sigma()) - {} - - /** Returns the mean of the distribution. */ - RealType mean() const { return _mean; } - /** Returns the standard deviation of the distribution. */ - RealType sigma() const { return _sigma; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return -std::numeric_limits::infinity(); } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return std::numeric_limits::infinity(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_mean, _sigma); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _mean = parm.mean(); - _sigma = parm.sigma(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Returns a normal variate. */ - template - result_type operator()(Engine& eng) - { - detail::unit_normal_distribution impl; - return impl(eng) * _sigma + _mean; - } - - /** Returns a normal variate with parameters specified by @c param. */ - template - result_type operator()(URNG& urng, const param_type& parm) - { - return normal_distribution(parm)(urng); - } - - /** Writes a @c normal_distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, normal_distribution, nd) - { - os << nd._mean << " " << nd._sigma; - return os; - } - - /** Reads a @c normal_distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, normal_distribution, nd) - { - is >> std::ws >> nd._mean >> std::ws >> nd._sigma; - return is; - } - - /** - * Returns true if the two instances of @c normal_distribution will - * return identical sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(normal_distribution, lhs, rhs) - { - return lhs._mean == rhs._mean && lhs._sigma == rhs._sigma; - } - - /** - * Returns true if the two instances of @c normal_distribution will - * return different sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(normal_distribution) - -private: - RealType _mean, _sigma; - -}; - -} // namespace random - -using random::normal_distribution; - -} // namespace boost - -#endif // BOOST_RANDOM_NORMAL_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/piecewise_constant_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/piecewise_constant_distribution.hpp deleted file mode 100644 index 488f41c0e34..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/piecewise_constant_distribution.hpp +++ /dev/null @@ -1,466 +0,0 @@ -/* boost random/piecewise_constant_distribution.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_PIECEWISE_CONSTANT_DISTRIBUTION_HPP_INCLUDED -#define BOOST_RANDOM_PIECEWISE_CONSTANT_DISTRIBUTION_HPP_INCLUDED - -#include -#include -#include -#include -#include -#include -#include -#include - -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST -#include -#endif - -#include -#include - -namespace boost { -namespace random { - -/** - * The class @c piecewise_constant_distribution models a \random_distribution. - */ -template -class piecewise_constant_distribution { -public: - typedef std::size_t input_type; - typedef RealType result_type; - - class param_type { - public: - - typedef piecewise_constant_distribution distribution_type; - - /** - * Constructs a @c param_type object, representing a distribution - * that produces values uniformly distributed in the range [0, 1). - */ - param_type() - { - _weights.push_back(WeightType(1)); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } - /** - * Constructs a @c param_type object from two iterator ranges - * containing the interval boundaries and the interval weights. - * If there are less than two boundaries, then this is equivalent to - * the default constructor and creates a single interval, [0, 1). - * - * The values of the interval boundaries must be strictly - * increasing, and the number of weights must be one less than - * the number of interval boundaries. If there are extra - * weights, they are ignored. - */ - template - param_type(IntervalIter intervals_first, IntervalIter intervals_last, - WeightIter weight_first) - : _intervals(intervals_first, intervals_last) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - _weights.push_back(WeightType(1)); - } else { - _weights.reserve(_intervals.size() - 1); - for(std::size_t i = 0; i < _intervals.size() - 1; ++i) { - _weights.push_back(*weight_first++); - } - } - } -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * Constructs a @c param_type object from an - * initializer_list containing the interval boundaries - * and a unary function specifying the weights. Each - * weight is determined by calling the function at the - * midpoint of the corresponding interval. - * - * If the initializer_list contains less than two elements, - * this is equivalent to the default constructor and the - * distribution will produce values uniformly distributed - * in the range [0, 1). - */ - template - param_type(const std::initializer_list& il, F f) - : _intervals(il.begin(), il.end()) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - _weights.push_back(WeightType(1)); - } else { - _weights.reserve(_intervals.size() - 1); - for(std::size_t i = 0; i < _intervals.size() - 1; ++i) { - RealType midpoint = (_intervals[i] + _intervals[i + 1]) / 2; - _weights.push_back(f(midpoint)); - } - } - } -#endif - /** - * Constructs a @c param_type object from Boost.Range - * ranges holding the interval boundaries and the weights. If - * there are less than two interval boundaries, this is equivalent - * to the default constructor and the distribution will produce - * values uniformly distributed in the range [0, 1). The - * number of weights must be one less than the number of - * interval boundaries. - */ - template - param_type(const IntervalRange& intervals_arg, - const WeightRange& weights_arg) - : _intervals(boost::begin(intervals_arg), boost::end(intervals_arg)), - _weights(boost::begin(weights_arg), boost::end(weights_arg)) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - _weights.push_back(WeightType(1)); - } - } - - /** - * Constructs the parameters for a distribution that approximates a - * function. The range of the distribution is [xmin, xmax). This - * range is divided into nw equally sized intervals and the weights - * are found by calling the unary function f on the midpoints of the - * intervals. - */ - template - param_type(std::size_t nw, RealType xmin, RealType xmax, F f) - { - std::size_t n = (nw == 0) ? 1 : nw; - double delta = (xmax - xmin) / n; - BOOST_ASSERT(delta > 0); - for(std::size_t k = 0; k < n; ++k) { - _weights.push_back(f(xmin + k*delta + delta/2)); - _intervals.push_back(xmin + k*delta); - } - _intervals.push_back(xmax); - } - - /** Returns a vector containing the interval boundaries. */ - std::vector intervals() const { return _intervals; } - - /** - * Returns a vector containing the probability densities - * over all the intervals of the distribution. - */ - std::vector densities() const - { - RealType sum = std::accumulate(_weights.begin(), _weights.end(), - static_cast(0)); - std::vector result; - result.reserve(_weights.size()); - for(std::size_t i = 0; i < _weights.size(); ++i) { - RealType width = _intervals[i + 1] - _intervals[i]; - result.push_back(_weights[i] / (sum * width)); - } - return result; - } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - detail::print_vector(os, parm._intervals); - detail::print_vector(os, parm._weights); - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - std::vector new_intervals; - std::vector new_weights; - detail::read_vector(is, new_intervals); - detail::read_vector(is, new_weights); - if(is) { - parm._intervals.swap(new_intervals); - parm._weights.swap(new_weights); - } - return is; - } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { - return lhs._intervals == rhs._intervals - && lhs._weights == rhs._weights; - } - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - - friend class piecewise_constant_distribution; - - std::vector _intervals; - std::vector _weights; - }; - - /** - * Creates a new @c piecewise_constant_distribution with - * a single interval, [0, 1). - */ - piecewise_constant_distribution() - { - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } - /** - * Constructs a piecewise_constant_distribution from two iterator ranges - * containing the interval boundaries and the interval weights. - * If there are less than two boundaries, then this is equivalent to - * the default constructor and creates a single interval, [0, 1). - * - * The values of the interval boundaries must be strictly - * increasing, and the number of weights must be one less than - * the number of interval boundaries. If there are extra - * weights, they are ignored. - * - * For example, - * - * @code - * double intervals[] = { 0.0, 1.0, 4.0 }; - * double weights[] = { 1.0, 1.0 }; - * piecewise_constant_distribution<> dist( - * &intervals[0], &intervals[0] + 3, &weights[0]); - * @endcode - * - * The distribution has a 50% chance of producing a - * value between 0 and 1 and a 50% chance of producing - * a value between 1 and 4. - */ - template - piecewise_constant_distribution(IntervalIter first_interval, - IntervalIter last_interval, - WeightIter first_weight) - : _intervals(first_interval, last_interval) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } else { - std::vector actual_weights; - actual_weights.reserve(_intervals.size() - 1); - for(std::size_t i = 0; i < _intervals.size() - 1; ++i) { - actual_weights.push_back(*first_weight++); - } - typedef discrete_distribution bins_type; - typename bins_type::param_type bins_param(actual_weights); - _bins.param(bins_param); - } - } -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * Constructs a piecewise_constant_distribution from an - * initializer_list containing the interval boundaries - * and a unary function specifying the weights. Each - * weight is determined by calling the function at the - * midpoint of the corresponding interval. - * - * If the initializer_list contains less than two elements, - * this is equivalent to the default constructor and the - * distribution will produce values uniformly distributed - * in the range [0, 1). - */ - template - piecewise_constant_distribution(std::initializer_list il, F f) - : _intervals(il.begin(), il.end()) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } else { - std::vector actual_weights; - actual_weights.reserve(_intervals.size() - 1); - for(std::size_t i = 0; i < _intervals.size() - 1; ++i) { - RealType midpoint = (_intervals[i] + _intervals[i + 1]) / 2; - actual_weights.push_back(f(midpoint)); - } - typedef discrete_distribution bins_type; - typename bins_type::param_type bins_param(actual_weights); - _bins.param(bins_param); - } - } -#endif - /** - * Constructs a piecewise_constant_distribution from Boost.Range - * ranges holding the interval boundaries and the weights. If - * there are less than two interval boundaries, this is equivalent - * to the default constructor and the distribution will produce - * values uniformly distributed in the range [0, 1). The - * number of weights must be one less than the number of - * interval boundaries. - */ - template - piecewise_constant_distribution(const IntervalsRange& intervals_arg, - const WeightsRange& weights_arg) - : _bins(weights_arg), - _intervals(boost::begin(intervals_arg), boost::end(intervals_arg)) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } - } - /** - * Constructs a piecewise_constant_distribution that approximates a - * function. The range of the distribution is [xmin, xmax). This - * range is divided into nw equally sized intervals and the weights - * are found by calling the unary function f on the midpoints of the - * intervals. - */ - template - piecewise_constant_distribution(std::size_t nw, - RealType xmin, - RealType xmax, - F f) - : _bins(nw, xmin, xmax, f) - { - if(nw == 0) { nw = 1; } - RealType delta = (xmax - xmin) / nw; - _intervals.reserve(nw + 1); - for(std::size_t i = 0; i < nw; ++i) { - _intervals.push_back(xmin + i * delta); - } - _intervals.push_back(xmax); - } - /** - * Constructs a piecewise_constant_distribution from its parameters. - */ - explicit piecewise_constant_distribution(const param_type& parm) - : _bins(parm._weights), - _intervals(parm._intervals) - { - } - - /** - * Returns a value distributed according to the parameters of the - * piecewist_constant_distribution. - */ - template - RealType operator()(URNG& urng) const - { - std::size_t i = _bins(urng); - return uniform_real(_intervals[i], _intervals[i+1])(urng); - } - - /** - * Returns a value distributed according to the parameters - * specified by param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return piecewise_constant_distribution(parm)(urng); - } - - /** Returns the smallest value that the distribution can produce. */ - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return _intervals.front(); } - /** Returns the largest value that the distribution can produce. */ - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return _intervals.back(); } - - /** - * Returns a vector containing the probability density - * over each interval. - */ - std::vector densities() const - { - std::vector result(_bins.probabilities()); - for(std::size_t i = 0; i < result.size(); ++i) { - result[i] /= (_intervals[i+1] - _intervals[i]); - } - return(result); - } - /** Returns a vector containing the interval boundaries. */ - std::vector intervals() const { return _intervals; } - - /** Returns the parameters of the distribution. */ - param_type param() const - { - return param_type(_intervals, _bins.probabilities()); - } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - std::vector new_intervals(parm._intervals); - typedef discrete_distribution bins_type; - typename bins_type::param_type bins_param(parm._weights); - _bins.param(bins_param); - _intervals.swap(new_intervals); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { _bins.reset(); } - - /** Writes a distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR( - os, piecewise_constant_distribution, pcd) - { - os << pcd.param(); - return os; - } - - /** Reads a distribution from a @c std::istream */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR( - is, piecewise_constant_distribution, pcd) - { - param_type parm; - if(is >> parm) { - pcd.param(parm); - } - return is; - } - - /** - * Returns true if the two distributions will return the - * same sequence of values, when passed equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR( - piecewise_constant_distribution, lhs, rhs) - { - return lhs._bins == rhs._bins && lhs._intervals == rhs._intervals; - } - /** - * Returns true if the two distributions may return different - * sequences of values, when passed equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(piecewise_constant_distribution) - -private: - discrete_distribution _bins; - std::vector _intervals; -}; - -} -} - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/piecewise_linear_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/piecewise_linear_distribution.hpp deleted file mode 100644 index 541c57fb826..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/piecewise_linear_distribution.hpp +++ /dev/null @@ -1,531 +0,0 @@ -/* boost random/piecewise_linear_distribution.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_PIECEWISE_LINEAR_DISTRIBUTION_HPP_INCLUDED -#define BOOST_RANDOM_PIECEWISE_LINEAR_DISTRIBUTION_HPP_INCLUDED - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST -#include -#endif - -#include -#include - -namespace boost { -namespace random { - -/** - * The class @c piecewise_linear_distribution models a \random_distribution. - */ -template -class piecewise_linear_distribution { -public: - typedef std::size_t input_type; - typedef RealType result_type; - - class param_type { - public: - - typedef piecewise_linear_distribution distribution_type; - - /** - * Constructs a @c param_type object, representing a distribution - * that produces values uniformly distributed in the range [0, 1). - */ - param_type() - { - _weights.push_back(RealType(1)); - _weights.push_back(RealType(1)); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } - /** - * Constructs a @c param_type object from two iterator ranges - * containing the interval boundaries and weights at the boundaries. - * If there are fewer than two boundaries, then this is equivalent to - * the default constructor and the distribution will produce values - * uniformly distributed in the range [0, 1). - * - * The values of the interval boundaries must be strictly - * increasing, and the number of weights must be the same as - * the number of interval boundaries. If there are extra - * weights, they are ignored. - */ - template - param_type(IntervalIter intervals_first, IntervalIter intervals_last, - WeightIter weight_first) - : _intervals(intervals_first, intervals_last) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _weights.push_back(RealType(1)); - _weights.push_back(RealType(1)); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } else { - _weights.reserve(_intervals.size()); - for(std::size_t i = 0; i < _intervals.size(); ++i) { - _weights.push_back(*weight_first++); - } - } - } -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * Constructs a @c param_type object from an initializer_list - * containing the interval boundaries and a unary function - * specifying the weights at the boundaries. Each weight is - * determined by calling the function at the corresponding point. - * - * If the initializer_list contains fewer than two elements, - * this is equivalent to the default constructor and the - * distribution will produce values uniformly distributed - * in the range [0, 1). - */ - template - param_type(const std::initializer_list& il, F f) - : _intervals(il.begin(), il.end()) - { - if(_intervals.size() < 2) { - _intervals.clear(); - _weights.push_back(RealType(1)); - _weights.push_back(RealType(1)); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } else { - _weights.reserve(_intervals.size()); - for(typename std::vector::const_iterator - iter = _intervals.begin(), end = _intervals.end(); - iter != end; ++iter) - { - _weights.push_back(f(*iter)); - } - } - } -#endif - /** - * Constructs a @c param_type object from Boost.Range ranges holding - * the interval boundaries and the weights at the boundaries. If - * there are fewer than two interval boundaries, this is equivalent - * to the default constructor and the distribution will produce - * values uniformly distributed in the range [0, 1). The - * number of weights must be equal to the number of - * interval boundaries. - */ - template - param_type(const IntervalRange& intervals_arg, - const WeightRange& weights_arg) - : _intervals(boost::begin(intervals_arg), boost::end(intervals_arg)), - _weights(boost::begin(weights_arg), boost::end(weights_arg)) - { - if(_intervals.size() < 2) { - _weights.clear(); - _weights.push_back(RealType(1)); - _weights.push_back(RealType(1)); - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - } - } - - /** - * Constructs the parameters for a distribution that approximates a - * function. The range of the distribution is [xmin, xmax). This - * range is divided into nw equally sized intervals and the weights - * are found by calling the unary function f on the boundaries of the - * intervals. - */ - template - param_type(std::size_t nw, RealType xmin, RealType xmax, F f) - { - std::size_t n = (nw == 0) ? 1 : nw; - double delta = (xmax - xmin) / n; - BOOST_ASSERT(delta > 0); - for(std::size_t k = 0; k < n; ++k) { - _weights.push_back(f(xmin + k*delta)); - _intervals.push_back(xmin + k*delta); - } - _weights.push_back(f(xmax)); - _intervals.push_back(xmax); - } - - /** Returns a vector containing the interval boundaries. */ - std::vector intervals() const { return _intervals; } - - /** - * Returns a vector containing the probability densities - * at all the interval boundaries. - */ - std::vector densities() const - { - RealType sum = static_cast(0); - for(std::size_t i = 0; i < _intervals.size() - 1; ++i) { - RealType width = _intervals[i + 1] - _intervals[i]; - sum += (_weights[i] + _weights[i + 1]) * width / 2; - } - std::vector result; - result.reserve(_weights.size()); - for(typename std::vector::const_iterator - iter = _weights.begin(), end = _weights.end(); - iter != end; ++iter) - { - result.push_back(*iter / sum); - } - return result; - } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - detail::print_vector(os, parm._intervals); - detail::print_vector(os, parm._weights); - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - std::vector new_intervals; - std::vector new_weights; - detail::read_vector(is, new_intervals); - detail::read_vector(is, new_weights); - if(is) { - parm._intervals.swap(new_intervals); - parm._weights.swap(new_weights); - } - return is; - } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { - return lhs._intervals == rhs._intervals - && lhs._weights == rhs._weights; - } - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - friend class piecewise_linear_distribution; - - std::vector _intervals; - std::vector _weights; - }; - - /** - * Creates a new @c piecewise_linear_distribution that - * produces values uniformly distributed in the range [0, 1). - */ - piecewise_linear_distribution() - { - default_init(); - } - /** - * Constructs a piecewise_linear_distribution from two iterator ranges - * containing the interval boundaries and the weights at the boundaries. - * If there are fewer than two boundaries, then this is equivalent to - * the default constructor and creates a distribution that - * produces values uniformly distributed in the range [0, 1). - * - * The values of the interval boundaries must be strictly - * increasing, and the number of weights must be equal to - * the number of interval boundaries. If there are extra - * weights, they are ignored. - * - * For example, - * - * @code - * double intervals[] = { 0.0, 1.0, 2.0 }; - * double weights[] = { 0.0, 1.0, 0.0 }; - * piecewise_constant_distribution<> dist( - * &intervals[0], &intervals[0] + 3, &weights[0]); - * @endcode - * - * produces a triangle distribution. - */ - template - piecewise_linear_distribution(IntervalIter first_interval, - IntervalIter last_interval, - WeightIter first_weight) - : _intervals(first_interval, last_interval) - { - if(_intervals.size() < 2) { - default_init(); - } else { - _weights.reserve(_intervals.size()); - for(std::size_t i = 0; i < _intervals.size(); ++i) { - _weights.push_back(*first_weight++); - } - init(); - } - } -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** - * Constructs a piecewise_linear_distribution from an - * initializer_list containing the interval boundaries - * and a unary function specifying the weights. Each - * weight is determined by calling the function at the - * corresponding interval boundary. - * - * If the initializer_list contains fewer than two elements, - * this is equivalent to the default constructor and the - * distribution will produce values uniformly distributed - * in the range [0, 1). - */ - template - piecewise_linear_distribution(std::initializer_list il, F f) - : _intervals(il.begin(), il.end()) - { - if(_intervals.size() < 2) { - default_init(); - } else { - _weights.reserve(_intervals.size()); - for(typename std::vector::const_iterator - iter = _intervals.begin(), end = _intervals.end(); - iter != end; ++iter) - { - _weights.push_back(f(*iter)); - } - init(); - } - } -#endif - /** - * Constructs a piecewise_linear_distribution from Boost.Range - * ranges holding the interval boundaries and the weights. If - * there are fewer than two interval boundaries, this is equivalent - * to the default constructor and the distribution will produce - * values uniformly distributed in the range [0, 1). The - * number of weights must be equal to the number of - * interval boundaries. - */ - template - piecewise_linear_distribution(const IntervalsRange& intervals_arg, - const WeightsRange& weights_arg) - : _intervals(boost::begin(intervals_arg), boost::end(intervals_arg)), - _weights(boost::begin(weights_arg), boost::end(weights_arg)) - { - if(_intervals.size() < 2) { - default_init(); - } else { - init(); - } - } - /** - * Constructs a piecewise_linear_distribution that approximates a - * function. The range of the distribution is [xmin, xmax). This - * range is divided into nw equally sized intervals and the weights - * are found by calling the unary function f on the interval boundaries. - */ - template - piecewise_linear_distribution(std::size_t nw, - RealType xmin, - RealType xmax, - F f) - { - if(nw == 0) { nw = 1; } - RealType delta = (xmax - xmin) / nw; - _intervals.reserve(nw + 1); - for(std::size_t i = 0; i < nw; ++i) { - RealType x = xmin + i * delta; - _intervals.push_back(x); - _weights.push_back(f(x)); - } - _intervals.push_back(xmax); - _weights.push_back(f(xmax)); - init(); - } - /** - * Constructs a piecewise_linear_distribution from its parameters. - */ - explicit piecewise_linear_distribution(const param_type& parm) - : _intervals(parm._intervals), - _weights(parm._weights) - { - init(); - } - - /** - * Returns a value distributed according to the parameters of the - * piecewise_linear_distribution. - */ - template - RealType operator()(URNG& urng) const - { - std::size_t i = _bins(urng); - bool is_in_rectangle = (i % 2 == 0); - i = i / 2; - uniform_real dist(_intervals[i], _intervals[i+1]); - if(is_in_rectangle) { - return dist(urng); - } else if(_weights[i] < _weights[i+1]) { - return (std::max)(dist(urng), dist(urng)); - } else { - return (std::min)(dist(urng), dist(urng)); - } - } - - /** - * Returns a value distributed according to the parameters - * specified by param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return piecewise_linear_distribution(parm)(urng); - } - - /** Returns the smallest value that the distribution can produce. */ - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return _intervals.front(); } - /** Returns the largest value that the distribution can produce. */ - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return _intervals.back(); } - - /** - * Returns a vector containing the probability densities - * at the interval boundaries. - */ - std::vector densities() const - { - RealType sum = static_cast(0); - for(std::size_t i = 0; i < _intervals.size() - 1; ++i) { - RealType width = _intervals[i + 1] - _intervals[i]; - sum += (_weights[i] + _weights[i + 1]) * width / 2; - } - std::vector result; - result.reserve(_weights.size()); - for(typename std::vector::const_iterator - iter = _weights.begin(), end = _weights.end(); - iter != end; ++iter) - { - result.push_back(*iter / sum); - } - return result; - } - /** Returns a vector containing the interval boundaries. */ - std::vector intervals() const { return _intervals; } - - /** Returns the parameters of the distribution. */ - param_type param() const - { - return param_type(_intervals, _weights); - } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - std::vector new_intervals(parm._intervals); - std::vector new_weights(parm._weights); - init(new_intervals, new_weights); - _intervals.swap(new_intervals); - _weights.swap(new_weights); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { _bins.reset(); } - - /** Writes a distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR( - os, piecewise_linear_distribution, pld) - { - os << pld.param(); - return os; - } - - /** Reads a distribution from a @c std::istream */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR( - is, piecewise_linear_distribution, pld) - { - param_type parm; - if(is >> parm) { - pld.param(parm); - } - return is; - } - - /** - * Returns true if the two distributions will return the - * same sequence of values, when passed equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR( - piecewise_linear_distribution, lhs, rhs) - { - return lhs._intervals == rhs._intervals && lhs._weights == rhs._weights; - } - /** - * Returns true if the two distributions may return different - * sequences of values, when passed equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(piecewise_linear_distribution) - -private: - - /// @cond \show_private - - void init(const std::vector& intervals_arg, - const std::vector& weights_arg) - { - using std::abs; - std::vector bin_weights; - bin_weights.reserve((intervals_arg.size() - 1) * 2); - for(std::size_t i = 0; i < intervals_arg.size() - 1; ++i) { - RealType width = intervals_arg[i + 1] - intervals_arg[i]; - RealType w1 = weights_arg[i]; - RealType w2 = weights_arg[i + 1]; - bin_weights.push_back((std::min)(w1, w2) * width); - bin_weights.push_back(abs(w1 - w2) * width / 2); - } - typedef discrete_distribution bins_type; - typename bins_type::param_type bins_param(bin_weights); - _bins.param(bins_param); - } - - void init() - { - init(_intervals, _weights); - } - - void default_init() - { - _intervals.clear(); - _intervals.push_back(RealType(0)); - _intervals.push_back(RealType(1)); - _weights.clear(); - _weights.push_back(RealType(1)); - _weights.push_back(RealType(1)); - init(); - } - - discrete_distribution _bins; - std::vector _intervals; - std::vector _weights; - - /// @endcond -}; - -} -} - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/poisson_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/poisson_distribution.hpp deleted file mode 100644 index 759f206e429..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/poisson_distribution.hpp +++ /dev/null @@ -1,360 +0,0 @@ -/* boost random/poisson_distribution.hpp header file - * - * Copyright Jens Maurer 2002 - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_POISSON_DISTRIBUTION_HPP -#define BOOST_RANDOM_POISSON_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { - -namespace detail { - -template -struct poisson_table { - static RealType value[10]; -}; - -template -RealType poisson_table::value[10] = { - 0.0, - 0.0, - 0.69314718055994529, - 1.7917594692280550, - 3.1780538303479458, - 4.7874917427820458, - 6.5792512120101012, - 8.5251613610654147, - 10.604602902745251, - 12.801827480081469 -}; - -} - -/** - * An instantiation of the class template @c poisson_distribution is a - * model of \random_distribution. The poisson distribution has - * \f$p(i) = \frac{e^{-\lambda}\lambda^i}{i!}\f$ - * - * This implementation is based on the PTRD algorithm described - * - * @blockquote - * "The transformed rejection method for generating Poisson random variables", - * Wolfgang Hormann, Insurance: Mathematics and Economics - * Volume 12, Issue 1, February 1993, Pages 39-45 - * @endblockquote - */ -template -class poisson_distribution { -public: - typedef IntType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef poisson_distribution distribution_type; - /** - * Construct a param_type object with the parameter "mean" - * - * Requires: mean > 0 - */ - explicit param_type(RealType mean_arg = RealType(1)) - : _mean(mean_arg) - { - BOOST_ASSERT(_mean > 0); - } - /* Returns the "mean" parameter of the distribution. */ - RealType mean() const { return _mean; } -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const param_type& parm) - { - os << parm._mean; - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, param_type& parm) - { - is >> parm._mean; - return is; - } -#endif - /** Returns true if the parameters have the same values. */ - friend bool operator==(const param_type& lhs, const param_type& rhs) - { - return lhs._mean == rhs._mean; - } - /** Returns true if the parameters have different values. */ - friend bool operator!=(const param_type& lhs, const param_type& rhs) - { - return !(lhs == rhs); - } - private: - RealType _mean; - }; - - /** - * Constructs a @c poisson_distribution with the parameter @c mean. - * - * Requires: mean > 0 - */ - explicit poisson_distribution(RealType mean_arg = RealType(1)) - : _mean(mean_arg) - { - BOOST_ASSERT(_mean > 0); - init(); - } - - /** - * Construct an @c poisson_distribution object from the - * parameters. - */ - explicit poisson_distribution(const param_type& parm) - : _mean(parm.mean()) - { - init(); - } - - /** - * Returns a random variate distributed according to the - * poisson distribution. - */ - template - IntType operator()(URNG& urng) const - { - if(use_inversion()) { - return invert(urng); - } else { - return generate(urng); - } - } - - /** - * Returns a random variate distributed according to the - * poisson distribution with parameters specified by param. - */ - template - IntType operator()(URNG& urng, const param_type& parm) const - { - return poisson_distribution(parm)(urng); - } - - /** Returns the "mean" parameter of the distribution. */ - RealType mean() const { return _mean; } - - /** Returns the smallest value that the distribution can produce. */ - IntType min BOOST_PREVENT_MACRO_SUBSTITUTION() const { return 0; } - /** Returns the largest value that the distribution can produce. */ - IntType max BOOST_PREVENT_MACRO_SUBSTITUTION() const - { return (std::numeric_limits::max)(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_mean); } - /** Sets parameters of the distribution. */ - void param(const param_type& parm) - { - _mean = parm.mean(); - init(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - /** Writes the parameters of the distribution to a @c std::ostream. */ - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, - const poisson_distribution& pd) - { - os << pd.param(); - return os; - } - - /** Reads the parameters of the distribution from a @c std::istream. */ - template - friend std::basic_istream& - operator>>(std::basic_istream& is, poisson_distribution& pd) - { - pd.read(is); - return is; - } -#endif - - /** Returns true if the two distributions will produce the same - sequence of values, given equal generators. */ - friend bool operator==(const poisson_distribution& lhs, - const poisson_distribution& rhs) - { - return lhs._mean == rhs._mean; - } - /** Returns true if the two distributions could produce different - sequences of values, given equal generators. */ - friend bool operator!=(const poisson_distribution& lhs, - const poisson_distribution& rhs) - { - return !(lhs == rhs); - } - -private: - - /// @cond show_private - - template - void read(std::basic_istream& is) { - param_type parm; - if(is >> parm) { - param(parm); - } - } - - bool use_inversion() const - { - return _mean < 10; - } - - static RealType log_factorial(IntType k) - { - BOOST_ASSERT(k >= 0); - BOOST_ASSERT(k < 10); - return detail::poisson_table::value[k]; - } - - void init() - { - using std::sqrt; - using std::exp; - - if(use_inversion()) { - _exp_mean = exp(-_mean); - } else { - _ptrd.smu = sqrt(_mean); - _ptrd.b = 0.931 + 2.53 * _ptrd.smu; - _ptrd.a = -0.059 + 0.02483 * _ptrd.b; - _ptrd.inv_alpha = 1.1239 + 1.1328 / (_ptrd.b - 3.4); - _ptrd.v_r = 0.9277 - 3.6224 / (_ptrd.b - 2); - } - } - - template - IntType generate(URNG& urng) const - { - using std::floor; - using std::abs; - using std::log; - - while(true) { - RealType u; - RealType v = uniform_01()(urng); - if(v <= 0.86 * _ptrd.v_r) { - u = v / _ptrd.v_r - 0.43; - return static_cast(floor( - (2*_ptrd.a/(0.5-abs(u)) + _ptrd.b)*u + _mean + 0.445)); - } - - if(v >= _ptrd.v_r) { - u = uniform_01()(urng) - 0.5; - } else { - u = v/_ptrd.v_r - 0.93; - u = ((u < 0)? -0.5 : 0.5) - u; - v = uniform_01()(urng) * _ptrd.v_r; - } - - RealType us = 0.5 - abs(u); - if(us < 0.013 && v > us) { - continue; - } - - RealType k = floor((2*_ptrd.a/us + _ptrd.b)*u+_mean+0.445); - v = v*_ptrd.inv_alpha/(_ptrd.a/(us*us) + _ptrd.b); - - RealType log_sqrt_2pi = 0.91893853320467267; - - if(k >= 10) { - if(log(v*_ptrd.smu) <= (k + 0.5)*log(_mean/k) - - _mean - - log_sqrt_2pi - + k - - (1/12. - (1/360. - 1/(1260.*k*k))/(k*k))/k) { - return static_cast(k); - } - } else if(k >= 0) { - if(log(v) <= k*log(_mean) - - _mean - - log_factorial(static_cast(k))) { - return static_cast(k); - } - } - } - } - - template - IntType invert(URNG& urng) const - { - RealType p = _exp_mean; - IntType x = 0; - RealType u = uniform_01()(urng); - while(u > p) { - u = u - p; - ++x; - p = _mean * p / x; - } - return x; - } - - RealType _mean; - - union { - // for ptrd - struct { - RealType v_r; - RealType a; - RealType b; - RealType smu; - RealType inv_alpha; - } _ptrd; - // for inversion - RealType _exp_mean; - }; - - /// @endcond -}; - -} // namespace random - -using random::poisson_distribution; - -} // namespace boost - -#include - -#endif // BOOST_RANDOM_POISSON_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/random_device.hpp b/contrib/libboost/boost_1_65_0/boost/random/random_device.hpp deleted file mode 100644 index 8f3903c953c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/random_device.hpp +++ /dev/null @@ -1,143 +0,0 @@ -/* boost random/random_device.hpp header file - * - * Copyright Jens Maurer 2000 - * Copyright Steven Watanabe 2010-2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * $Id$ - * - * Revision history - * 2000-02-18 Portability fixes (thanks to Beman Dawes) - */ - -// See http://www.boost.org/libs/random for documentation. - - -#ifndef BOOST_RANDOM_RANDOM_DEVICE_HPP -#define BOOST_RANDOM_RANDOM_DEVICE_HPP - -#include -#include -#include -#include -#include // force autolink to find Boost.System - -namespace boost { -namespace random { - -/** - * Class \random_device models a \nondeterministic_random_number_generator. - * It uses one or more implementation-defined stochastic processes to - * generate a sequence of uniformly distributed non-deterministic random - * numbers. For those environments where a non-deterministic random number - * generator is not available, class random_device must not be implemented. See - * - * @blockquote - * "Randomness Recommendations for Security", D. Eastlake, S. Crocker, - * J. Schiller, Network Working Group, RFC 1750, December 1994 - * @endblockquote - * - * for further discussions. - * - * @xmlnote - * Some operating systems abstract the computer hardware enough - * to make it difficult to non-intrusively monitor stochastic processes. - * However, several do provide a special device for exactly this purpose. - * It seems to be impossible to emulate the functionality using Standard - * C++ only, so users should be aware that this class may not be available - * on all platforms. - * @endxmlnote - * - * Implementation Note for Linux - * - * On the Linux operating system, token is interpreted as a filesystem - * path. It is assumed that this path denotes an operating system - * pseudo-device which generates a stream of non-deterministic random - * numbers. The pseudo-device should never signal an error or end-of-file. - * Otherwise, @c std::ios_base::failure is thrown. By default, - * \random_device uses the /dev/urandom pseudo-device to retrieve - * the random numbers. Another option would be to specify the /dev/random - * pseudo-device, which blocks on reads if the entropy pool has no more - * random bits available. - * - * Implementation Note for Windows - * - * On the Windows operating system, token is interpreted as the name - * of a cryptographic service provider. By default \random_device uses - * MS_DEF_PROV. - * - * Performance - * - * The test program - * nondet_random_speed.cpp measures the execution times of the - * random_device.hpp implementation of the above algorithms in a tight - * loop. The performance has been evaluated on an - * Intel(R) Core(TM) i7 CPU Q 840 \@ 1.87GHz, 1867 Mhz with - * Visual C++ 2010, Microsoft Windows 7 Professional and with gcc 4.4.5, - * Ubuntu Linux 2.6.35-25-generic. - * - * - * - * - * - *
Platformtime per invocation [microseconds]
Windows 2.9
Linux 1.7
- * - * The measurement error is estimated at +/- 1 usec. - */ -class random_device : private noncopyable -{ -public: - typedef unsigned int result_type; - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - - /** Returns the smallest value that the \random_device can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () { return 0; } - /** Returns the largest value that the \random_device can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () { return ~0u; } - - /** Constructs a @c random_device, optionally using the default device. */ - BOOST_RANDOM_DECL random_device(); - /** - * Constructs a @c random_device, optionally using the given token as an - * access specification (for example, a URL) to some implementation-defined - * service for monitoring a stochastic process. - */ - BOOST_RANDOM_DECL explicit random_device(const std::string& token); - - BOOST_RANDOM_DECL ~random_device(); - - /** - * Returns: An entropy estimate for the random numbers returned by - * operator(), in the range min() to log2( max()+1). A deterministic - * random number generator (e.g. a pseudo-random number engine) - * has entropy 0. - * - * Throws: Nothing. - */ - BOOST_RANDOM_DECL double entropy() const; - /** Returns a random value in the range [min, max]. */ - BOOST_RANDOM_DECL unsigned int operator()(); - - /** Fills a range with random 32-bit values. */ - template - void generate(Iter begin, Iter end) - { - for(; begin != end; ++begin) { - *begin = (*this)(); - } - } - -private: - class impl; - impl * pimpl; -}; - -} // namespace random - -using random::random_device; - -} // namespace boost - -#endif /* BOOST_RANDOM_RANDOM_DEVICE_HPP */ diff --git a/contrib/libboost/boost_1_65_0/boost/random/random_number_generator.hpp b/contrib/libboost/boost_1_65_0/boost/random/random_number_generator.hpp deleted file mode 100644 index ac975e32987..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/random_number_generator.hpp +++ /dev/null @@ -1,73 +0,0 @@ -/* boost random/random_number_generator.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_RANDOM_NUMBER_GENERATOR_HPP -#define BOOST_RANDOM_RANDOM_NUMBER_GENERATOR_HPP - -#include -#include - -#include - -namespace boost { -namespace random { - -/** - * Instantiations of class template random_number_generator model a - * RandomNumberGenerator (std:25.2.11 [lib.alg.random.shuffle]). On - * each invocation, it returns a uniformly distributed integer in - * the range [0..n). - * - * The template parameter IntType shall denote some integer-like value type. - */ -template -class random_number_generator -{ -public: - typedef URNG base_type; - typedef IntType argument_type; - typedef IntType result_type; - /** - * Constructs a random_number_generator functor with the given - * \uniform_random_number_generator as the underlying source of - * random numbers. - */ - random_number_generator(base_type& rng) : _rng(rng) {} - - // compiler-generated copy ctor is fine - // assignment is disallowed because there is a reference member - - /** - * Returns a value in the range [0, n) - */ - result_type operator()(argument_type n) - { - BOOST_ASSERT(n > 0); - return uniform_int_distribution(0, n-1)(_rng); - } - -private: - base_type& _rng; -}; - -} // namespace random - -using random::random_number_generator; - -} // namespace boost - -#include - -#endif // BOOST_RANDOM_RANDOM_NUMBER_GENERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/ranlux.hpp b/contrib/libboost/boost_1_65_0/boost/random/ranlux.hpp deleted file mode 100644 index 82a7ca66f12..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/ranlux.hpp +++ /dev/null @@ -1,99 +0,0 @@ -/* boost random/ranlux.hpp header file - * - * Copyright Jens Maurer 2002 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 created - */ - -#ifndef BOOST_RANDOM_RANLUX_HPP -#define BOOST_RANDOM_RANLUX_HPP - -#include -#include -#include - -namespace boost { - -namespace random { - -namespace detail { -/** - * The ranlux family of generators are described in - * - * @blockquote - * "A portable high-quality random number generator for lattice field theory - * calculations", M. Luescher, Computer Physics Communications, 79 (1994) - * pp 100-110. - * @endblockquote - * - * The levels are given in - * - * @blockquote - * "RANLUX: A Fortran implementation of the high-quality - * pseudorandom number generator of Luescher", F. James, - * Computer Physics Communications 79 (1994) 111-114 - * @endblockquote - */ -class ranlux_documentation {}; -} - -typedef subtract_with_carry_engine ranlux_base; -typedef subtract_with_carry_01_engine ranlux_base_01; -typedef subtract_with_carry_01_engine ranlux64_base_01; - - -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux3; -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux4; - -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux3_01; -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux4_01; - -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux64_3_01; -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux64_4_01; - -#if !defined(BOOST_NO_INT64_T) && !defined(BOOST_NO_INTEGRAL_INT64_T) -typedef subtract_with_carry_engine ranlux64_base; -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux64_3; -/** @copydoc boost::random::detail::ranlux_documentation */ -typedef discard_block_engine ranlux64_4; -#endif /* !BOOST_NO_INT64_T && !BOOST_NO_INTEGRAL_INT64_T */ - - -typedef subtract_with_carry_engine ranlux24_base; -typedef subtract_with_carry_engine ranlux48_base; - -typedef discard_block_engine ranlux24; -#if !defined(BOOST_NO_INT64_T) && !defined(BOOST_NO_INTEGRAL_INT64_T) -typedef discard_block_engine ranlux48; -#endif -} - -using random::ranlux3; -using random::ranlux4; -using random::ranlux3_01; -using random::ranlux4_01; -using random::ranlux64_3_01; -using random::ranlux64_4_01; -#if !defined(BOOST_NO_INT64_T) && !defined(BOOST_NO_INTEGRAL_INT64_T) -using random::ranlux64_3; -using random::ranlux64_4; -#endif - -} // namespace boost - -#endif // BOOST_RANDOM_LINEAR_CONGRUENTIAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/seed_seq.hpp b/contrib/libboost/boost_1_65_0/boost/random/seed_seq.hpp deleted file mode 100644 index d76aef4f5e5..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/seed_seq.hpp +++ /dev/null @@ -1,118 +0,0 @@ -/* boost random/seed_seq.hpp header file - * - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_SEED_SEQ_HPP -#define BOOST_RANDOM_SEED_SEQ_HPP - -#include -#include -#include -#include -#include -#include -#include -#include - -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST -#include -#endif - -namespace boost { -namespace random { - -/** - * The class @c seed_seq stores a sequence of 32-bit words - * for seeding a \pseudo_random_number_generator. These - * words will be combined to fill the entire state of the - * generator. - */ -class seed_seq { -public: - typedef boost::uint_least32_t result_type; - - /** Initializes a seed_seq to hold an empty sequence. */ - seed_seq() {} -#ifndef BOOST_NO_CXX11_HDR_INITIALIZER_LIST - /** Initializes the sequence from an initializer_list. */ - template - seed_seq(const std::initializer_list& il) : v(il.begin(), il.end()) {} -#endif - /** Initializes the sequence from an iterator range. */ - template - seed_seq(Iter first, Iter last) : v(first, last) {} - /** Initializes the sequence from Boost.Range range. */ - template - explicit seed_seq(const Range& range) - : v(boost::begin(range), boost::end(range)) {} - - /** - * Fills a range with 32-bit values based on the stored sequence. - * - * Requires: Iter must be a Random Access Iterator whose value type - * is an unsigned integral type at least 32 bits wide. - */ - template - void generate(Iter first, Iter last) const - { - typedef typename std::iterator_traits::value_type value_type; - std::fill(first, last, static_cast(0x8b8b8b8bu)); - std::size_t s = v.size(); - std::size_t n = last - first; - std::size_t t = - (n >= 623) ? 11 : - (n >= 68) ? 7 : - (n >= 39) ? 5 : - (n >= 7) ? 3 : - (n - 1)/2; - std::size_t p = (n - t) / 2; - std::size_t q = p + t; - std::size_t m = (std::max)(s+1, n); - value_type mask = 0xffffffffu; - for(std::size_t k = 0; k < m; ++k) { - value_type r1 = static_cast - (*(first + k%n) ^ *(first + (k+p)%n) ^ *(first + (k+n-1)%n)); - r1 = r1 ^ (r1 >> 27); - r1 = (r1 * 1664525u) & mask; - value_type r2 = static_cast(r1 + - ((k == 0) ? s : - (k <= s) ? k % n + v[k - 1] : - (k % n))); - *(first + (k+p)%n) = (*(first + (k+p)%n) + r1) & mask; - *(first + (k+q)%n) = (*(first + (k+q)%n) + r2) & mask; - *(first + k%n) = r2; - } - for(std::size_t k = m; k < m + n; ++k) { - value_type r3 = static_cast - ((*(first + k%n) + *(first + (k+p)%n) + *(first + (k+n-1)%n)) - & mask); - r3 = r3 ^ (r3 >> 27); - r3 = (r3 * 1566083941u) & mask; - value_type r4 = static_cast(r3 - k%m); - *(first + (k+p)%n) ^= r3; - *(first + (k+q)%n) ^= r4; - *(first + k%n) = r4; - } - } - /** Returns the size of the sequence. */ - std::size_t size() const { return v.size(); } - /** Writes the stored sequence to iter. */ - template - void param(Iter out) { std::copy(v.begin(), v.end(), out); } -private: - std::vector v; -}; - -} -} - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/shuffle_order.hpp b/contrib/libboost/boost_1_65_0/boost/random/shuffle_order.hpp deleted file mode 100644 index 54645303e80..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/shuffle_order.hpp +++ /dev/null @@ -1,269 +0,0 @@ -/* boost random/shuffle_order.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_SHUFFLE_ORDER_HPP -#define BOOST_RANDOM_SHUFFLE_ORDER_HPP - -#include -#include // std::copy -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { - -/** - * Instatiations of class template @c shuffle_order_engine model a - * \pseudo_random_number_generator. It mixes the output - * of some (usually \linear_congruential_engine) - * \uniform_random_number_generator to get better statistical properties. - * The algorithm is described in - * - * @blockquote - * "Improving a poor random number generator", Carter Bays - * and S.D. Durham, ACM Transactions on Mathematical Software, - * Vol 2, No. 1, March 1976, pp. 59-64. - * http://doi.acm.org/10.1145/355666.355670 - * @endblockquote - * - * The output of the base generator is buffered in an array of - * length k. Every output X(n) has a second role: It gives an - * index into the array where X(n+1) will be retrieved. Used - * array elements are replaced with fresh output from the base - * generator. - * - * Template parameters are the base generator and the array - * length k, which should be around 100. - */ -template -class shuffle_order_engine -{ -public: - typedef UniformRandomNumberGenerator base_type; - typedef typename base_type::result_type result_type; - - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - BOOST_STATIC_CONSTANT(std::size_t, buffer_size = k); - BOOST_STATIC_CONSTANT(std::size_t, table_size = k); - - BOOST_STATIC_ASSERT(std::numeric_limits::is_integer); - BOOST_STATIC_ASSERT(k > 0); - - /** - * Constructs a @c shuffle_order_engine by invoking the - * default constructor of the base generator. - * - * Complexity: Exactly k+1 invocations of the base generator. - */ - shuffle_order_engine() : _rng() { init(); } - /** - * Constructs a @c shuffle_output_engine by invoking the one-argument - * constructor of the base generator with the parameter seed. - * - * Complexity: Exactly k+1 invocations of the base generator. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(shuffle_order_engine, - result_type, s) - { _rng.seed(s); init(); } - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(shuffle_order_engine, SeedSeq, seq) - { _rng.seed(seq); init(); } - /** - * Constructs a @c shuffle_output_engine by using a copy - * of the provided generator. - * - * Precondition: The template argument UniformRandomNumberGenerator - * shall denote a CopyConstructible type. - * - * Complexity: Exactly k+1 invocations of the base generator. - */ - explicit shuffle_order_engine(const base_type & rng) : _rng(rng) { init(); } - -#ifndef BOOST_NO_CXX11_RVALUE_REFERENCES - explicit shuffle_order_engine(base_type&& rng) : _rng(rng) { init(); } -#endif - - template shuffle_order_engine(It& first, It last) - : _rng(first, last) { init(); } - void seed() { _rng.seed(); init(); } - /** - * Invokes the one-argument seed method of the base generator - * with the parameter seed and re-initializes the internal buffer array. - * - * Complexity: Exactly k+1 invocations of the base generator. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(shuffle_order_engine, - result_type, seed_arg) - { _rng.seed(seed_arg); init(); } - /** - * Invokes the one-argument seed method of the base generator - * with the parameter seq and re-initializes the internal buffer array. - * - * Complexity: Exactly k+1 invocations of the base generator. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(shuffle_order_engine, SeedSeq, seq) - { _rng.seed(seq); init(); } - template void seed(It& first, It last) - { _rng.seed(first, last); init(); } - - const base_type& base() const { return _rng; } - - result_type operator()() { - // calculating the range every time may seem wasteful. However, this - // makes the information locally available for the optimizer. - typedef typename boost::random::traits::make_unsigned::type base_unsigned; - const base_unsigned brange = - detail::subtract()((max)(), (min)()); - const base_unsigned off = - detail::subtract()(y, (min)()); - - base_unsigned j; - if(k == 1) { - j = 0; - } else if(brange < (std::numeric_limits::max)() / k) { - // try to do it in the native type if we know that it won't - // overflow - j = k * off / (brange + 1); - } else if(brange < (std::numeric_limits::max)() / k) { - // Otherwise try to use uint64_t - j = static_cast( - static_cast(off) * k / - (static_cast(brange) + 1)); - } else { - boost::uintmax_t divisor = - static_cast(brange) + 1; - j = static_cast(detail::muldiv(off, k, divisor)); - } - // assert(0 <= j && j < k); - y = v[j]; - v[j] = _rng(); - return y; - } - - /** Advances the generator by z steps. */ - void discard(boost::uintmax_t z) - { - for(boost::uintmax_t j = 0; j < z; ++j) { - (*this)(); - } - } - - /** Fills a range with pseudo-random values. */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (base_type::min)(); } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (base_type::max)(); } - - /** Writes a @c shuffle_order_engine to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, shuffle_order_engine, s) - { - os << s._rng; - for(std::size_t i = 0; i < k; ++i) - os << ' ' << s.v[i]; - os << ' ' << s.y; - return os; - } - - /** Reads a @c shuffle_order_engine from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, shuffle_order_engine, s) - { - is >> s._rng; - for(std::size_t i = 0; i < k; ++i) - is >> std::ws >> s.v[i]; - is >> std::ws >> s.y; - return is; - } - - /** Returns true if the two generators will produce identical sequences. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(shuffle_order_engine, x, y) - { return x._rng == y._rng && x.y == y.y && std::equal(x.v, x.v+k, y.v); } - /** Returns true if the two generators will produce different sequences. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(shuffle_order_engine) - -private: - - /// \cond show_private - - void init() - { - // we cannot use std::generate, because it uses pass-by-value for _rng - for(result_type * p = v; p != v+k; ++p) - *p = _rng(); - y = _rng(); - } - - /// \endcond - - base_type _rng; - result_type v[k]; - result_type y; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool shuffle_order_engine::has_fixed_range; -template -const std::size_t shuffle_order_engine::table_size; -template -const std::size_t shuffle_order_engine::buffer_size; -#endif - -/** - * According to Harry Erwin (private e-mail), the specialization - * @c kreutzer1986 was suggested in: - * - * @blockquote - * "System Simulation: Programming Styles and Languages (International - * Computer Science Series)", Wolfgang Kreutzer, Addison-Wesley, December 1986. - * @endblockquote - */ -typedef shuffle_order_engine< - linear_congruential_engine, - 97> kreutzer1986; - -/** - * The specialization @c knuth_b is specified by the C++ standard. - * It is described in - * - * @blockquote - * "The Art of Computer Programming, Second Edition, Volume 2, - * Seminumerical Algorithms", Donald Knuth, Addison-Wesley, 1981. - * @endblockquote - */ -typedef shuffle_order_engine knuth_b; - -} // namespace random - -using random::kreutzer1986; - -} // namespace boost - -#include - -#endif // BOOST_RANDOM_SHUFFLE_OUTPUT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/shuffle_output.hpp b/contrib/libboost/boost_1_65_0/boost/random/shuffle_output.hpp deleted file mode 100644 index 478e1029bd8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/shuffle_output.hpp +++ /dev/null @@ -1,51 +0,0 @@ -/* boost random/shuffle_output.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_SHUFFLE_OUTPUT_HPP -#define BOOST_RANDOM_SHUFFLE_OUTPUT_HPP - -#include - -namespace boost { -namespace random { - -/// \cond - -template -class shuffle_output : public shuffle_order_engine -{ - typedef shuffle_order_engine base_t; -public: - typedef typename base_t::result_type result_type; - shuffle_output() {} - template - explicit shuffle_output(T& arg) : base_t(arg) {} - template - explicit shuffle_output(const T& arg) : base_t(arg) {} - template - shuffle_output(It& first, It last) : base_t(first, last) {} - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (this->base().min)(); } - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return (this->base().max)(); } -}; - -/// \endcond - -} -} - -#endif // BOOST_RANDOM_SHUFFLE_OUTPUT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/student_t_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/student_t_distribution.hpp deleted file mode 100644 index e28b0e3bfb3..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/student_t_distribution.hpp +++ /dev/null @@ -1,180 +0,0 @@ -/* boost random/student_t_distribution.hpp header file - * - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_STUDENT_T_DISTRIBUTION_HPP -#define BOOST_RANDOM_STUDENT_T_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The Student t distribution is a real valued distribution with one - * parameter n, the number of degrees of freedom. - * - * It has \f$\displaystyle p(x) = - * \frac{1}{\sqrt{n\pi}} - * \frac{\Gamma((n+1)/2)}{\Gamma(n/2)} - * \left(1+\frac{x^2}{n}\right)^{-(n+1)/2} - * \f$. - */ -template -class student_t_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef student_t_distribution distribution_type; - - /** - * Constructs a @c param_type with "n" degrees of freedom. - * - * Requires: n > 0 - */ - explicit param_type(RealType n_arg = RealType(1.0)) - : _n(n_arg) - {} - - /** Returns the number of degrees of freedom of the distribution. */ - RealType n() const { return _n; } - - /** Writes a @c param_type to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { os << parm._n; return os; } - - /** Reads a @c param_type from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { is >> parm._n; return is; } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._n == rhs._n; } - - /** Returns true if the two sets of parameters are the different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _n; - }; - - /** - * Constructs an @c student_t_distribution with "n" degrees of freedom. - * - * Requires: n > 0 - */ - explicit student_t_distribution(RealType n_arg = RealType(1.0)) - : _normal(), _chi_squared(n_arg) - {} - /** Constructs an @c student_t_distribution from its parameters. */ - explicit student_t_distribution(const param_type& parm) - : _normal(), _chi_squared(parm.n()) - {} - - /** - * Returns a random variate distributed according to the - * Student t distribution. - */ - template - RealType operator()(URNG& urng) - { - using std::sqrt; - return _normal(urng) / sqrt(_chi_squared(urng) / n()); - } - - /** - * Returns a random variate distributed accordint to the Student - * t distribution with parameters specified by @c param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return student_t_distribution(parm)(urng); - } - - /** Returns the number of degrees of freedom. */ - RealType n() const { return _chi_squared.n(); } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return -std::numeric_limits::infinity(); } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return std::numeric_limits::infinity(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(n()); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - typedef chi_squared_distribution chi_squared_type; - typename chi_squared_type::param_type chi_squared_param(parm.n()); - _chi_squared.param(chi_squared_param); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() - { - _normal.reset(); - _chi_squared.reset(); - } - - /** Writes a @c student_t_distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, student_t_distribution, td) - { - os << td.param(); - return os; - } - - /** Reads a @c student_t_distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, student_t_distribution, td) - { - param_type parm; - if(is >> parm) { - td.param(parm); - } - return is; - } - - /** - * Returns true if the two instances of @c student_t_distribution will - * return identical sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(student_t_distribution, lhs, rhs) - { return lhs._normal == rhs._normal && lhs._chi_squared == rhs._chi_squared; } - - /** - * Returns true if the two instances of @c student_t_distribution will - * return different sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(student_t_distribution) - -private: - normal_distribution _normal; - chi_squared_distribution _chi_squared; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_STUDENT_T_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/subtract_with_carry.hpp b/contrib/libboost/boost_1_65_0/boost/random/subtract_with_carry.hpp deleted file mode 100644 index acaf69ac698..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/subtract_with_carry.hpp +++ /dev/null @@ -1,613 +0,0 @@ -/* boost random/subtract_with_carry.hpp header file - * - * Copyright Jens Maurer 2002 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2002-03-02 created - */ - -#ifndef BOOST_RANDOM_SUBTRACT_WITH_CARRY_HPP -#define BOOST_RANDOM_SUBTRACT_WITH_CARRY_HPP - -#include // std::pow -#include -#include // std::equal -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace boost { -namespace random { - -namespace detail { - -struct subtract_with_carry_discard -{ - template - static void apply(Engine& eng, boost::uintmax_t z) - { - typedef typename Engine::result_type IntType; - const std::size_t short_lag = Engine::short_lag; - const std::size_t long_lag = Engine::long_lag; - std::size_t k = eng.k; - IntType carry = eng.carry; - if(k != 0) { - // increment k until it becomes 0. - if(k < short_lag) { - std::size_t limit = (short_lag - k) < z? - short_lag : (k + static_cast(z)); - for(std::size_t j = k; j < limit; ++j) { - carry = eng.do_update(j, j + long_lag - short_lag, carry); - } - } - std::size_t limit = (long_lag - k) < z? - long_lag : (k + static_cast(z)); - std::size_t start = (k < short_lag ? short_lag : k); - for(std::size_t j = start; j < limit; ++j) { - carry = eng.do_update(j, j - short_lag, carry); - } - } - - k = ((z % long_lag) + k) % long_lag; - - if(k < z) { - // main loop: update full blocks from k = 0 to long_lag - for(std::size_t i = 0; i < (z - k) / long_lag; ++i) { - for(std::size_t j = 0; j < short_lag; ++j) { - carry = eng.do_update(j, j + long_lag - short_lag, carry); - } - for(std::size_t j = short_lag; j < long_lag; ++j) { - carry = eng.do_update(j, j - short_lag, carry); - } - } - - // Update the last partial block - std::size_t limit = short_lag < k? short_lag : k; - for(std::size_t j = 0; j < limit; ++j) { - carry = eng.do_update(j, j + long_lag - short_lag, carry); - } - for(std::size_t j = short_lag; j < k; ++j) { - carry = eng.do_update(j, j - short_lag, carry); - } - } - eng.carry = carry; - eng.k = k; - } -}; - -} - -/** - * Instantiations of @c subtract_with_carry_engine model a - * \pseudo_random_number_generator. The algorithm is - * described in - * - * @blockquote - * "A New Class of Random Number Generators", George - * Marsaglia and Arif Zaman, Annals of Applied Probability, - * Volume 1, Number 3 (1991), 462-480. - * @endblockquote - */ -template -class subtract_with_carry_engine -{ -public: - typedef IntType result_type; - BOOST_STATIC_CONSTANT(std::size_t, word_size = w); - BOOST_STATIC_CONSTANT(std::size_t, long_lag = r); - BOOST_STATIC_CONSTANT(std::size_t, short_lag = s); - BOOST_STATIC_CONSTANT(uint32_t, default_seed = 19780503u); - - // Required by the old Boost.Random concepts - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - // Backwards compatibility - BOOST_STATIC_CONSTANT(result_type, modulus = (result_type(1) << w)); - - BOOST_STATIC_ASSERT(std::numeric_limits::is_integer); - - /** - * Constructs a new @c subtract_with_carry_engine and seeds - * it with the default seed. - */ - subtract_with_carry_engine() { seed(); } - /** - * Constructs a new @c subtract_with_carry_engine and seeds - * it with @c value. - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(subtract_with_carry_engine, - IntType, value) - { seed(value); } - /** - * Constructs a new @c subtract_with_carry_engine and seeds - * it with values produced by @c seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(subtract_with_carry_engine, - SeedSeq, seq) - { seed(seq); } - /** - * Constructs a new @c subtract_with_carry_engine and seeds - * it with values from a range. first is updated to point - * one past the last value consumed. If there are not - * enough elements in the range to fill the entire state of - * the generator, throws @c std::invalid_argument. - */ - template subtract_with_carry_engine(It& first, It last) - { seed(first,last); } - - // compiler-generated copy ctor and assignment operator are fine - - /** Seeds the generator with the default seed. */ - void seed() { seed(default_seed); } - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(subtract_with_carry_engine, - IntType, value) - { - typedef linear_congruential_engine gen_t; - gen_t intgen(static_cast(value == 0 ? default_seed : value)); - detail::generator_seed_seq gen(intgen); - seed(gen); - } - - /** Seeds the generator with values produced by @c seq.generate(). */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(subtract_with_carry, SeedSeq, seq) - { - detail::seed_array_int(seq, x); - carry = (x[long_lag-1] == 0); - k = 0; - } - - /** - * Seeds the generator with values from a range. Updates @c first to - * point one past the last consumed value. If the range does not - * contain enough elements to fill the entire state of the generator, - * throws @c std::invalid_argument. - */ - template - void seed(It& first, It last) - { - detail::fill_array_int(first, last, x); - carry = (x[long_lag-1] == 0); - k = 0; - } - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return 0; } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return boost::low_bits_mask_t::sig_bits; } - - /** Returns the next value of the generator. */ - result_type operator()() - { - std::size_t short_index = - (k < short_lag)? - (k + long_lag - short_lag) : - (k - short_lag); - carry = do_update(k, short_index, carry); - IntType result = x[k]; - ++k; - if(k >= long_lag) - k = 0; - return result; - } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - detail::subtract_with_carry_discard::apply(*this, z); - } - - /** Fills a range with random values. */ - template - void generate(It first, It last) - { detail::generate_from_int(*this, first, last); } - - /** Writes a @c subtract_with_carry_engine to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, subtract_with_carry_engine, f) - { - for(unsigned int j = 0; j < f.long_lag; ++j) - os << f.compute(j) << ' '; - os << f.carry; - return os; - } - - /** Reads a @c subtract_with_carry_engine from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, subtract_with_carry_engine, f) - { - for(unsigned int j = 0; j < f.long_lag; ++j) - is >> f.x[j] >> std::ws; - is >> f.carry; - f.k = 0; - return is; - } - - /** - * Returns true if the two generators will produce identical - * sequences of values. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(subtract_with_carry_engine, x, y) - { - for(unsigned int j = 0; j < r; ++j) - if(x.compute(j) != y.compute(j)) - return false; - return true; - } - - /** - * Returns true if the two generators will produce different - * sequences of values. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(subtract_with_carry_engine) - -private: - /// \cond show_private - // returns x(i-r+index), where index is in 0..r-1 - IntType compute(unsigned int index) const - { - return x[(k+index) % long_lag]; - } - - friend struct detail::subtract_with_carry_discard; - - IntType do_update(std::size_t current, std::size_t short_index, IntType carry) - { - IntType delta; - IntType temp = x[current] + carry; - if (x[short_index] >= temp) { - // x(n) >= 0 - delta = x[short_index] - temp; - carry = 0; - } else { - // x(n) < 0 - delta = modulus - temp + x[short_index]; - carry = 1; - } - x[current] = delta; - return carry; - } - /// \endcond - - // state representation; next output (state) is x(i) - // x[0] ... x[k] x[k+1] ... x[long_lag-1] represents - // x(i-k) ... x(i) x(i+1) ... x(i-k+long_lag-1) - // speed: base: 20-25 nsec - // ranlux_4: 230 nsec, ranlux_7: 430 nsec, ranlux_14: 810 nsec - // This state representation makes operator== and save/restore more - // difficult, because we've already computed "too much" and thus - // have to undo some steps to get at x(i-r) etc. - - // state representation: next output (state) is x(i) - // x[0] ... x[k] x[k+1] ... x[long_lag-1] represents - // x(i-k) ... x(i) x(i-long_lag+1) ... x(i-k-1) - // speed: base 28 nsec - // ranlux_4: 370 nsec, ranlux_7: 688 nsec, ranlux_14: 1343 nsec - IntType x[long_lag]; - std::size_t k; - IntType carry; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool subtract_with_carry_engine::has_fixed_range; -template -const IntType subtract_with_carry_engine::modulus; -template -const std::size_t subtract_with_carry_engine::word_size; -template -const std::size_t subtract_with_carry_engine::long_lag; -template -const std::size_t subtract_with_carry_engine::short_lag; -template -const uint32_t subtract_with_carry_engine::default_seed; -#endif - - -// use a floating-point representation to produce values in [0..1) -/** - * Instantiations of \subtract_with_carry_01_engine model a - * \pseudo_random_number_generator. The algorithm is - * described in - * - * @blockquote - * "A New Class of Random Number Generators", George - * Marsaglia and Arif Zaman, Annals of Applied Probability, - * Volume 1, Number 3 (1991), 462-480. - * @endblockquote - */ -template -class subtract_with_carry_01_engine -{ -public: - typedef RealType result_type; - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - BOOST_STATIC_CONSTANT(std::size_t, word_size = w); - BOOST_STATIC_CONSTANT(std::size_t, long_lag = r); - BOOST_STATIC_CONSTANT(std::size_t, short_lag = s); - BOOST_STATIC_CONSTANT(boost::uint32_t, default_seed = 19780503u); - - BOOST_STATIC_ASSERT(!std::numeric_limits::is_integer); - - /** Creates a new \subtract_with_carry_01_engine using the default seed. */ - subtract_with_carry_01_engine() { init_modulus(); seed(); } - /** Creates a new subtract_with_carry_01_engine and seeds it with value. */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(subtract_with_carry_01_engine, - boost::uint32_t, value) - { init_modulus(); seed(value); } - /** - * Creates a new \subtract_with_carry_01_engine and seeds with values - * produced by seq.generate(). - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(subtract_with_carry_01_engine, - SeedSeq, seq) - { init_modulus(); seed(seq); } - /** - * Creates a new \subtract_with_carry_01_engine and seeds it with values - * from a range. Advances first to point one past the last consumed - * value. If the range does not contain enough elements to fill the - * entire state, throws @c std::invalid_argument. - */ - template subtract_with_carry_01_engine(It& first, It last) - { init_modulus(); seed(first,last); } - -private: - /// \cond show_private - void init_modulus() - { -#ifndef BOOST_NO_STDC_NAMESPACE - // allow for Koenig lookup - using std::pow; -#endif - _modulus = pow(RealType(2), RealType(word_size)); - } - /// \endcond - -public: - // compiler-generated copy ctor and assignment operator are fine - - /** Seeds the generator with the default seed. */ - void seed() { seed(default_seed); } - - /** Seeds the generator with @c value. */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(subtract_with_carry_01_engine, - boost::uint32_t, value) - { - typedef linear_congruential_engine gen_t; - gen_t intgen(value == 0 ? default_seed : value); - detail::generator_seed_seq gen(intgen); - seed(gen); - } - - /** Seeds the generator with values produced by @c seq.generate(). */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(subtract_with_carry_01_engine, - SeedSeq, seq) - { - detail::seed_array_real(seq, x); - carry = (x[long_lag-1] ? result_type(0) : result_type(1 / _modulus)); - k = 0; - } - - /** - * Seeds the generator with values from a range. Updates first to - * point one past the last consumed element. If there are not - * enough elements in the range to fill the entire state, throws - * @c std::invalid_argument. - */ - template - void seed(It& first, It last) - { - detail::fill_array_real(first, last, x); - carry = (x[long_lag-1] ? result_type(0) : result_type(1 / _modulus)); - k = 0; - } - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () - { return result_type(0); } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () - { return result_type(1); } - - /** Returns the next value of the generator. */ - result_type operator()() - { - std::size_t short_index = - (k < short_lag) ? - (k + long_lag - short_lag) : - (k - short_lag); - carry = do_update(k, short_index, carry); - RealType result = x[k]; - ++k; - if(k >= long_lag) - k = 0; - return result; - } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { detail::subtract_with_carry_discard::apply(*this, z); } - - /** Fills a range with random values. */ - template - void generate(Iter first, Iter last) - { detail::generate_from_real(*this, first, last); } - - /** Writes a \subtract_with_carry_01_engine to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, subtract_with_carry_01_engine, f) - { - std::ios_base::fmtflags oldflags = - os.flags(os.dec | os.fixed | os.left); - for(unsigned int j = 0; j < f.long_lag; ++j) - os << (f.compute(j) * f._modulus) << ' '; - os << (f.carry * f._modulus); - os.flags(oldflags); - return os; - } - - /** Reads a \subtract_with_carry_01_engine from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, subtract_with_carry_01_engine, f) - { - RealType value; - for(unsigned int j = 0; j < long_lag; ++j) { - is >> value >> std::ws; - f.x[j] = value / f._modulus; - } - is >> value; - f.carry = value / f._modulus; - f.k = 0; - return is; - } - - /** Returns true if the two generators will produce identical sequences. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(subtract_with_carry_01_engine, x, y) - { - for(unsigned int j = 0; j < r; ++j) - if(x.compute(j) != y.compute(j)) - return false; - return true; - } - - /** Returns true if the two generators will produce different sequences. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(subtract_with_carry_01_engine) - -private: - /// \cond show_private - RealType compute(unsigned int index) const - { - return x[(k+index) % long_lag]; - } - - friend struct detail::subtract_with_carry_discard; - - RealType do_update(std::size_t current, std::size_t short_index, RealType carry) - { - RealType delta = x[short_index] - x[current] - carry; - if(delta < 0) { - delta += RealType(1); - carry = RealType(1)/_modulus; - } else { - carry = 0; - } - x[current] = delta; - return carry; - } - /// \endcond - std::size_t k; - RealType carry; - RealType x[long_lag]; - RealType _modulus; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool subtract_with_carry_01_engine::has_fixed_range; -template -const std::size_t subtract_with_carry_01_engine::word_size; -template -const std::size_t subtract_with_carry_01_engine::long_lag; -template -const std::size_t subtract_with_carry_01_engine::short_lag; -template -const uint32_t subtract_with_carry_01_engine::default_seed; -#endif - - -/// \cond show_deprecated - -template -class subtract_with_carry : - public subtract_with_carry_engine::value, s, r> -{ - typedef subtract_with_carry_engine::value, s, r> base_type; -public: - subtract_with_carry() {} - BOOST_RANDOM_DETAIL_GENERATOR_CONSTRUCTOR(subtract_with_carry, Gen, gen) - { seed(gen); } - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(subtract_with_carry, - IntType, val) - { seed(val); } - template - subtract_with_carry(It& first, It last) : base_type(first, last) {} - void seed() { base_type::seed(); } - BOOST_RANDOM_DETAIL_GENERATOR_SEED(subtract_with_carry, Gen, gen) - { - detail::generator_seed_seq seq(gen); - base_type::seed(seq); - } - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(subtract_with_carry, IntType, val) - { base_type::seed(val); } - template - void seed(It& first, It last) { base_type::seed(first, last); } -}; - -template -class subtract_with_carry_01 : - public subtract_with_carry_01_engine -{ - typedef subtract_with_carry_01_engine base_type; -public: - subtract_with_carry_01() {} - BOOST_RANDOM_DETAIL_GENERATOR_CONSTRUCTOR(subtract_with_carry_01, Gen, gen) - { seed(gen); } - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(subtract_with_carry_01, - uint32_t, val) - { seed(val); } - template - subtract_with_carry_01(It& first, It last) : base_type(first, last) {} - void seed() { base_type::seed(); } - BOOST_RANDOM_DETAIL_GENERATOR_SEED(subtract_with_carry_01, Gen, gen) - { - detail::generator_seed_seq seq(gen); - base_type::seed(seq); - } - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(subtract_with_carry_01, uint32_t, val) - { base_type::seed(val); } - template - void seed(It& first, It last) { base_type::seed(first, last); } -}; - -/// \endcond - -namespace detail { - -template -struct generator_bits; - -template -struct generator_bits > { - static std::size_t value() { return w; } -}; - -template -struct generator_bits > { - static std::size_t value() { return w; } -}; - -} - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_SUBTRACT_WITH_CARRY_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/taus88.hpp b/contrib/libboost/boost_1_65_0/boost/random/taus88.hpp deleted file mode 100644 index 68214f3aed0..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/taus88.hpp +++ /dev/null @@ -1,45 +0,0 @@ -/* boost random/taus88.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org/libs/random for documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_TAUS88_HPP -#define BOOST_RANDOM_TAUS88_HPP - -#include -#include - -namespace boost { -namespace random { - -/** - * The specialization taus88 was suggested in - * - * @blockquote - * "Maximally Equidistributed Combined Tausworthe Generators", - * Pierre L'Ecuyer, Mathematics of Computation, Volume 65, - * Number 213, January 1996, Pages 203-213 - * @endblockquote - */ -typedef xor_combine_engine< - xor_combine_engine< - linear_feedback_shift_engine, 0, - linear_feedback_shift_engine, 0>, 0, - linear_feedback_shift_engine, 0> taus88; - -} // namespace random - -using random::taus88; - -} // namespace boost - -#endif // BOOST_RANDOM_TAUS88_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/traits.hpp b/contrib/libboost/boost_1_65_0/boost/random/traits.hpp deleted file mode 100644 index 975421a4d0a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/traits.hpp +++ /dev/null @@ -1,107 +0,0 @@ -/* boost random/traits.hpp header file - * - * Copyright John Maddock 2015 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * These traits classes serve two purposes: they are designed to mostly - * work out of the box for multiprecision types (ie number types that are - * C++ class types and not integers or floats from type-traits point of view), - * they are also a potential point of specialization for user-defined - * number types. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_TRAITS_HPP -#define BOOST_RANDOM_TRAITS_HPP - -#include -#include -#include -#include -#include - -namespace boost { -namespace random { -namespace traits { - // \cond show_private - template - struct make_unsigned_imp - { - typedef typename boost::make_unsigned::type type; - }; - template - struct make_unsigned_imp - { - BOOST_STATIC_ASSERT(std::numeric_limits::is_specialized); - BOOST_STATIC_ASSERT(std::numeric_limits::is_signed == false); - BOOST_STATIC_ASSERT(std::numeric_limits::is_integer == true); - typedef T type; - }; - // \endcond - /** \brief Converts the argument type T to an unsigned type. - * - * This trait has a single member `type` which is the unsigned type corresponding to T. - * Note that - * if T is signed, then member `type` *should define a type with one more bit precision than T*. For built-in - * types this trait defaults to `boost::make_unsigned::type`. For user defined types it simply asserts that - * the argument type T is an unsigned integer (using std::numeric_limits). - * User defined specializations may be provided for other cases. - */ - template - struct make_unsigned - // \cond show_private - : public make_unsigned_imp < T, boost::is_integral::value > - // \endcond - {}; - // \cond show_private - template - struct make_unsigned_or_unbounded_imp - { - typedef typename boost::make_unsigned::type type; - }; - template - struct make_unsigned_or_unbounded_imp - { - BOOST_STATIC_ASSERT(std::numeric_limits::is_specialized); - BOOST_STATIC_ASSERT((std::numeric_limits::is_signed == false) || (std::numeric_limits::is_bounded == false)); - BOOST_STATIC_ASSERT(std::numeric_limits::is_integer == true); - typedef T type; - }; - // \endcond - /** \brief Converts the argument type T to either an unsigned type or an unbounded integer type. - * - * This trait has a single member `type` which is either the unsigned type corresponding to T or an unbounded - * integer type. This trait is used to generate types suitable for the calculation of a range: as a result - * if T is signed, then member `type` *should define a type with one more bit precision than T*. For built-in - * types this trait defaults to `boost::make_unsigned::type`. For user defined types it simply asserts that - * the argument type T is either an unbounded integer, or an unsigned one (using std::numeric_limits). - * User defined specializations may be provided for other cases. - */ - template - struct make_unsigned_or_unbounded - // \cond show_private - : public make_unsigned_or_unbounded_imp < T, boost::is_integral::value > - // \endcond - {}; - /** \brief Traits class that indicates whether type T is an integer - */ - template - struct is_integral - : public mpl::bool_::value || (std::numeric_limits::is_integer)> - {}; - /** \brief Traits class that indicates whether type T is a signed integer - */ - template struct is_signed - : public mpl::bool_ < boost::is_signed::value || (std::numeric_limits::is_specialized && std::numeric_limits::is_integer && std::numeric_limits::is_signed)> - {}; - -} -} -} - -#endif diff --git a/contrib/libboost/boost_1_65_0/boost/random/triangle_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/triangle_distribution.hpp deleted file mode 100644 index 6d37e2bb926..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/triangle_distribution.hpp +++ /dev/null @@ -1,232 +0,0 @@ -/* boost random/triangle_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_TRIANGLE_DISTRIBUTION_HPP -#define BOOST_RANDOM_TRIANGLE_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * Instantiations of @c triangle_distribution model a \random_distribution. - * A @c triangle_distribution has three parameters, @c a, @c b, and @c c, - * which are the smallest, the most probable and the largest values of - * the distribution respectively. - */ -template -class triangle_distribution -{ -public: - typedef RealType input_type; - typedef RealType result_type; - - class param_type - { - public: - - typedef triangle_distribution distribution_type; - - /** Constructs the parameters of a @c triangle_distribution. */ - explicit param_type(RealType a_arg = RealType(0.0), - RealType b_arg = RealType(0.5), - RealType c_arg = RealType(1.0)) - : _a(a_arg), _b(b_arg), _c(c_arg) - { - BOOST_ASSERT(_a <= _b && _b <= _c); - } - - /** Returns the minimum value of the distribution. */ - RealType a() const { return _a; } - /** Returns the mode of the distribution. */ - RealType b() const { return _b; } - /** Returns the maximum value of the distribution. */ - RealType c() const { return _c; } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._a << " " << parm._b << " " << parm._c; - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - double a_in, b_in, c_in; - if(is >> a_in >> std::ws >> b_in >> std::ws >> c_in) { - if(a_in <= b_in && b_in <= c_in) { - parm._a = a_in; - parm._b = b_in; - parm._c = c_in; - } else { - is.setstate(std::ios_base::failbit); - } - } - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._a == rhs._a && lhs._b == rhs._b && lhs._c == rhs._c; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _a; - RealType _b; - RealType _c; - }; - - /** - * Constructs a @c triangle_distribution with the parameters - * @c a, @c b, and @c c. - * - * Preconditions: a <= b <= c. - */ - explicit triangle_distribution(RealType a_arg = RealType(0.0), - RealType b_arg = RealType(0.5), - RealType c_arg = RealType(1.0)) - : _a(a_arg), _b(b_arg), _c(c_arg) - { - BOOST_ASSERT(_a <= _b && _b <= _c); - init(); - } - - /** Constructs a @c triangle_distribution from its parameters. */ - explicit triangle_distribution(const param_type& parm) - : _a(parm.a()), _b(parm.b()), _c(parm.c()) - { - init(); - } - - // compiler-generated copy ctor and assignment operator are fine - - /** Returns the @c a parameter of the distribution */ - result_type a() const { return _a; } - /** Returns the @c b parameter of the distribution */ - result_type b() const { return _b; } - /** Returns the @c c parameter of the distribution */ - result_type c() const { return _c; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _a; } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _c; } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_a, _b, _c); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _a = parm.a(); - _b = parm.b(); - _c = parm.c(); - init(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** - * Returns a random variate distributed according to the - * triangle distribution. - */ - template - result_type operator()(Engine& eng) - { - using std::sqrt; - result_type u = uniform_01()(eng); - if( u <= q1 ) - return _a + p1*sqrt(u); - else - return _c - d3*sqrt(d2*u-d1); - } - - /** - * Returns a random variate distributed according to the - * triangle distribution with parameters specified by param. - */ - template - result_type operator()(Engine& eng, const param_type& parm) - { return triangle_distribution(parm)(eng); } - - /** Writes the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, triangle_distribution, td) - { - os << td.param(); - return os; - } - - /** Reads the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, triangle_distribution, td) - { - param_type parm; - if(is >> parm) { - td.param(parm); - } - return is; - } - - /** - * Returns true if the two distributions will produce identical - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(triangle_distribution, lhs, rhs) - { return lhs._a == rhs._a && lhs._b == rhs._b && lhs._c == rhs._c; } - - /** - * Returns true if the two distributions may produce different - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(triangle_distribution) - -private: - /// \cond show_private - void init() - { - using std::sqrt; - d1 = _b - _a; - d2 = _c - _a; - d3 = sqrt(_c - _b); - q1 = d1 / d2; - p1 = sqrt(d1 * d2); - } - /// \endcond - - RealType _a, _b, _c; - RealType d1, d2, d3, q1, p1; -}; - -} // namespace random - -using random::triangle_distribution; - -} // namespace boost - -#endif // BOOST_RANDOM_TRIANGLE_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/uniform_01.hpp b/contrib/libboost/boost_1_65_0/boost/random/uniform_01.hpp deleted file mode 100644 index 37258f58064..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/uniform_01.hpp +++ /dev/null @@ -1,257 +0,0 @@ -/* boost random/uniform_01.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_UNIFORM_01_HPP -#define BOOST_RANDOM_UNIFORM_01_HPP - -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { -namespace random { - -#ifdef BOOST_RANDOM_DOXYGEN - -/** - * The distribution function uniform_01 models a \random_distribution. - * On each invocation, it returns a random floating-point value - * uniformly distributed in the range [0..1). - * - * The template parameter RealType shall denote a float-like value type - * with support for binary operators +, -, and /. - * - * Note: The current implementation is buggy, because it may not fill - * all of the mantissa with random bits. I'm unsure how to fill a - * (to-be-invented) @c boost::bigfloat class with random bits efficiently. - * It's probably time for a traits class. - */ -template -class uniform_01 -{ -public: - typedef RealType input_type; - typedef RealType result_type; - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const; - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const; - void reset(); - - template - result_type operator()(Engine& eng); - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, const new_uniform_01&) - { - return os; - } - - template - friend std::basic_istream& - operator>>(std::basic_istream& is, new_uniform_01&) - { - return is; - } -#endif -}; - -#else - -namespace detail { - -template -class new_uniform_01 -{ -public: - typedef RealType input_type; - typedef RealType result_type; - // compiler-generated copy ctor and copy assignment are fine - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return result_type(0); } - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return result_type(1); } - void reset() { } - - template - result_type operator()(Engine& eng) { - for (;;) { - typedef typename Engine::result_type base_result; - result_type factor = result_type(1) / - (result_type(base_result((eng.max)()-(eng.min)())) + - result_type(std::numeric_limits::is_integer ? 1 : 0)); - result_type result = result_type(base_result(eng() - (eng.min)())) * factor; - if (result < result_type(1)) - return result; - } - } - -#ifndef BOOST_RANDOM_NO_STREAM_OPERATORS - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, const new_uniform_01&) - { - return os; - } - - template - friend std::basic_istream& - operator>>(std::basic_istream& is, new_uniform_01&) - { - return is; - } -#endif -}; - -template -class backward_compatible_uniform_01 -{ - typedef boost::random::detail::ptr_helper traits; -public: - typedef UniformRandomNumberGenerator base_type; - typedef RealType result_type; - - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - -#if !defined(BOOST_NO_LIMITS_COMPILE_TIME_CONSTANTS) - BOOST_STATIC_ASSERT(!std::numeric_limits::is_integer); -#endif - - explicit backward_compatible_uniform_01(typename traits::rvalue_type rng) - : _rng(rng), - _factor(result_type(1) / - (result_type((base().max)()-(base().min)()) + - result_type(std::numeric_limits::is_integer ? 1 : 0))) - { - } - // compiler-generated copy ctor and copy assignment are fine - - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return result_type(0); } - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return result_type(1); } - typename traits::value_type& base() { return traits::ref(_rng); } - const typename traits::value_type& base() const { return traits::ref(_rng); } - void reset() { } - - result_type operator()() { - for (;;) { - result_type result = result_type(base()() - (base().min)()) * _factor; - if (result < result_type(1)) - return result; - } - } - -#if !defined(BOOST_NO_OPERATORS_IN_NAMESPACE) && !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, const backward_compatible_uniform_01& u) - { - os << u._rng; - return os; - } - - template - friend std::basic_istream& - operator>>(std::basic_istream& is, backward_compatible_uniform_01& u) - { - is >> u._rng; - return is; - } -#endif - -private: - typedef typename traits::value_type::result_type base_result; - UniformRandomNumberGenerator _rng; - result_type _factor; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool backward_compatible_uniform_01::has_fixed_range; -#endif - -template::is_specialized> -struct select_uniform_01 -{ - template - struct apply - { - typedef backward_compatible_uniform_01 type; - }; -}; - -template -struct select_uniform_01 -{ - template - struct apply - { - typedef new_uniform_01 type; - }; -}; - -} - -// Because it is so commonly used: uniform distribution on the real [0..1) -// range. This allows for specializations to avoid a costly int -> float -// conversion plus float multiplication -template -class uniform_01 - : public detail::select_uniform_01::BOOST_NESTED_TEMPLATE apply::type -{ - typedef typename detail::select_uniform_01::BOOST_NESTED_TEMPLATE apply::type impl_type; - typedef boost::random::detail::ptr_helper traits; -public: - - uniform_01() {} - - explicit uniform_01(typename traits::rvalue_type rng) - : impl_type(rng) - { - } - -#if !defined(BOOST_NO_OPERATORS_IN_NAMESPACE) && !defined(BOOST_NO_MEMBER_TEMPLATE_FRIENDS) - template - friend std::basic_ostream& - operator<<(std::basic_ostream& os, const uniform_01& u) - { - os << static_cast(u); - return os; - } - - template - friend std::basic_istream& - operator>>(std::basic_istream& is, uniform_01& u) - { - is >> static_cast(u); - return is; - } -#endif -}; - -#endif - -} // namespace random - -using random::uniform_01; - -} // namespace boost - -#include - -#endif // BOOST_RANDOM_UNIFORM_01_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/uniform_int.hpp b/contrib/libboost/boost_1_65_0/boost/random/uniform_int.hpp deleted file mode 100644 index 4362652593d..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/uniform_int.hpp +++ /dev/null @@ -1,99 +0,0 @@ -/* boost random/uniform_int.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-04-08 added min -#include - -namespace boost { - -/** - * The distribution function uniform_int models a \random_distribution. - * On each invocation, it returns a random integer value uniformly - * distributed in the set of integer numbers {min, min+1, min+2, ..., max}. - * - * The template parameter IntType shall denote an integer-like value type. - * - * This class is deprecated. Please use @c uniform_int_distribution in - * new code. - */ -template -class uniform_int : public random::uniform_int_distribution -{ - typedef random::uniform_int_distribution base_type; -public: - - class param_type : public base_type::param_type - { - public: - typedef uniform_int distribution_type; - /** - * Constructs the parameters of a uniform_int distribution. - * - * Requires: min <= max - */ - explicit param_type(IntType min_arg = 0, IntType max_arg = 9) - : base_type::param_type(min_arg, max_arg) - {} - }; - - /** - * Constructs a uniform_int object. @c min and @c max are - * the parameters of the distribution. - * - * Requires: min <= max - */ - explicit uniform_int(IntType min_arg = 0, IntType max_arg = 9) - : base_type(min_arg, max_arg) - {} - - /** Constructs a uniform_int distribution from its parameters. */ - explicit uniform_int(const param_type& parm) - : base_type(parm) - {} - - /** Returns the parameters of the distribution */ - param_type param() const { return param_type(this->a(), this->b()); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) { this->base_type::param(parm); } - - // Codergear seems to have trouble with a using declaration here - - template - IntType operator()(Engine& eng) const - { - return static_cast(*this)(eng); - } - - template - IntType operator()(Engine& eng, const param_type& parm) const - { - return static_cast(*this)(eng, parm); - } - - template - IntType operator()(Engine& eng, IntType n) const - { - BOOST_ASSERT(n > 0); - return static_cast(*this)(eng, param_type(0, n - 1)); - } -}; - -} // namespace boost - -#endif // BOOST_RANDOM_UNIFORM_INT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/uniform_int_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/uniform_int_distribution.hpp deleted file mode 100644 index e0d3a9bebcd..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/uniform_int_distribution.hpp +++ /dev/null @@ -1,419 +0,0 @@ -/* boost random/uniform_int_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-04-08 added min -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#ifdef BOOST_NO_CXX11_EXPLICIT_CONVERSION_OPERATORS -#include -#endif - -namespace boost { -namespace random { -namespace detail { - - -#ifdef BOOST_MSVC -#pragma warning(push) -// disable division by zero warning, since we can't -// actually divide by zero. -#pragma warning(disable:4723) -#endif - -template -T generate_uniform_int( - Engine& eng, T min_value, T max_value, - boost::mpl::true_ /** is_integral */) -{ - typedef T result_type; - typedef typename boost::random::traits::make_unsigned_or_unbounded::type range_type; - typedef typename Engine::result_type base_result; - // ranges are always unsigned or unbounded - typedef typename boost::random::traits::make_unsigned_or_unbounded::type base_unsigned; - const range_type range = random::detail::subtract()(max_value, min_value); - const base_result bmin = (eng.min)(); - const base_unsigned brange = - random::detail::subtract()((eng.max)(), (eng.min)()); - - if(range == 0) { - return min_value; - } else if(brange == range) { - // this will probably never happen in real life - // basically nothing to do; just take care we don't overflow / underflow - base_unsigned v = random::detail::subtract()(eng(), bmin); - return random::detail::add()(v, min_value); - } else if(brange < range) { - // use rejection method to handle things like 0..3 --> 0..4 - for(;;) { - // concatenate several invocations of the base RNG - // take extra care to avoid overflows - - // limit == floor((range+1)/(brange+1)) - // Therefore limit*(brange+1) <= range+1 - range_type limit; - if(range == (std::numeric_limits::max)()) { - limit = range/(range_type(brange)+1); - if(range % (range_type(brange)+1) == range_type(brange)) - ++limit; - } else { - limit = (range+1)/(range_type(brange)+1); - } - - // We consider "result" as expressed to base (brange+1): - // For every power of (brange+1), we determine a random factor - range_type result = range_type(0); - range_type mult = range_type(1); - - // loop invariants: - // result < mult - // mult <= range - while(mult <= limit) { - // Postcondition: result <= range, thus no overflow - // - // limit*(brange+1)<=range+1 def. of limit (1) - // eng()-bmin<=brange eng() post. (2) - // and mult<=limit. loop condition (3) - // Therefore mult*(eng()-bmin+1)<=range+1 by (1),(2),(3) (4) - // Therefore mult*(eng()-bmin)+mult<=range+1 rearranging (4) (5) - // result(static_cast(random::detail::subtract()(eng(), bmin)) * mult); - - // equivalent to (mult * (brange+1)) == range+1, but avoids overflow. - if(mult * range_type(brange) == range - mult + 1) { - // The destination range is an integer power of - // the generator's range. - return(result); - } - - // Postcondition: mult <= range - // - // limit*(brange+1)<=range+1 def. of limit (1) - // mult<=limit loop condition (2) - // Therefore mult*(brange+1)<=range+1 by (1), (2) (3) - // mult*(brange+1)!=range+1 preceding if (4) - // Therefore mult*(brange+1) limit loop condition (1) - // Suppose range/mult >= brange+1 Assumption (2) - // range >= mult*(brange+1) by (2) (3) - // range+1 > mult*(brange+1) by (3) (4) - // range+1 > (limit+1)*(brange+1) by (1), (4) (5) - // (range+1)/(brange+1) > limit+1 by (5) (6) - // limit < floor((range+1)/(brange+1)) by (6) (7) - // limit==floor((range+1)/(brange+1)) def. of limit (8) - // not (2) reductio (9) - // - // loop postcondition: (range/mult)*mult+(mult-1) >= range - // - // (range/mult)*mult + range%mult == range identity (1) - // range%mult < mult def. of % (2) - // (range/mult)*mult+mult > range by (1), (2) (3) - // (range/mult)*mult+(mult-1) >= range by (3) (4) - // - // Note that the maximum value of result at this point is (mult-1), - // so after this final step, we generate numbers that can be - // at least as large as range. We have to really careful to avoid - // overflow in this final addition and in the rejection. Anything - // that overflows is larger than range and can thus be rejected. - - // range/mult < brange+1 -> no endless loop - range_type result_increment = - generate_uniform_int( - eng, - static_cast(0), - static_cast(range/mult), - boost::mpl::true_()); - if(std::numeric_limits::is_bounded && ((std::numeric_limits::max)() / mult < result_increment)) { - // The multiplcation would overflow. Reject immediately. - continue; - } - result_increment *= mult; - // unsigned integers are guaranteed to wrap on overflow. - result += result_increment; - if(result < result_increment) { - // The addition overflowed. Reject. - continue; - } - if(result > range) { - // Too big. Reject. - continue; - } - return random::detail::add()(result, min_value); - } - } else { // brange > range -#ifdef BOOST_NO_CXX11_EXPLICIT_CONVERSION_OPERATORS - typedef typename mpl::if_c< - std::numeric_limits::is_specialized && std::numeric_limits::is_specialized - && (std::numeric_limits::digits >= std::numeric_limits::digits), - range_type, base_unsigned>::type mixed_range_type; -#else - typedef base_unsigned mixed_range_type; -#endif - - mixed_range_type bucket_size; - // it's safe to add 1 to range, as long as we cast it first, - // because we know that it is less than brange. However, - // we do need to be careful not to cause overflow by adding 1 - // to brange. We use mixed_range_type throughout for mixed - // arithmetic between base_unsigned and range_type - in the case - // that range_type has more bits than base_unsigned it is always - // safe to use range_type for this albeit it may be more effient - // to use base_unsigned. The latter is a narrowing conversion though - // which may be disallowed if range_type is a multiprecision type - // and there are no explicit converison operators. - - if(brange == (std::numeric_limits::max)()) { - bucket_size = static_cast(brange) / (static_cast(range)+1); - if(static_cast(brange) % (static_cast(range)+1) == static_cast(range)) { - ++bucket_size; - } - } else { - bucket_size = static_cast(brange + 1) / (static_cast(range)+1); - } - for(;;) { - mixed_range_type result = - random::detail::subtract()(eng(), bmin); - result /= bucket_size; - // result and range are non-negative, and result is possibly larger - // than range, so the cast is safe - if(result <= static_cast(range)) - return random::detail::add()(result, min_value); - } - } -} - -#ifdef BOOST_MSVC -#pragma warning(pop) -#endif - -template -inline T generate_uniform_int( - Engine& eng, T min_value, T max_value, - boost::mpl::false_ /** is_integral */) -{ - uniform_int_float wrapper(eng); - return generate_uniform_int(wrapper, min_value, max_value, boost::mpl::true_()); -} - -template -inline T generate_uniform_int(Engine& eng, T min_value, T max_value) -{ - typedef typename Engine::result_type base_result; - return generate_uniform_int(eng, min_value, max_value, - boost::random::traits::is_integral()); -} - -} - -/** - * The class template uniform_int_distribution models a \random_distribution. - * On each invocation, it returns a random integer value uniformly - * distributed in the set of integers {min, min+1, min+2, ..., max}. - * - * The template parameter IntType shall denote an integer-like value type. - */ -template -class uniform_int_distribution -{ -public: - typedef IntType input_type; - typedef IntType result_type; - - class param_type - { - public: - - typedef uniform_int_distribution distribution_type; - - /** - * Constructs the parameters of a uniform_int_distribution. - * - * Requires min <= max - */ - explicit param_type( - IntType min_arg = 0, - IntType max_arg = (std::numeric_limits::max)()) - : _min(min_arg), _max(max_arg) - { - BOOST_ASSERT(_min <= _max); - } - - /** Returns the minimum value of the distribution. */ - IntType a() const { return _min; } - /** Returns the maximum value of the distribution. */ - IntType b() const { return _max; } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._min << " " << parm._max; - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - IntType min_in, max_in; - if(is >> min_in >> std::ws >> max_in) { - if(min_in <= max_in) { - parm._min = min_in; - parm._max = max_in; - } else { - is.setstate(std::ios_base::failbit); - } - } - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._min == rhs._min && lhs._max == rhs._max; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - - IntType _min; - IntType _max; - }; - - /** - * Constructs a uniform_int_distribution. @c min and @c max are - * the parameters of the distribution. - * - * Requires: min <= max - */ - explicit uniform_int_distribution( - IntType min_arg = 0, - IntType max_arg = (std::numeric_limits::max)()) - : _min(min_arg), _max(max_arg) - { - BOOST_ASSERT(min_arg <= max_arg); - } - /** Constructs a uniform_int_distribution from its parameters. */ - explicit uniform_int_distribution(const param_type& parm) - : _min(parm.a()), _max(parm.b()) {} - - /** Returns the minimum value of the distribution */ - IntType min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _min; } - /** Returns the maximum value of the distribution */ - IntType max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _max; } - - /** Returns the minimum value of the distribution */ - IntType a() const { return _min; } - /** Returns the maximum value of the distribution */ - IntType b() const { return _max; } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_min, _max); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _min = parm.a(); - _max = parm.b(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Returns an integer uniformly distributed in the range [min, max]. */ - template - result_type operator()(Engine& eng) const - { return detail::generate_uniform_int(eng, _min, _max); } - - /** - * Returns an integer uniformly distributed in the range - * [param.a(), param.b()]. - */ - template - result_type operator()(Engine& eng, const param_type& parm) const - { return detail::generate_uniform_int(eng, parm.a(), parm.b()); } - - /** Writes the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, uniform_int_distribution, ud) - { - os << ud.param(); - return os; - } - - /** Reads the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, uniform_int_distribution, ud) - { - param_type parm; - if(is >> parm) { - ud.param(parm); - } - return is; - } - - /** - * Returns true if the two distributions will produce identical sequences - * of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(uniform_int_distribution, lhs, rhs) - { return lhs._min == rhs._min && lhs._max == rhs._max; } - - /** - * Returns true if the two distributions may produce different sequences - * of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(uniform_int_distribution) - -private: - IntType _min; - IntType _max; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_UNIFORM_INT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/uniform_on_sphere.hpp b/contrib/libboost/boost_1_65_0/boost/random/uniform_on_sphere.hpp deleted file mode 100644 index 72c25ef7814..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/uniform_on_sphere.hpp +++ /dev/null @@ -1,284 +0,0 @@ -/* boost random/uniform_on_sphere.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-02-18 moved to individual header files - */ - -#ifndef BOOST_RANDOM_UNIFORM_ON_SPHERE_HPP -#define BOOST_RANDOM_UNIFORM_ON_SPHERE_HPP - -#include -#include // std::transform -#include // std::bind2nd, std::divides -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * Instantiations of class template uniform_on_sphere model a - * \random_distribution. Such a distribution produces random - * numbers uniformly distributed on the unit sphere of arbitrary - * dimension @c dim. The @c Cont template parameter must be a STL-like - * container type with begin and end operations returning non-const - * ForwardIterators of type @c Cont::iterator. - */ -template > -class uniform_on_sphere -{ -public: - typedef RealType input_type; - typedef Cont result_type; - - class param_type - { - public: - - typedef uniform_on_sphere distribution_type; - - /** - * Constructs the parameters of a uniform_on_sphere - * distribution, given the dimension of the sphere. - */ - explicit param_type(int dim_arg = 2) : _dim(dim_arg) - { - BOOST_ASSERT(_dim >= 0); - } - - /** Returns the dimension of the sphere. */ - int dim() const { return _dim; } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._dim; - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - is >> parm._dim; - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._dim == rhs._dim; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - int _dim; - }; - - /** - * Constructs a @c uniform_on_sphere distribution. - * @c dim is the dimension of the sphere. - * - * Requires: dim >= 0 - */ - explicit uniform_on_sphere(int dim_arg = 2) - : _container(dim_arg), _dim(dim_arg) { } - - /** - * Constructs a @c uniform_on_sphere distribution from its parameters. - */ - explicit uniform_on_sphere(const param_type& parm) - : _container(parm.dim()), _dim(parm.dim()) { } - - // compiler-generated copy ctor and assignment operator are fine - - /** Returns the dimension of the sphere. */ - int dim() const { return _dim; } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_dim); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _dim = parm.dim(); - _container.resize(_dim); - } - - /** - * Returns the smallest value that the distribution can produce. - * Note that this is required to approximate the standard library's - * requirements. The behavior is defined according to lexicographical - * comparison so that for a container type of std::vector, - * dist.min() <= x <= dist.max() where x is any value produced - * by the distribution. - */ - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const - { - result_type result(_dim); - if(_dim != 0) { - result.front() = RealType(-1.0); - } - return result; - } - /** - * Returns the largest value that the distribution can produce. - * Note that this is required to approximate the standard library's - * requirements. The behavior is defined according to lexicographical - * comparison so that for a container type of std::vector, - * dist.min() <= x <= dist.max() where x is any value produced - * by the distribution. - */ - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { - result_type result(_dim); - if(_dim != 0) { - result.front() = RealType(1.0); - } - return result; - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() {} - - /** - * Returns a point uniformly distributed over the surface of - * a sphere of dimension dim(). - */ - template - const result_type & operator()(Engine& eng) - { - using std::sqrt; - switch(_dim) - { - case 0: break; - case 1: - { - if(uniform_01()(eng) < 0.5) { - *_container.begin() = -1; - } else { - *_container.begin() = 1; - } - break; - } - case 2: - { - uniform_01 uniform; - RealType sqsum; - RealType x, y; - do { - x = uniform(eng) * 2 - 1; - y = uniform(eng) * 2 - 1; - sqsum = x*x + y*y; - } while(sqsum == 0 || sqsum > 1); - RealType mult = 1/sqrt(sqsum); - typename Cont::iterator iter = _container.begin(); - *iter = x * mult; - iter++; - *iter = y * mult; - break; - } - case 3: - { - uniform_01 uniform; - RealType sqsum; - RealType x, y; - do { - x = uniform(eng) * 2 - 1; - y = uniform(eng) * 2 - 1; - sqsum = x*x + y*y; - } while(sqsum > 1); - RealType mult = 2 * sqrt(1 - sqsum); - typename Cont::iterator iter = _container.begin(); - *iter = x * mult; - ++iter; - *iter = y * mult; - ++iter; - *iter = 2 * sqsum - 1; - break; - } - default: - { - detail::unit_normal_distribution normal; - RealType sqsum; - do { - sqsum = 0; - for(typename Cont::iterator it = _container.begin(); - it != _container.end(); - ++it) { - RealType val = normal(eng); - *it = val; - sqsum += val * val; - } - } while(sqsum == 0); - // for all i: result[i] /= sqrt(sqsum) - std::transform(_container.begin(), _container.end(), _container.begin(), - std::bind2nd(std::multiplies(), 1/sqrt(sqsum))); - } - } - return _container; - } - - /** - * Returns a point uniformly distributed over the surface of - * a sphere of dimension param.dim(). - */ - template - result_type operator()(Engine& eng, const param_type& parm) const - { - return uniform_on_sphere(parm)(eng); - } - - /** Writes the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, uniform_on_sphere, sd) - { - os << sd._dim; - return os; - } - - /** Reads the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, uniform_on_sphere, sd) - { - is >> sd._dim; - sd._container.resize(sd._dim); - return is; - } - - /** - * Returns true if the two distributions will produce identical - * sequences of values, given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(uniform_on_sphere, lhs, rhs) - { return lhs._dim == rhs._dim; } - - /** - * Returns true if the two distributions may produce different - * sequences of values, given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(uniform_on_sphere) - -private: - result_type _container; - int _dim; -}; - -} // namespace random - -using random::uniform_on_sphere; - -} // namespace boost - -#endif // BOOST_RANDOM_UNIFORM_ON_SPHERE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/uniform_real.hpp b/contrib/libboost/boost_1_65_0/boost/random/uniform_real.hpp deleted file mode 100644 index c98626e7792..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/uniform_real.hpp +++ /dev/null @@ -1,82 +0,0 @@ -/* boost random/uniform_real.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-04-08 added min -#include -#include -#include - -namespace boost { - -/** - * The distribution function uniform_real models a random distribution. - * On each invocation, it returns a random floating-point value uniformly - * distributed in the range [min..max). - * - * This class is deprecated. Please use @c uniform_real_distribution in - * new code. - */ -template -class uniform_real : public random::uniform_real_distribution -{ - typedef random::uniform_real_distribution base_type; -public: - - class param_type : public base_type::param_type - { - public: - typedef uniform_real distribution_type; - /** - * Constructs the parameters of a uniform_real distribution. - * - * Requires: min <= max - */ - explicit param_type(RealType min_arg = RealType(0.0), - RealType max_arg = RealType(1.0)) - : base_type::param_type(min_arg, max_arg) - {} - }; - - /** - * Constructs a uniform_real object. @c min and @c max are the - * parameters of the distribution. - * - * Requires: min <= max - */ - explicit uniform_real(RealType min_arg = RealType(0.0), - RealType max_arg = RealType(1.0)) - : base_type(min_arg, max_arg) - { - BOOST_ASSERT(min_arg < max_arg); - } - - /** Constructs a uniform_real distribution from its parameters. */ - explicit uniform_real(const param_type& parm) - : base_type(parm) - {} - - /** Returns the parameters of the distribution */ - param_type param() const { return param_type(this->a(), this->b()); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) { this->base_type::param(parm); } -}; - -} // namespace boost - -#endif // BOOST_RANDOM_UNIFORM_REAL_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/uniform_real_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/uniform_real_distribution.hpp deleted file mode 100644 index 820fce9314c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/uniform_real_distribution.hpp +++ /dev/null @@ -1,241 +0,0 @@ -/* boost random/uniform_real_distribution.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_UNIFORM_REAL_DISTRIBUTION_HPP -#define BOOST_RANDOM_UNIFORM_REAL_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { -namespace detail { - -template -T generate_uniform_real( - Engine& eng, T min_value, T max_value, - boost::mpl::false_ /** is_integral */) -{ - for(;;) { - typedef T result_type; - result_type numerator = static_cast(eng() - (eng.min)()); - result_type divisor = static_cast((eng.max)() - (eng.min)()); - BOOST_ASSERT(divisor > 0); - BOOST_ASSERT(numerator >= 0 && numerator <= divisor); - T result = numerator / divisor * (max_value - min_value) + min_value; - if(result < max_value) return result; - } -} - -template -T generate_uniform_real( - Engine& eng, T min_value, T max_value, - boost::mpl::true_ /** is_integral */) -{ - for(;;) { - typedef T result_type; - typedef typename Engine::result_type base_result; - result_type numerator = static_cast(subtract()(eng(), (eng.min)())); - result_type divisor = static_cast(subtract()((eng.max)(), (eng.min)())) + 1; - BOOST_ASSERT(divisor > 0); - BOOST_ASSERT(numerator >= 0 && numerator <= divisor); - T result = numerator / divisor * (max_value - min_value) + min_value; - if(result < max_value) return result; - } -} - -template -inline T generate_uniform_real(Engine& eng, T min_value, T max_value) -{ - if(max_value / 2 - min_value / 2 > (std::numeric_limits::max)() / 2) - return 2 * generate_uniform_real(eng, T(min_value / 2), T(max_value / 2)); - typedef typename Engine::result_type base_result; - return generate_uniform_real(eng, min_value, max_value, - boost::is_integral()); -} - -} - -/** - * The class template uniform_real_distribution models a \random_distribution. - * On each invocation, it returns a random floating-point value uniformly - * distributed in the range [min..max). - */ -template -class uniform_real_distribution -{ -public: - typedef RealType input_type; - typedef RealType result_type; - - class param_type - { - public: - - typedef uniform_real_distribution distribution_type; - - /** - * Constructs the parameters of a uniform_real_distribution. - * - * Requires min <= max - */ - explicit param_type(RealType min_arg = RealType(0.0), - RealType max_arg = RealType(1.0)) - : _min(min_arg), _max(max_arg) - { - BOOST_ASSERT(_min < _max); - } - - /** Returns the minimum value of the distribution. */ - RealType a() const { return _min; } - /** Returns the maximum value of the distribution. */ - RealType b() const { return _max; } - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._min << " " << parm._max; - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - RealType min_in, max_in; - if(is >> min_in >> std::ws >> max_in) { - if(min_in <= max_in) { - parm._min = min_in; - parm._max = max_in; - } else { - is.setstate(std::ios_base::failbit); - } - } - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._min == rhs._min && lhs._max == rhs._max; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - - RealType _min; - RealType _max; - }; - - /** - * Constructs a uniform_real_distribution. @c min and @c max are - * the parameters of the distribution. - * - * Requires: min <= max - */ - explicit uniform_real_distribution( - RealType min_arg = RealType(0.0), - RealType max_arg = RealType(1.0)) - : _min(min_arg), _max(max_arg) - { - BOOST_ASSERT(min_arg < max_arg); - } - /** Constructs a uniform_real_distribution from its parameters. */ - explicit uniform_real_distribution(const param_type& parm) - : _min(parm.a()), _max(parm.b()) {} - - /** Returns the minimum value of the distribution */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _min; } - /** Returns the maximum value of the distribution */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _max; } - - /** Returns the minimum value of the distribution */ - RealType a() const { return _min; } - /** Returns the maximum value of the distribution */ - RealType b() const { return _max; } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_min, _max); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _min = parm.a(); - _max = parm.b(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Returns a value uniformly distributed in the range [min, max). */ - template - result_type operator()(Engine& eng) const - { return detail::generate_uniform_real(eng, _min, _max); } - - /** - * Returns a value uniformly distributed in the range - * [param.a(), param.b()). - */ - template - result_type operator()(Engine& eng, const param_type& parm) const - { return detail::generate_uniform_real(eng, parm.a(), parm.b()); } - - /** Writes the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, uniform_real_distribution, ud) - { - os << ud.param(); - return os; - } - - /** Reads the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, uniform_real_distribution, ud) - { - param_type parm; - if(is >> parm) { - ud.param(parm); - } - return is; - } - - /** - * Returns true if the two distributions will produce identical sequences - * of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(uniform_real_distribution, lhs, rhs) - { return lhs._min == rhs._min && lhs._max == rhs._max; } - - /** - * Returns true if the two distributions may produce different sequences - * of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(uniform_real_distribution) - -private: - RealType _min; - RealType _max; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_UNIFORM_INT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/uniform_smallint.hpp b/contrib/libboost/boost_1_65_0/boost/random/uniform_smallint.hpp deleted file mode 100644 index c1afd7fbcdf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/uniform_smallint.hpp +++ /dev/null @@ -1,307 +0,0 @@ -/* boost random/uniform_smallint.hpp header file - * - * Copyright Jens Maurer 2000-2001 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - * Revision history - * 2001-04-08 added min -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#ifdef BOOST_NO_CXX11_EXPLICIT_CONVERSION_OPERATORS -#include -#endif - -namespace boost { -namespace random { - -// uniform integer distribution on a small range [min, max] - -/** - * The distribution function uniform_smallint models a \random_distribution. - * On each invocation, it returns a random integer value uniformly distributed - * in the set of integer numbers {min, min+1, min+2, ..., max}. It assumes - * that the desired range (max-min+1) is small compared to the range of the - * underlying source of random numbers and thus makes no attempt to limit - * quantization errors. - * - * Let \f$r_{\mathtt{out}} = (\mbox{max}-\mbox{min}+1)\f$ the desired range of - * integer numbers, and - * let \f$r_{\mathtt{base}}\f$ be the range of the underlying source of random - * numbers. Then, for the uniform distribution, the theoretical probability - * for any number i in the range \f$r_{\mathtt{out}}\f$ will be - * \f$\displaystyle p_{\mathtt{out}}(i) = \frac{1}{r_{\mathtt{out}}}\f$. - * Likewise, assume a uniform distribution on \f$r_{\mathtt{base}}\f$ for - * the underlying source of random numbers, i.e. - * \f$\displaystyle p_{\mathtt{base}}(i) = \frac{1}{r_{\mathtt{base}}}\f$. - * Let \f$p_{\mathtt{out\_s}}(i)\f$ denote the random - * distribution generated by @c uniform_smallint. Then the sum over all - * i in \f$r_{\mathtt{out}}\f$ of - * \f$\displaystyle - * \left(\frac{p_{\mathtt{out\_s}}(i)}{p_{\mathtt{out}}(i)} - 1\right)^2\f$ - * shall not exceed - * \f$\displaystyle \frac{r_{\mathtt{out}}}{r_{\mathtt{base}}^2} - * (r_{\mathtt{base}} \mbox{ mod } r_{\mathtt{out}}) - * (r_{\mathtt{out}} - r_{\mathtt{base}} \mbox{ mod } r_{\mathtt{out}})\f$. - * - * The template parameter IntType shall denote an integer-like value type. - * - * @xmlnote - * The property above is the square sum of the relative differences - * in probabilities between the desired uniform distribution - * \f$p_{\mathtt{out}}(i)\f$ and the generated distribution - * \f$p_{\mathtt{out\_s}}(i)\f$. - * The property can be fulfilled with the calculation - * \f$(\mbox{base\_rng} \mbox{ mod } r_{\mathtt{out}})\f$, as follows: - * Let \f$r = r_{\mathtt{base}} \mbox{ mod } r_{\mathtt{out}}\f$. - * The base distribution on \f$r_{\mathtt{base}}\f$ is folded onto the - * range \f$r_{\mathtt{out}}\f$. The numbers i < r have assigned - * \f$\displaystyle - * \left\lfloor\frac{r_{\mathtt{base}}}{r_{\mathtt{out}}}\right\rfloor+1\f$ - * numbers of the base distribution, the rest has only \f$\displaystyle - * \left\lfloor\frac{r_{\mathtt{base}}}{r_{\mathtt{out}}}\right\rfloor\f$. - * Therefore, - * \f$\displaystyle p_{\mathtt{out\_s}}(i) = - * \left(\left\lfloor\frac{r_{\mathtt{base}}} - * {r_{\mathtt{out}}}\right\rfloor+1\right) / - * r_{\mathtt{base}}\f$ for i < r and \f$\displaystyle p_{\mathtt{out\_s}}(i) = - * \left\lfloor\frac{r_{\mathtt{base}}} - * {r_{\mathtt{out}}}\right\rfloor/r_{\mathtt{base}}\f$ otherwise. - * Substituting this in the - * above sum formula leads to the desired result. - * @endxmlnote - * - * Note: The upper bound for - * \f$(r_{\mathtt{base}} \mbox{ mod } r_{\mathtt{out}}) - * (r_{\mathtt{out}} - r_{\mathtt{base}} \mbox{ mod } r_{\mathtt{out}})\f$ is - * \f$\displaystyle \frac{r_{\mathtt{out}}^2}{4}\f$. Regarding the upper bound - * for the square sum of the relative quantization error of - * \f$\displaystyle \frac{r_\mathtt{out}^3}{4r_{\mathtt{base}}^2}\f$, it - * seems wise to either choose \f$r_{\mathtt{base}}\f$ so that - * \f$r_{\mathtt{base}} > 10r_{\mathtt{out}}^2\f$ or ensure that - * \f$r_{\mathtt{base}}\f$ is - * divisible by \f$r_{\mathtt{out}}\f$. - */ -template -class uniform_smallint -{ -public: - typedef IntType input_type; - typedef IntType result_type; - - class param_type - { - public: - - typedef uniform_smallint distribution_type; - - /** constructs the parameters of a @c uniform_smallint distribution. */ - param_type(IntType min_arg = 0, IntType max_arg = 9) - : _min(min_arg), _max(max_arg) - { - BOOST_ASSERT(_min <= _max); - } - - /** Returns the minimum value. */ - IntType a() const { return _min; } - /** Returns the maximum value. */ - IntType b() const { return _max; } - - - /** Writes the parameters to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { - os << parm._min << " " << parm._max; - return os; - } - - /** Reads the parameters from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { - is >> parm._min >> std::ws >> parm._max; - return is; - } - - /** Returns true if the two sets of parameters are equal. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._min == rhs._min && lhs._max == rhs._max; } - - /** Returns true if the two sets of parameters are different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - IntType _min; - IntType _max; - }; - - /** - * Constructs a @c uniform_smallint. @c min and @c max are the - * lower and upper bounds of the output range, respectively. - */ - explicit uniform_smallint(IntType min_arg = 0, IntType max_arg = 9) - : _min(min_arg), _max(max_arg) {} - - /** - * Constructs a @c uniform_smallint from its parameters. - */ - explicit uniform_smallint(const param_type& parm) - : _min(parm.a()), _max(parm.b()) {} - - /** Returns the minimum value of the distribution. */ - result_type a() const { return _min; } - /** Returns the maximum value of the distribution. */ - result_type b() const { return _max; } - /** Returns the minimum value of the distribution. */ - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _min; } - /** Returns the maximum value of the distribution. */ - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return _max; } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_min, _max); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _min = parm.a(); - _max = parm.b(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Returns a value uniformly distributed in the range [min(), max()]. */ - template - result_type operator()(Engine& eng) const - { - typedef typename Engine::result_type base_result; - return generate(eng, boost::random::traits::is_integral()); - } - - /** Returns a value uniformly distributed in the range [param.a(), param.b()]. */ - template - result_type operator()(Engine& eng, const param_type& parm) const - { return uniform_smallint(parm)(eng); } - - /** Writes the distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, uniform_smallint, ud) - { - os << ud._min << " " << ud._max; - return os; - } - - /** Reads the distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, uniform_smallint, ud) - { - is >> ud._min >> std::ws >> ud._max; - return is; - } - - /** - * Returns true if the two distributions will produce identical - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(uniform_smallint, lhs, rhs) - { return lhs._min == rhs._min && lhs._max == rhs._max; } - - /** - * Returns true if the two distributions may produce different - * sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(uniform_smallint) - -private: - - // \cond show_private - template - result_type generate(Engine& eng, boost::mpl::true_) const - { - // equivalent to (eng() - eng.min()) % (_max - _min + 1) + _min, - // but guarantees no overflow. - typedef typename Engine::result_type base_result; - typedef typename boost::random::traits::make_unsigned::type base_unsigned; - typedef typename boost::random::traits::make_unsigned_or_unbounded::type range_type; -#ifdef BOOST_NO_CXX11_EXPLICIT_CONVERSION_OPERATORS - typedef typename mpl::if_c< - std::numeric_limits::is_specialized && std::numeric_limits::is_specialized - && (std::numeric_limits::digits >= std::numeric_limits::digits), - range_type, base_unsigned>::type mixed_range_type; -#else - typedef base_unsigned mixed_range_type; -#endif - range_type range = random::detail::subtract()(_max, _min); - base_unsigned base_range = - random::detail::subtract()((eng.max)(), (eng.min)()); - base_unsigned val = - random::detail::subtract()(eng(), (eng.min)()); - if(range >= base_range) { - return boost::random::detail::add()( - static_cast(val), _min); - } else { - // This involves mixed arithmetic between the base generators range - // type, and the result_type's range type. mixed_range_type is - // normally the same as base_unsigned which is the most efficient - // option, but requires a narrowing explcit cast if result_type - // is a multiprecision type. If no such casts are available then use - // multiprecision arithmetic throughout instead. - mixed_range_type modulus = static_cast(range)+1; - return boost::random::detail::add()( - static_cast(val) % modulus, _min); - } - } - - template - result_type generate(Engine& eng, boost::mpl::false_) const - { - typedef typename Engine::result_type base_result; - typedef typename boost::random::traits::make_unsigned::type range_type; - range_type range = random::detail::subtract()(_max, _min); - base_result val = boost::uniform_01()(eng); - // what is the worst that can possibly happen here? - // base_result may not be able to represent all the values in [0, range] - // exactly. If this happens, it will cause round off error and we - // won't be able to produce all the values in the range. We don't - // care about this because the user has already told us not to by - // using uniform_smallint. However, we do need to be careful - // to clamp the result, or floating point rounding can produce - // an out of range result. - range_type offset = static_cast(val * (static_cast(range) + 1)); - if(offset > range) return _max; - return boost::random::detail::add()(offset , _min); - } - // \endcond - - result_type _min; - result_type _max; -}; - -} // namespace random - -using random::uniform_smallint; - -} // namespace boost - -#endif // BOOST_RANDOM_UNIFORM_SMALLINT_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/variate_generator.hpp b/contrib/libboost/boost_1_65_0/boost/random/variate_generator.hpp deleted file mode 100644 index 6d5aac4e472..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/variate_generator.hpp +++ /dev/null @@ -1,122 +0,0 @@ -/* boost random/variate_generator.hpp header file - * - * Copyright Jens Maurer 2002 - * Copyright Steven Watanabe 2011 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_RANDOM_GENERATOR_HPP -#define BOOST_RANDOM_RANDOM_GENERATOR_HPP - -#include - -#include - -namespace boost { - -/// \cond hide_private_members - -namespace random { - -///\endcond - -/** - * A random variate generator is used to join a random number - * generator together with a random number distribution. - * Boost.Random provides a vast choice of \generators as well - * as \distributions. - * - * The argument for the template parameter Engine shall be of - * the form U, U&, or U*, where U models a - * \uniform_random_number_generator. Then, the member - * engine_value_type names U (not the pointer or reference to U). - * - * Specializations of @c variate_generator satisfy the - * requirements of CopyConstructible. They also satisfy the - * requirements of Assignable unless the template parameter - * Engine is of the form U&. - * - * The complexity of all functions specified in this section - * is constant. No function described in this section except - * the constructor throws an exception. - */ -template -class variate_generator -{ -private: - typedef boost::random::detail::ptr_helper helper_type; -public: - typedef typename helper_type::value_type engine_value_type; - typedef Engine engine_type; - typedef Distribution distribution_type; - typedef typename Distribution::result_type result_type; - - /** - * Constructs a @c variate_generator object with the associated - * \uniform_random_number_generator eng and the associated - * \random_distribution d. - * - * Throws: If and what the copy constructor of Engine or - * Distribution throws. - */ - variate_generator(Engine e, Distribution d) - : _eng(e), _dist(d) { } - - /** Returns: distribution()(engine()) */ - result_type operator()() { return _dist(engine()); } - /** - * Returns: distribution()(engine(), value). - */ - template - result_type operator()(const T& value) { return _dist(engine(), value); } - - /** - * Returns: A reference to the associated uniform random number generator. - */ - engine_value_type& engine() { return helper_type::ref(_eng); } - /** - * Returns: A reference to the associated uniform random number generator. - */ - const engine_value_type& engine() const { return helper_type::ref(_eng); } - - /** Returns: A reference to the associated \random_distribution. */ - distribution_type& distribution() { return _dist; } - /** - * Returns: A reference to the associated random distribution. - */ - const distribution_type& distribution() const { return _dist; } - - /** - * Precondition: distribution().min() is well-formed - * - * Returns: distribution().min() - */ - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return (distribution().min)(); } - /** - * Precondition: distribution().max() is well-formed - * - * Returns: distribution().max() - */ - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return (distribution().max)(); } - -private: - Engine _eng; - distribution_type _dist; -}; - -} // namespace random - -using random::variate_generator; - -} // namespace boost - -#include - -#endif // BOOST_RANDOM_RANDOM_GENERATOR_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/weibull_distribution.hpp b/contrib/libboost/boost_1_65_0/boost/random/weibull_distribution.hpp deleted file mode 100644 index 55e3c04903a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/weibull_distribution.hpp +++ /dev/null @@ -1,177 +0,0 @@ -/* boost random/weibull_distribution.hpp header file - * - * Copyright Steven Watanabe 2010 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - */ - -#ifndef BOOST_RANDOM_WEIBULL_DISTRIBUTION_HPP -#define BOOST_RANDOM_WEIBULL_DISTRIBUTION_HPP - -#include -#include -#include -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * The Weibull distribution is a real valued distribution with two - * parameters a and b, producing values >= 0. - * - * It has \f$\displaystyle p(x) = \frac{a}{b}\left(\frac{x}{b}\right)^{a-1}e^{-\left(\frac{x}{b}\right)^a}\f$. - */ -template -class weibull_distribution { -public: - typedef RealType result_type; - typedef RealType input_type; - - class param_type { - public: - typedef weibull_distribution distribution_type; - - /** - * Constructs a @c param_type from the "a" and "b" parameters - * of the distribution. - * - * Requires: a > 0 && b > 0 - */ - explicit param_type(RealType a_arg = 1.0, RealType b_arg = 1.0) - : _a(a_arg), _b(b_arg) - {} - - /** Returns the "a" parameter of the distribtuion. */ - RealType a() const { return _a; } - /** Returns the "b" parameter of the distribution. */ - RealType b() const { return _b; } - - /** Writes a @c param_type to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, param_type, parm) - { os << parm._a << ' ' << parm._b; return os; } - - /** Reads a @c param_type from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, param_type, parm) - { is >> parm._a >> std::ws >> parm._b; return is; } - - /** Returns true if the two sets of parameters are the same. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(param_type, lhs, rhs) - { return lhs._a == rhs._a && lhs._b == rhs._b; } - - /** Returns true if the two sets of parameters are the different. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(param_type) - - private: - RealType _a; - RealType _b; - }; - - /** - * Constructs a @c weibull_distribution from its "a" and "b" parameters. - * - * Requires: a > 0 && b > 0 - */ - explicit weibull_distribution(RealType a_arg = 1.0, RealType b_arg = 1.0) - : _a(a_arg), _b(b_arg) - {} - /** Constructs a @c weibull_distribution from its parameters. */ - explicit weibull_distribution(const param_type& parm) - : _a(parm.a()), _b(parm.b()) - {} - - /** - * Returns a random variate distributed according to the - * @c weibull_distribution. - */ - template - RealType operator()(URNG& urng) const - { - using std::pow; - using std::log; - return _b*pow(-log(1 - uniform_01()(urng)), 1/_a); - } - - /** - * Returns a random variate distributed accordint to the Weibull - * distribution with parameters specified by @c param. - */ - template - RealType operator()(URNG& urng, const param_type& parm) const - { - return weibull_distribution(parm)(urng); - } - - /** Returns the "a" parameter of the distribution. */ - RealType a() const { return _a; } - /** Returns the "b" parameter of the distribution. */ - RealType b() const { return _b; } - - /** Returns the smallest value that the distribution can produce. */ - RealType min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return 0; } - /** Returns the largest value that the distribution can produce. */ - RealType max BOOST_PREVENT_MACRO_SUBSTITUTION () const - { return std::numeric_limits::infinity(); } - - /** Returns the parameters of the distribution. */ - param_type param() const { return param_type(_a, _b); } - /** Sets the parameters of the distribution. */ - void param(const param_type& parm) - { - _a = parm.a(); - _b = parm.b(); - } - - /** - * Effects: Subsequent uses of the distribution do not depend - * on values produced by any engine prior to invoking reset. - */ - void reset() { } - - /** Writes a @c weibull_distribution to a @c std::ostream. */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, weibull_distribution, wd) - { - os << wd.param(); - return os; - } - - /** Reads a @c weibull_distribution from a @c std::istream. */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, weibull_distribution, wd) - { - param_type parm; - if(is >> parm) { - wd.param(parm); - } - return is; - } - - /** - * Returns true if the two instances of @c weibull_distribution will - * return identical sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(weibull_distribution, lhs, rhs) - { return lhs._a == rhs._a && lhs._b == rhs._b; } - - /** - * Returns true if the two instances of @c weibull_distribution will - * return different sequences of values given equal generators. - */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(weibull_distribution) - -private: - RealType _a; - RealType _b; -}; - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_WEIBULL_DISTRIBUTION_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/random/xor_combine.hpp b/contrib/libboost/boost_1_65_0/boost/random/xor_combine.hpp deleted file mode 100644 index fe8a58b40f1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/random/xor_combine.hpp +++ /dev/null @@ -1,208 +0,0 @@ -/* boost random/xor_combine.hpp header file - * - * Copyright Jens Maurer 2002 - * Distributed under the Boost Software License, Version 1.0. (See - * accompanying file LICENSE_1_0.txt or copy at - * http://www.boost.org/LICENSE_1_0.txt) - * - * See http://www.boost.org for most recent version including documentation. - * - * $Id$ - * - */ - -#ifndef BOOST_RANDOM_XOR_COMBINE_HPP -#define BOOST_RANDOM_XOR_COMBINE_HPP - -#include -#include -#include -#include // for std::min and std::max -#include -#include -#include // uint32_t -#include -#include -#include -#include - -namespace boost { -namespace random { - -/** - * Instantiations of @c xor_combine_engine model a - * \pseudo_random_number_generator. To produce its output it - * invokes each of the base generators, shifts their results - * and xors them together. - */ -template -class xor_combine_engine -{ -public: - typedef URNG1 base1_type; - typedef URNG2 base2_type; - typedef typename base1_type::result_type result_type; - - BOOST_STATIC_CONSTANT(bool, has_fixed_range = false); - BOOST_STATIC_CONSTANT(int, shift1 = s1); - BOOST_STATIC_CONSTANT(int, shift2 = s2); - - /** - * Constructors a @c xor_combine_engine by default constructing - * both base generators. - */ - xor_combine_engine() : _rng1(), _rng2() { } - - /** Constructs a @c xor_combine by copying two base generators. */ - xor_combine_engine(const base1_type & rng1, const base2_type & rng2) - : _rng1(rng1), _rng2(rng2) { } - - /** - * Constructs a @c xor_combine_engine, seeding both base generators - * with @c v. - * - * @xmlwarning - * The exact algorithm used by this function may change in the future. - * @endxmlwarning - */ - BOOST_RANDOM_DETAIL_ARITHMETIC_CONSTRUCTOR(xor_combine_engine, - result_type, v) - { seed(v); } - - /** - * Constructs a @c xor_combine_engine, seeding both base generators - * with values produced by @c seq. - */ - BOOST_RANDOM_DETAIL_SEED_SEQ_CONSTRUCTOR(xor_combine_engine, - SeedSeq, seq) - { seed(seq); } - - /** - * Constructs a @c xor_combine_engine, seeding both base generators - * with values from the iterator range [first, last) and changes - * first to point to the element after the last one used. If there - * are not enough elements in the range to seed both generators, - * throws @c std::invalid_argument. - */ - template xor_combine_engine(It& first, It last) - : _rng1(first, last), _rng2( /* advanced by other call */ first, last) { } - - /** Calls @c seed() for both base generators. */ - void seed() { _rng1.seed(); _rng2.seed(); } - - /** @c seeds both base generators with @c v. */ - BOOST_RANDOM_DETAIL_ARITHMETIC_SEED(xor_combine_engine, result_type, v) - { _rng1.seed(v); _rng2.seed(v); } - - /** @c seeds both base generators with values produced by @c seq. */ - BOOST_RANDOM_DETAIL_SEED_SEQ_SEED(xor_combine_engine, SeedSeq, seq) - { _rng1.seed(seq); _rng2.seed(seq); } - - /** - * seeds both base generators with values from the iterator - * range [first, last) and changes first to point to the element - * after the last one used. If there are not enough elements in - * the range to seed both generators, throws @c std::invalid_argument. - */ - template void seed(It& first, It last) - { - _rng1.seed(first, last); - _rng2.seed(first, last); - } - - /** Returns the first base generator. */ - const base1_type& base1() const { return _rng1; } - - /** Returns the second base generator. */ - const base2_type& base2() const { return _rng2; } - - /** Returns the next value of the generator. */ - result_type operator()() - { - return (_rng1() << s1) ^ (_rng2() << s2); - } - - /** Fills a range with random values */ - template - void generate(Iter first, Iter last) - { detail::generate_from_int(*this, first, last); } - - /** Advances the state of the generator by @c z. */ - void discard(boost::uintmax_t z) - { - _rng1.discard(z); - _rng2.discard(z); - } - - /** Returns the smallest value that the generator can produce. */ - static result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () { return (std::min)((URNG1::min)(), (URNG2::min)()); } - /** Returns the largest value that the generator can produce. */ - static result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () { return (std::max)((URNG1::min)(), (URNG2::max)()); } - - /** - * Writes the textual representation of the generator to a @c std::ostream. - */ - BOOST_RANDOM_DETAIL_OSTREAM_OPERATOR(os, xor_combine_engine, s) - { - os << s._rng1 << ' ' << s._rng2; - return os; - } - - /** - * Reads the textual representation of the generator from a @c std::istream. - */ - BOOST_RANDOM_DETAIL_ISTREAM_OPERATOR(is, xor_combine_engine, s) - { - is >> s._rng1 >> std::ws >> s._rng2; - return is; - } - - /** Returns true if the two generators will produce identical sequences. */ - BOOST_RANDOM_DETAIL_EQUALITY_OPERATOR(xor_combine_engine, x, y) - { return x._rng1 == y._rng1 && x._rng2 == y._rng2; } - - /** Returns true if the two generators will produce different sequences. */ - BOOST_RANDOM_DETAIL_INEQUALITY_OPERATOR(xor_combine_engine) - -private: - base1_type _rng1; - base2_type _rng2; -}; - -#ifndef BOOST_NO_INCLASS_MEMBER_INITIALIZATION -// A definition is required even for integral static constants -template -const bool xor_combine_engine::has_fixed_range; -template -const int xor_combine_engine::shift1; -template -const int xor_combine_engine::shift2; -#endif - -/// \cond show_private - -/** Provided for backwards compatibility. */ -template -class xor_combine : public xor_combine_engine -{ - typedef xor_combine_engine base_type; -public: - typedef typename base_type::result_type result_type; - xor_combine() {} - xor_combine(result_type val) : base_type(val) {} - template - xor_combine(It& first, It last) : base_type(first, last) {} - xor_combine(const URNG1 & rng1, const URNG2 & rng2) - : base_type(rng1, rng2) { } - - result_type min BOOST_PREVENT_MACRO_SUBSTITUTION () const { return (std::min)((this->base1().min)(), (this->base2().min)()); } - result_type max BOOST_PREVENT_MACRO_SUBSTITUTION () const { return (std::max)((this->base1().min)(), (this->base2().max)()); } -}; - -/// \endcond - -} // namespace random -} // namespace boost - -#endif // BOOST_RANDOM_XOR_COMBINE_HPP From a0f6f406c865374b5dc4348a9cbc74f12eef8637 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 03:06:06 +0300 Subject: [PATCH 078/108] Initial support for build with libc++ instead of libstdc++ (incomplete) [#CLICKHOUSE-3174]. --- CMakeLists.txt | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 2ec52c7f427..f3669a53afb 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -89,18 +89,25 @@ else () set (CXX11_ABI_FLAGS "") endif () -set (COMPILER_FLAGS "${COMPILER_FLAGS} ${CXX11_ABI_FLAGS}") +set (COMPILER_FLAGS "${COMPILER_FLAGS} ${CXX11_ABI_FLAGS}") option (PIPE "-pipe compiler option [less /tmp usage, more ram usage]" ON) if (PIPE) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -pipe") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -pipe") endif () include (cmake/test_cpu.cmake) option (ARCHNATIVE "Enable -march=native compiler flag" OFF) if (ARCHNATIVE) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") +endif () + +option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" OFF) +if (USE_LIBCXX) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -pthread") # NOTE: Why this is not the default and why this is needed only with libc++? + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") + set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=1") # More checks in debug build. endif () set (CMAKE_BUILD_COLOR_MAKEFILE ON) @@ -115,11 +122,18 @@ set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3") set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline") if (NOT APPLE AND NOT (CMAKE_CXX_COMPILER_ID STREQUAL "Clang" AND CMAKE_SYSTEM MATCHES "FreeBSD")) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -static-libstdc++") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc") + if (NOT USE_LIBCXX) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libstdc++") + endif () endif () if (NOT APPLE) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${GLIBC_COMPATIBILITY_LINK_FLAGS} ${CXX11_ABI_FLAGS}") + + if (USE_LIBCXX AND (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -stdlib=libc++ -Wl,-Bstatic -lc++ -lc++abi -Wl,-Bdynamic") + endif () endif () include (cmake/test_compiler.cmake) From 84c0470bcbfe1ab84b595d71655e6fb95e7c90bc Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 9 Sep 2017 03:32:34 +0300 Subject: [PATCH 079/108] cmake: ccache detect proper version (#1222) --- CMakeLists.txt | 8 ++------ cmake/find_ccache.cmake | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 6 deletions(-) create mode 100644 cmake/find_ccache.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index f3669a53afb..985e5bfc6e9 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -30,11 +30,7 @@ cmake_policy (SET CMP0012 NEW) # Don't dereference TRUE and FALSE # Write compile_commands.json set(CMAKE_EXPORT_COMPILE_COMMANDS 1) -find_program (CCACHE_FOUND ccache) -if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CMAKE_CXX_COMPILER MATCHES "ccache") - set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "ccache") - set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK "ccache") -endif () +include (cmake/find_ccache.cmake) if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") message (STATUS "CMAKE_BUILD_TYPE is not set, set to default = RELWITHDEBINFO") @@ -190,7 +186,7 @@ if (UNBUNDLED OR NOT (CMAKE_SYSTEM MATCHES "Linux" OR APPLE) OR ARCH_32) option (NO_WERROR "Disable -Werror compiler option" ON) endif () -message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES} MAKE_STATIC_LIBRARIES=${MAKE_STATIC_LIBRARIES} UNBUNDLED=${UNBUNDLED} CCACHE=${CCACHE_FOUND}") +message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES} MAKE_STATIC_LIBRARIES=${MAKE_STATIC_LIBRARIES} UNBUNDLED=${UNBUNDLED} CCACHE=${CCACHE_FOUND} ${CCACHE_VERSION}") include(GNUInstallDirs) diff --git a/cmake/find_ccache.cmake b/cmake/find_ccache.cmake new file mode 100644 index 00000000000..beb4e306156 --- /dev/null +++ b/cmake/find_ccache.cmake @@ -0,0 +1,15 @@ + +find_program (CCACHE_FOUND ccache) +if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CMAKE_CXX_COMPILER MATCHES "ccache") + execute_process(COMMAND ${CCACHE_FOUND} "-V" OUTPUT_VARIABLE CCACHE_VERSION) + string(REGEX REPLACE "ccache version ([0-9\\.]+).*" "\\1" CCACHE_VERSION ${CCACHE_VERSION} ) + string(COMPARE GREATER ${CCACHE_VERSION} "3.2.0" CCACHE_VERSION_OK) + + if (CCACHE_VERSION_OK) + #message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}") + set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) + set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) + else () + message(STATUS "Not using ${CCACHE_FOUND} ${CCACHE_VERSION} bug: https://bugzilla.samba.org/show_bug.cgi?id=8118") + endif () +endif () From c46c6b66d5f7b121221fdfd610e1cfb792f48191 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 03:36:53 +0300 Subject: [PATCH 080/108] Fixed static linking of libc++ (ugly) [#CLICKHOUSE-3174]. --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f3669a53afb..3220b59f77a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -132,7 +132,7 @@ if (NOT APPLE) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${GLIBC_COMPATIBILITY_LINK_FLAGS} ${CXX11_ABI_FLAGS}") if (USE_LIBCXX AND (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -stdlib=libc++ -Wl,-Bstatic -lc++ -lc++abi -Wl,-Bdynamic") + link_libraries (-Wl,-Bstatic -stdlib=libc++ -static-libstdc++ c++ c++abi -Wl,-Bdynamic) endif () endif () From cb70a5a77c53c4309d8aee4a1b6c3544a4404342 Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 9 Sep 2017 03:40:21 +0300 Subject: [PATCH 081/108] Try fix macos build (#1221) * Try fix macos build * Try fix macos build --- dbms/src/Dictionaries/LibraryDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index 5785f176fe9..a2e073594fc 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -85,7 +85,7 @@ bool dataToBlock(const void * data, Block & block) for (size_t row_n = 0; row_n < columns_received->data[col_n].size; ++row_n) { - columns[row_n]->insert(columns_received->data[col_n].data[row_n]); + columns[row_n]->insert(static_cast(columns_received->data[col_n].data[row_n])); } } return false; From 32d8d7d90803762ad227cea21e673b610cb08c97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 04:01:36 +0300 Subject: [PATCH 082/108] Fixed build with libc++ [#CLICKHOUSE-3174]. --- CMakeLists.txt | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index e4b719d3c73..87c4ef63e65 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -119,16 +119,14 @@ set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 if (NOT APPLE AND NOT (CMAKE_CXX_COMPILER_ID STREQUAL "Clang" AND CMAKE_SYSTEM MATCHES "FreeBSD")) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc") - if (NOT USE_LIBCXX) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libstdc++") - endif () + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libstdc++") endif () if (NOT APPLE) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${GLIBC_COMPATIBILITY_LINK_FLAGS} ${CXX11_ABI_FLAGS}") if (USE_LIBCXX AND (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")) - link_libraries (-Wl,-Bstatic -stdlib=libc++ -static-libstdc++ c++ c++abi -Wl,-Bdynamic) + link_libraries (-Wl,-Bstatic -stdlib=libc++ c++ c++abi -Wl,-Bdynamic) endif () endif () From 0e9681cdc281171deb4ba2bdb1edc5ceb66c697a Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 9 Sep 2017 04:04:36 +0300 Subject: [PATCH 083/108] Always use ccache for gcc (#1223) * Try fix macos build * Try fix macos build * Alvays use ccache for gcc --- cmake/find_ccache.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find_ccache.cmake b/cmake/find_ccache.cmake index beb4e306156..926002a44ae 100644 --- a/cmake/find_ccache.cmake +++ b/cmake/find_ccache.cmake @@ -5,7 +5,7 @@ if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CM string(REGEX REPLACE "ccache version ([0-9\\.]+).*" "\\1" CCACHE_VERSION ${CCACHE_VERSION} ) string(COMPARE GREATER ${CCACHE_VERSION} "3.2.0" CCACHE_VERSION_OK) - if (CCACHE_VERSION_OK) + if (CCACHE_VERSION_OK OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") #message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}") set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) From 92d7d6bd44907a080d4efb94b2cc25ace2e97930 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 05:12:53 +0300 Subject: [PATCH 084/108] Fixed error that reproduced only when using libc++ [#CLICKHOUSE-3174]. --- dbms/src/Interpreters/Aggregator.cpp | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index e4f544a3b05..cf97941c8c2 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1605,7 +1605,7 @@ void NO_INLINE Aggregator::mergeBucketImpl( /** Combines aggregation states together, turns them into blocks, and outputs streams. - * If the aggregation states are two-level, then it produces blocks strictly in order bucket_num. + * If the aggregation states are two-level, then it produces blocks strictly in order of 'bucket_num'. * (This is important for distributed processing.) * In doing so, it can handle different buckets in parallel, using up to `threads` threads. */ @@ -1636,6 +1636,16 @@ public: return res.str(); } + ~MergingAndConvertingBlockInputStream() + { + LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); + + /// We need to wait for threads to finish before destructor of 'parallel_merge_data', + /// because the threads access 'parallel_merge_data'. + if (parallel_merge_data) + parallel_merge_data->pool.wait(); + } + protected: Block readImpl() override { @@ -1732,19 +1742,13 @@ private: struct ParallelMergeData { - ThreadPool pool; std::map ready_blocks; std::exception_ptr exception; std::mutex mutex; std::condition_variable condvar; + ThreadPool pool; explicit ParallelMergeData(size_t threads) : pool(threads) {} - - ~ParallelMergeData() - { - LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); - pool.wait(); - } }; std::unique_ptr parallel_merge_data; From 4ba8d5a3c6655f7590678e3900d6abd24b6a76a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 06:30:31 +0300 Subject: [PATCH 085/108] Updated build instruction [#CLICKHOUSE-3174]. --- docs/en/development/build.rst | 30 +----------------------------- docs/ru/development/build.rst | 30 +----------------------------- 2 files changed, 2 insertions(+), 58 deletions(-) diff --git a/docs/en/development/build.rst b/docs/en/development/build.rst index 180689e2417..93bfbbe60ab 100644 --- a/docs/en/development/build.rst +++ b/docs/en/development/build.rst @@ -122,35 +122,7 @@ Install recent version of clang. Clang is embedded into ClickHouse package and used at runtime. Minimum version is 3.8.0. It is optional. - -You can build clang from sources: - -.. code-block:: bash - - cd .. - sudo apt-get install subversion - mkdir llvm - cd llvm - svn co http://llvm.org/svn/llvm-project/llvm/tags/RELEASE_400/final llvm - cd llvm/tools - svn co http://llvm.org/svn/llvm-project/cfe/tags/RELEASE_400/final clang - cd .. - cd projects/ - svn co http://llvm.org/svn/llvm-project/compiler-rt/tags/RELEASE_400/final compiler-rt - cd ../.. - mkdir build - cd build/ - cmake -D CMAKE_BUILD_TYPE:STRING=Release ../llvm - make -j $THREADS - sudo make install - hash clang - -Or install it from packages. On Ubuntu 16.04 or newer: - -.. code-block:: bash - - sudo apt-get install clang - +To install clang, look at ``utils/prepare-environment/install-clang.sh`` You may also build ClickHouse with clang for development purposes. For production releases, GCC is used. diff --git a/docs/ru/development/build.rst b/docs/ru/development/build.rst index 180689e2417..93bfbbe60ab 100644 --- a/docs/ru/development/build.rst +++ b/docs/ru/development/build.rst @@ -122,35 +122,7 @@ Install recent version of clang. Clang is embedded into ClickHouse package and used at runtime. Minimum version is 3.8.0. It is optional. - -You can build clang from sources: - -.. code-block:: bash - - cd .. - sudo apt-get install subversion - mkdir llvm - cd llvm - svn co http://llvm.org/svn/llvm-project/llvm/tags/RELEASE_400/final llvm - cd llvm/tools - svn co http://llvm.org/svn/llvm-project/cfe/tags/RELEASE_400/final clang - cd .. - cd projects/ - svn co http://llvm.org/svn/llvm-project/compiler-rt/tags/RELEASE_400/final compiler-rt - cd ../.. - mkdir build - cd build/ - cmake -D CMAKE_BUILD_TYPE:STRING=Release ../llvm - make -j $THREADS - sudo make install - hash clang - -Or install it from packages. On Ubuntu 16.04 or newer: - -.. code-block:: bash - - sudo apt-get install clang - +To install clang, look at ``utils/prepare-environment/install-clang.sh`` You may also build ClickHouse with clang for development purposes. For production releases, GCC is used. From 5a8f45c13189d92cc667ddaf20bbb8ec7577b5f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 06:46:18 +0300 Subject: [PATCH 086/108] Fixed insignificant error, shown by TSan [#CLICKHOUSE-2]. --- dbms/src/Common/LRUCache.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/LRUCache.h b/dbms/src/Common/LRUCache.h index 85be0d5d0e4..4899d8f35e7 100644 --- a/dbms/src/Common/LRUCache.h +++ b/dbms/src/Common/LRUCache.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -254,8 +255,8 @@ private: const Delay expiration_delay; mutable std::mutex mutex; - size_t hits = 0; - size_t misses = 0; + std::atomic hits {0}; + std::atomic misses {0}; WeightFunction weight_function; From 253893bf4da36cf0a5c97f293240c16b5633dbb0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 07:06:54 +0300 Subject: [PATCH 087/108] Fixed insignificant error, shown by TSan [#CLICKHOUSE-2]. --- dbms/src/Common/MemoryTracker.cpp | 12 ++++++------ dbms/src/Common/MemoryTracker.h | 5 +++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 72bbd983328..76695b1bd19 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -49,7 +49,7 @@ void MemoryTracker::alloc(Int64 size) { Int64 will_be = amount += size; - if (!next) + if (!next.load(std::memory_order_relaxed)) CurrentMetrics::add(metric, size); Int64 current_limit = limit.load(std::memory_order_relaxed); @@ -89,8 +89,8 @@ void MemoryTracker::alloc(Int64 size) if (will_be > peak.load(std::memory_order_relaxed)) /// Races doesn't matter. Could rewrite with CAS, but not worth. peak.store(will_be, std::memory_order_relaxed); - if (next) - next->alloc(size); + if (auto loaded_next = next.load(std::memory_order_relaxed)) + loaded_next->alloc(size); } @@ -108,8 +108,8 @@ void MemoryTracker::free(Int64 size) amount -= size_to_subtract; /// NOTE above code is not atomic. It's easy to fix. - if (next) - next->free(size); + if (auto loaded_next = next.load(std::memory_order_relaxed)) + loaded_next->free(size); else CurrentMetrics::sub(metric, size_to_subtract); } @@ -117,7 +117,7 @@ void MemoryTracker::free(Int64 size) void MemoryTracker::reset() { - if (!next) + if (!next.load(std::memory_order_relaxed)) CurrentMetrics::sub(metric, amount); amount.store(0, std::memory_order_relaxed); diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index c06fc33444e..acc8da1e8e2 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -26,7 +26,7 @@ class MemoryTracker /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. - MemoryTracker * next = nullptr; + std::atomic next {}; /// You could specify custom metric to track memory usage. CurrentMetrics::Metric metric = CurrentMetrics::MemoryTracking; @@ -78,9 +78,10 @@ public: fault_probability = value; } + /// next should be changed only once: from nullptr to some value. void setNext(MemoryTracker * elem) { - next = elem; + next.store(elem, std::memory_order_relaxed); } /// The memory consumption could be shown in realtime via CurrentMetrics counter From 2e74123075ebcc5612b693f806f9b7c577e602bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Sep 2017 07:08:04 +0300 Subject: [PATCH 088/108] Fixed insignificant error in Poco, shown by TSan [#CLICKHOUSE-2]. --- .../libpoco/Net/include/Poco/Net/TCPServer.h | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/contrib/libpoco/Net/include/Poco/Net/TCPServer.h b/contrib/libpoco/Net/include/Poco/Net/TCPServer.h index fccb658f77b..78fd718d451 100644 --- a/contrib/libpoco/Net/include/Poco/Net/TCPServer.h +++ b/contrib/libpoco/Net/include/Poco/Net/TCPServer.h @@ -28,6 +28,8 @@ #include "Poco/Thread.h" #include "Poco/ThreadPool.h" +#include + namespace Poco { namespace Net { @@ -63,7 +65,7 @@ class Net_API TCPServer: public Poco::Runnable /// on the number of connections waiting to be served. /// /// It is possible to specify a maximum number of queued connections. - /// This prevents the connection queue from overflowing in the + /// This prevents the connection queue from overflowing in the /// case of an extreme server load. In such a case, connections that /// cannot be queued are silently and immediately closed. /// @@ -121,7 +123,7 @@ public: const TCPServerParams& params() const; /// Returns a const reference to the TCPServerParam object - /// used by the server's TCPServerDispatcher. + /// used by the server's TCPServerDispatcher. void start(); /// Starts the server. A new thread will be @@ -138,7 +140,7 @@ public: /// Already handled connections will continue to be served. /// /// Once the server has been stopped, it cannot be restarted. - + int currentThreads() const; /// Returns the number of currently used connection threads. @@ -147,13 +149,13 @@ public: int totalConnections() const; /// Returns the total number of handled connections. - + int currentConnections() const; /// Returns the number of currently handled connections. int maxConcurrentConnections() const; - /// Returns the maximum number of concurrently handled connections. - + /// Returns the maximum number of concurrently handled connections. + int queuedConnections() const; /// Returns the number of queued connections. @@ -180,11 +182,11 @@ private: TCPServer(); TCPServer(const TCPServer&); TCPServer& operator = (const TCPServer&); - + ServerSocket _socket; TCPServerDispatcher* _pDispatcher; Poco::Thread _thread; - bool _stopped; + std::atomic _stopped; }; From 0c41b876472606e856f557a68765a4a87dfb7167 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 01:26:01 +0300 Subject: [PATCH 089/108] Fixed insignificant error in Poco, shown by TSan [#CLICKHOUSE-2]. --- .../Foundation/src/DirectoryWatcher.cpp | 69 ++++++++++--------- .../include/Poco/Net/HTTPServerConnection.h | 6 +- .../include/Poco/Net/TCPServerDispatcher.h | 26 +++---- 3 files changed, 53 insertions(+), 48 deletions(-) diff --git a/contrib/libpoco/Foundation/src/DirectoryWatcher.cpp b/contrib/libpoco/Foundation/src/DirectoryWatcher.cpp index 52af9d55145..f67a6d15e9b 100644 --- a/contrib/libpoco/Foundation/src/DirectoryWatcher.cpp +++ b/contrib/libpoco/Foundation/src/DirectoryWatcher.cpp @@ -45,6 +45,7 @@ #endif #include #include +#include namespace Poco { @@ -66,7 +67,7 @@ public: { return _owner; } - + virtual void run() = 0; virtual void stop() = 0; virtual bool supportsMoveEvents() const = 0; @@ -78,21 +79,21 @@ protected: size(0) { } - + ItemInfo(const ItemInfo& other): path(other.path), size(other.size), lastModified(other.lastModified) { } - + explicit ItemInfo(const File& f): path(f.path()), size(f.isFile() ? f.getSize() : 0), lastModified(f.getLastModified()) { } - + std::string path; File::FileSize size; Timestamp lastModified; @@ -109,7 +110,7 @@ protected: ++it; } } - + void compare(ItemInfoMap& oldEntries, ItemInfoMap& newEntries) { for (ItemInfoMap::iterator itn = newEntries.begin(); itn != newEntries.end(); ++itn) @@ -150,7 +151,7 @@ private: DirectoryWatcherStrategy(); DirectoryWatcherStrategy(const DirectoryWatcherStrategy&); DirectoryWatcherStrategy& operator = (const DirectoryWatcherStrategy&); - + DirectoryWatcher& _owner; }; @@ -168,21 +169,21 @@ public: if (!_hStopped) throw SystemException("cannot create event"); } - + ~WindowsDirectoryWatcherStrategy() { CloseHandle(_hStopped); } - + void run() { ItemInfoMap entries; scan(entries); - + DWORD filter = FILE_NOTIFY_CHANGE_FILE_NAME | FILE_NOTIFY_CHANGE_DIR_NAME; if (owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED) filter |= FILE_NOTIFY_CHANGE_SIZE | FILE_NOTIFY_CHANGE_LAST_WRITE; - + std::string path(owner().directory().path()); #if defined(POCO_WIN32_UTF8) std::wstring upath; @@ -204,7 +205,7 @@ public: } return; } - + bool stopped = false; while (!stopped) { @@ -237,21 +238,21 @@ public: catch (Poco::Exception& exc) { owner().scanError(&owner(), exc); - } + } } FindCloseChangeNotification(hChange); } - + void stop() { SetEvent(_hStopped); } - + bool supportsMoveEvents() const { return false; } - + private: HANDLE _hStopped; }; @@ -271,12 +272,12 @@ public: _fd = inotify_init(); if (_fd == -1) throw Poco::IOException("cannot initialize inotify", errno); } - + ~LinuxDirectoryWatcherStrategy() { close(_fd); } - + void run() { int mask = 0; @@ -302,7 +303,7 @@ public: owner().scanError(&owner(), exc); } } - + Poco::Buffer buffer(4096); while (!_stopped) { @@ -323,16 +324,16 @@ public: while (n > 0) { struct inotify_event* pEvent = reinterpret_cast(buffer.begin() + i); - + if (pEvent->len > 0) - { + { if (!owner().eventsSuspended()) { Poco::Path p(owner().directory().path()); p.makeDirectory(); p.setFileName(pEvent->name); Poco::File f(p.toString()); - + if ((pEvent->mask & IN_CREATE) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_ADDED)) { DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_ADDED); @@ -360,7 +361,7 @@ public: } } } - + i += sizeof(inotify_event) + pEvent->len; n -= sizeof(inotify_event) + pEvent->len; } @@ -368,12 +369,12 @@ public: } } } - + void stop() { _stopped = true; } - + bool supportsMoveEvents() const { return true; @@ -381,7 +382,7 @@ public: private: int _fd; - bool _stopped; + std::atomic _stopped; }; @@ -464,7 +465,7 @@ public: private: int _queueFD; int _dirFD; - bool _stopped; + std::atomic _stopped; }; @@ -478,11 +479,11 @@ public: DirectoryWatcherStrategy(owner) { } - + ~PollingDirectoryWatcherStrategy() { } - + void run() { ItemInfoMap entries; @@ -502,7 +503,7 @@ public: } } } - + void stop() { _stopped.set(); @@ -529,7 +530,7 @@ DirectoryWatcher::DirectoryWatcher(const std::string& path, int eventMask, int s init(); } - + DirectoryWatcher::DirectoryWatcher(const Poco::File& directory, int eventMask, int scanInterval): _directory(directory), _eventMask(eventMask), @@ -552,11 +553,11 @@ DirectoryWatcher::~DirectoryWatcher() } } - + void DirectoryWatcher::suspendEvents() { poco_assert (_eventsSuspended > 0); - + _eventsSuspended--; } @@ -571,7 +572,7 @@ void DirectoryWatcher::init() { if (!_directory.exists()) throw Poco::FileNotFoundException(_directory.path()); - + if (!_directory.isDirectory()) throw Poco::InvalidArgumentException("not a directory", _directory.path()); @@ -587,7 +588,7 @@ void DirectoryWatcher::init() _thread.start(*this); } - + void DirectoryWatcher::run() { _pStrategy->run(); diff --git a/contrib/libpoco/Net/include/Poco/Net/HTTPServerConnection.h b/contrib/libpoco/Net/include/Poco/Net/HTTPServerConnection.h index 5076332b9ea..6cc5b51744a 100644 --- a/contrib/libpoco/Net/include/Poco/Net/HTTPServerConnection.h +++ b/contrib/libpoco/Net/include/Poco/Net/HTTPServerConnection.h @@ -27,6 +27,8 @@ #include "Poco/Net/HTTPServerParams.h" #include "Poco/Mutex.h" +#include + namespace Poco { namespace Net { @@ -45,7 +47,7 @@ public: virtual ~HTTPServerConnection(); /// Destroys the HTTPServerConnection. - + void run(); /// Handles all HTTP requests coming in. @@ -56,7 +58,7 @@ protected: private: HTTPServerParams::Ptr _pParams; HTTPRequestHandlerFactory::Ptr _pFactory; - bool _stopped; + std::atomic _stopped; Poco::FastMutex _mutex; }; diff --git a/contrib/libpoco/Net/include/Poco/Net/TCPServerDispatcher.h b/contrib/libpoco/Net/include/Poco/Net/TCPServerDispatcher.h index a485617f319..7a23eba2090 100644 --- a/contrib/libpoco/Net/include/Poco/Net/TCPServerDispatcher.h +++ b/contrib/libpoco/Net/include/Poco/Net/TCPServerDispatcher.h @@ -29,6 +29,8 @@ #include "Poco/ThreadPool.h" #include "Poco/Mutex.h" +#include + namespace Poco { namespace Net { @@ -52,35 +54,35 @@ public: void release(); /// Decrements the object's reference count /// and deletes the object if the count - /// reaches zero. + /// reaches zero. void run(); /// Runs the dispatcher. - + void enqueue(const StreamSocket& socket); /// Queues the given socket connection. void stop(); /// Stops the dispatcher. - + int currentThreads() const; /// Returns the number of currently used threads. int maxThreads() const; /// Returns the maximum number of threads available. - + int totalConnections() const; /// Returns the total number of handled connections. - + int currentConnections() const; - /// Returns the number of currently handled connections. + /// Returns the number of currently handled connections. int maxConcurrentConnections() const; - /// Returns the maximum number of concurrently handled connections. - + /// Returns the maximum number of concurrently handled connections. + int queuedConnections() const; - /// Returns the number of queued connections. - + /// Returns the number of queued connections. + int refusedConnections() const; /// Returns the number of refused connections. @@ -93,7 +95,7 @@ protected: void beginConnection(); /// Updates the performance counters. - + void endConnection(); /// Updates the performance counters. @@ -109,7 +111,7 @@ private: int _currentConnections; int _maxConcurrentConnections; int _refusedConnections; - bool _stopped; + std::atomic _stopped; Poco::NotificationQueue _queue; TCPServerConnectionFactory::Ptr _pConnectionFactory; Poco::ThreadPool& _threadPool; From fe717a4d6e169030cd7d43d6c5cdd87a73777d0d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 01:28:38 +0300 Subject: [PATCH 090/108] Fixed not an issue, shown by TSan; minor modifications [#CLICKHOUSE-2]. --- dbms/src/Interpreters/Quota.cpp | 45 ++++++++++++++++++++++----------- dbms/src/Interpreters/Quota.h | 39 ++++++++++++++++++++-------- 2 files changed, 58 insertions(+), 26 deletions(-) diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp index ff459e88ece..4a3d40aa7fa 100644 --- a/dbms/src/Interpreters/Quota.cpp +++ b/dbms/src/Interpreters/Quota.cpp @@ -37,10 +37,11 @@ template void QuotaValues::initFromConfig(const String & config_elem, Po template void QuotaValues>::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config); -void QuotaForInterval::initFromConfig(const String & config_elem, time_t duration_, time_t offset_, Poco::Util::AbstractConfiguration & config) +void QuotaForInterval::initFromConfig(const String & config_elem, time_t duration_, bool randomize_, time_t offset_, Poco::Util::AbstractConfiguration & config) { - rounded_time = 0; + rounded_time.store(0, std::memory_order_relaxed); duration = duration_; + randomize = randomize_; offset = offset_; max.initFromConfig(config_elem, config); } @@ -61,8 +62,10 @@ String QuotaForInterval::toString() const { std::stringstream res; + auto loaded_rounded_time = rounded_time.load(std::memory_order_relaxed); + res << std::fixed << std::setprecision(3) - << "Interval: " << LocalDateTime(rounded_time) << " - " << LocalDateTime(rounded_time + duration) << ".\n" + << "Interval: " << LocalDateTime(loaded_rounded_time) << " - " << LocalDateTime(loaded_rounded_time + duration) << ".\n" << "Queries: " << used.queries << ".\n" << "Errors: " << used.errors << ".\n" << "Result rows: " << used.result_rows << ".\n" @@ -107,10 +110,22 @@ void QuotaForInterval::checkAndAddExecutionTime(time_t current_time, const Strin void QuotaForInterval::updateTime(time_t current_time) { - if (current_time >= rounded_time + static_cast(duration)) + /** If current time is greater than end of interval, + * then clear accumulated quota values and switch to next interval [rounded_time, rounded_time + duration). + */ + + auto loaded_rounded_time = rounded_time.load(std::memory_order_acquire); + while (true) { - rounded_time = (current_time - offset) / duration * duration + offset; - used.clear(); + if (current_time < loaded_rounded_time + static_cast(duration)) + break; + + time_t new_rounded_time = (current_time - offset) / duration * duration + offset; + if (rounded_time.compare_exchange_strong(loaded_rounded_time, new_rounded_time)) + { + used.clear(); + break; + } } } @@ -136,7 +151,7 @@ void QuotaForInterval::check( message << " has been exceeded. " << resource_name << ": " << used_amount << ", max: " << max_amount << ". " - << "Interval will end at " << LocalDateTime(rounded_time + duration) << ". " + << "Interval will end at " << LocalDateTime(rounded_time.load(std::memory_order_relaxed) + duration) << ". " << "Name of quota template: '" << quota_name << "'."; throw Exception(message.str(), ErrorCodes::QUOTA_EXPIRED); @@ -158,14 +173,14 @@ void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::A time_t duration = config.getInt(interval_config_elem + ".duration", 0); time_t offset = 0; - if (!duration) /// Skip quaotas with zero duration + if (!duration) /// Skip quotas with zero duration continue; bool randomize = config.getBool(interval_config_elem + ".randomize", false); if (randomize) offset = std::uniform_int_distribution(0, duration - 1)(rng); - cont[duration].initFromConfig(interval_config_elem, duration, offset, config); + cont[duration].initFromConfig(interval_config_elem, duration, randomize, offset, config); } } @@ -182,7 +197,7 @@ void QuotaForIntervals::setMax(const QuotaForIntervals & quota) for (auto & x : quota.cont) { if (!cont.count(x.first)) - cont[x.first] = x.second; + cont.emplace(x.first, x.second); else cont[x.first].max = x.second.max; } @@ -254,7 +269,7 @@ void Quota::loadFromConfig(const String & config_elem, const String & name_, Poc QuotaForIntervals new_max(name, {}); new_max.initFromConfig(config_elem, config, rng); - if (!(new_max == max)) + if (!new_max.hasEqualConfiguration(max)) { max = new_max; for (auto & quota : quota_for_keys) @@ -310,9 +325,9 @@ void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config) for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it) { - if (!cont[*it]) - cont[*it] = std::make_unique(); - cont[*it]->loadFromConfig("quotas." + *it, *it, config, rng); + if (!cont.count(*it)) + cont.try_emplace(*it); + cont[*it].loadFromConfig("quotas." + *it, *it, config, rng); } } @@ -322,7 +337,7 @@ QuotaForIntervalsPtr Quotas::get(const String & name, const String & quota_key, if (cont.end() == it) throw Exception("Unknown quota " + name, ErrorCodes::UNKNOWN_QUOTA); - return it->second->get(quota_key, user_name, ip); + return it->second.get(quota_key, user_name, ip); } } diff --git a/dbms/src/Interpreters/Quota.h b/dbms/src/Interpreters/Quota.h index 0421ce0df6d..3331af1cfa2 100644 --- a/dbms/src/Interpreters/Quota.h +++ b/dbms/src/Interpreters/Quota.h @@ -99,16 +99,17 @@ inline auto QuotaValues>::tuple() const /// Time, rounded down to start of interval; limits for that interval and accumulated values. struct QuotaForInterval { - time_t rounded_time = 0; + std::atomic rounded_time {0}; size_t duration = 0; + bool randomize = false; time_t offset = 0; /// Offset of interval for randomization (to avoid DoS if intervals for many users end at one time). QuotaValues max; QuotaValues> used; - QuotaForInterval() {} + QuotaForInterval() = default; QuotaForInterval(time_t duration_) : duration(duration_) {} - void initFromConfig(const String & config_elem, time_t duration_, time_t offset_, Poco::Util::AbstractConfiguration & config); + void initFromConfig(const String & config_elem, time_t duration_, bool randomize_, time_t offset_, Poco::Util::AbstractConfiguration & config); /// Increase current value. void addQuery() noexcept; @@ -125,14 +126,30 @@ struct QuotaForInterval /// Get a text, describing what quota is exceeded. String toString() const; + /// Only compare configuration, not accumulated (used) values or random offsets. bool operator== (const QuotaForInterval & rhs) const { - return - rounded_time == rhs.rounded_time && - duration == rhs.duration && - max == rhs.max && - used == rhs.used; + return randomize == rhs.randomize + && duration == rhs.duration + && max == rhs.max; } + + QuotaForInterval & operator= (const QuotaForInterval & rhs) + { + rounded_time.store(rhs.rounded_time.load(std::memory_order_relaxed)); + duration = rhs.duration; + randomize = rhs.randomize; + offset = rhs.offset; + max = rhs.max; + used = rhs.used; + return *this; + } + + QuotaForInterval(const QuotaForInterval & rhs) + { + *this = rhs; + } + private: /// Reset counters of used resources, if interval for quota is expired. void updateTime(time_t current_time); @@ -192,7 +209,7 @@ public: /// Get text, describing what part of quota has been exceeded. String toString() const; - bool operator== (const QuotaForIntervals & rhs) const + bool hasEqualConfiguration(const QuotaForIntervals & rhs) const { return cont == rhs.cont && quota_name == rhs.quota_name; } @@ -233,13 +250,13 @@ class Quotas { private: /// Name of quota -> quota. - using Container = std::unordered_map>; + using Container = std::unordered_map; Container cont; public: void loadFromConfig(Poco::Util::AbstractConfiguration & config); QuotaForIntervalsPtr get(const String & name, const String & quota_key, - const String & user_name, const Poco::Net::IPAddress & ip); + const String & user_name, const Poco::Net::IPAddress & ip); }; } From 93f1e274fb29b6a7cb20676252085a19d8d5a62d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 01:43:11 +0300 Subject: [PATCH 091/108] Removed unused code from pcg-random, that generates compiler warning by default [#CLICKHOUSE-2]. --- contrib/libpcg-random/include/pcg_extras.hpp | 21 -------------------- 1 file changed, 21 deletions(-) diff --git a/contrib/libpcg-random/include/pcg_extras.hpp b/contrib/libpcg-random/include/pcg_extras.hpp index 9bbe8edfe9a..929c756b151 100644 --- a/contrib/libpcg-random/include/pcg_extras.hpp +++ b/contrib/libpcg-random/include/pcg_extras.hpp @@ -582,27 +582,6 @@ public: } }; -/* - * Sometimes you might want a distinct seed based on when the program - * was compiled. That way, a particular instance of the program will - * behave the same way, but when recompiled it'll produce a different - * value. - */ - -template -struct static_arbitrary_seed { -private: - static constexpr IntType fnv(IntType hash, const char* pos) { - return *pos == '\0' - ? hash - : fnv((hash * IntType(16777619U)) ^ *pos, (pos+1)); - } - -public: - static constexpr IntType value = fnv(IntType(2166136261U ^ sizeof(IntType)), - __DATE__ __TIME__ __FILE__); -}; - // Sometimes, when debugging or testing, it's handy to be able print the name // of a (in human-readable form). This code allows the idiom: // From a6d5e9c7e8c352f2561813ef2669c816ac8f8d72 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 02:04:22 +0300 Subject: [PATCH 092/108] Fixed not an issue, shown by TSan [#CLICKHOUSE-2]. --- dbms/src/Storages/Distributed/DirectoryMonitor.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.h b/dbms/src/Storages/Distributed/DirectoryMonitor.h index cb7ff07a5ef..f1b6546c195 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.h +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.h @@ -2,6 +2,7 @@ #include +#include #include #include #include @@ -45,7 +46,7 @@ private: std::chrono::milliseconds default_sleep_time; std::chrono::milliseconds sleep_time; std::chrono::time_point last_decrease_time {std::chrono::system_clock::now()}; - bool quit {false}; + std::atomic quit {false}; std::mutex mutex; std::condition_variable cond; Logger * log; From 5a8f47a640ca38ddee81eacf1626dcc0d24a9896 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 02:07:32 +0300 Subject: [PATCH 093/108] Fixed not an issue, shown by TSan [#CLICKHOUSE-2]. --- dbms/src/Storages/Distributed/DirectoryMonitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 03831ba8428..668500853e9 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -115,7 +115,7 @@ void StorageDistributedDirectoryMonitor::run() std::unique_lock lock{mutex}; - const auto quit_requested = [this] { return quit; }; + const auto quit_requested = [this] { return quit.load(std::memory_order_relaxed); }; while (!quit_requested()) { From cd540c26747e9b71543a364eff7c1cf7c00a9ff8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 02:17:38 +0300 Subject: [PATCH 094/108] Replaced Mersenne Twister to PCG-Random [#CLICKHOUSE-2]. --- dbms/CMakeLists.txt | 1 + dbms/src/AggregateFunctions/CMakeLists.txt | 1 - dbms/src/Common/ArrayCache.h | 3 ++- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 7 ++++--- dbms/src/Common/tests/array_cache.cpp | 3 ++- .../Common/tests/gtest_rw_lock_fifo.cpp.cpp | 7 ++++--- .../DataStreams/narrowBlockInputStreams.cpp | 3 ++- dbms/src/Dictionaries/CacheDictionary.cpp | 2 +- dbms/src/Dictionaries/CacheDictionary.h | 4 ++-- .../ComplexKeyCacheDictionary.cpp | 2 +- .../Dictionaries/ComplexKeyCacheDictionary.h | 4 ++-- dbms/src/Interpreters/Context.cpp | 4 ++-- dbms/src/Interpreters/DDLWorker.cpp | 6 +++++- dbms/src/Interpreters/ExternalDictionaries.h | 5 +++-- dbms/src/Interpreters/Quota.cpp | 7 ++++--- dbms/src/Interpreters/Quota.h | 6 +++--- dbms/src/Server/Benchmark.cpp | 3 ++- .../MergeTree/BackgroundProcessingPool.cpp | 4 +++- .../Storages/MergeTree/ReshardingWorker.cpp | 20 ++++++------------- .../src/Storages/StorageReplicatedMergeTree.h | 4 ++-- utils/iotest/iotest.cpp | 8 +++----- utils/iotest/iotest_nonblock.cpp | 8 +++----- utils/test-data-generator/main.cpp | 5 +++-- utils/test-data-generator/markov-model.cpp | 3 ++- 24 files changed, 62 insertions(+), 58 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 0feacf7f849..dbf23edc479 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -206,6 +206,7 @@ target_link_libraries (dbms target_include_directories (dbms BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR}) target_include_directories (dbms BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) target_include_directories (dbms PUBLIC ${DBMS_INCLUDE_DIR}) +target_include_directories (dbms PUBLIC ${PCG_RANDOM_INCLUDE_DIR}) # only for copy_headers.sh: target_include_directories (dbms PRIVATE ${COMMON_INCLUDE_DIR}) diff --git a/dbms/src/AggregateFunctions/CMakeLists.txt b/dbms/src/AggregateFunctions/CMakeLists.txt index 4414d89f4a0..f3fb20b6101 100644 --- a/dbms/src/AggregateFunctions/CMakeLists.txt +++ b/dbms/src/AggregateFunctions/CMakeLists.txt @@ -29,4 +29,3 @@ list(REMOVE_ITEM clickhouse_aggregate_functions_headers add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources}) target_link_libraries(clickhouse_aggregate_functions dbms) target_include_directories (clickhouse_aggregate_functions PRIVATE ${COMMON_INCLUDE_DIR}) -target_include_directories (clickhouse_aggregate_functions PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) diff --git a/dbms/src/Common/ArrayCache.h b/dbms/src/Common/ArrayCache.h index 38ec9f88bd7..eda20ca6bf2 100644 --- a/dbms/src/Common/ArrayCache.h +++ b/dbms/src/Common/ArrayCache.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -160,7 +161,7 @@ private: mutable std::mutex mutex; - std::mt19937_64 rng {static_cast(randomSeed())}; + pcg64 rng{randomSeed()}; struct Chunk : private boost::noncopyable { diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 05c5c16d3d2..765bf494580 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1,9 +1,11 @@ #include +#include #include #include #include #include #include +#include namespace ProfileEvents @@ -137,9 +139,8 @@ struct ZooKeeperArgs } /// Shuffle the hosts to distribute the load among ZooKeeper nodes. - std::random_device rd; - std::mt19937 g(rd()); - std::shuffle(hosts_strings.begin(), hosts_strings.end(), g); + pcg64 rng(randomSeed()); + std::shuffle(hosts_strings.begin(), hosts_strings.end(), rng); for (auto & host : hosts_strings) { diff --git a/dbms/src/Common/tests/array_cache.cpp b/dbms/src/Common/tests/array_cache.cpp index 8dec433e3d6..b962e86c53c 100644 --- a/dbms/src/Common/tests/array_cache.cpp +++ b/dbms/src/Common/tests/array_cache.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -51,7 +52,7 @@ int main(int argc, char ** argv) { threads.emplace_back([&] { - std::mt19937 generator(randomSeed()); + pcg64 generator(randomSeed()); for (size_t i = 0; i < num_iterations; ++i) { diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp index 8dee71fb379..48f04e9fc01 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -23,7 +24,7 @@ TEST(Common, RWLockFIFO_1) static auto fifo_lock = RWLockFIFO::create(); static thread_local std::random_device rd; - static thread_local std::mt19937 gen(rd()); + static thread_local pcg64 gen(rd()); auto func = [&] (size_t threads, int round) { for (int i = 0; i < cycles; ++i) @@ -85,7 +86,7 @@ TEST(Common, RWLockFIFO_Recursive) static auto fifo_lock = RWLockFIFO::create(); static thread_local std::random_device rd; - static thread_local std::mt19937 gen(rd()); + static thread_local pcg64 gen(rd()); std::thread t1([&] () { for (int i = 0; i < 2 * cycles; ++i) @@ -106,7 +107,7 @@ TEST(Common, RWLockFIFO_Recursive) std::this_thread::sleep_for(sleep_for); auto lock2 = fifo_lock->getLock(RWLockFIFO::Read); - + EXPECT_ANY_THROW({fifo_lock->getLock(RWLockFIFO::Write);}); } diff --git a/dbms/src/DataStreams/narrowBlockInputStreams.cpp b/dbms/src/DataStreams/narrowBlockInputStreams.cpp index 25927e451e4..07c26bc9b67 100644 --- a/dbms/src/DataStreams/narrowBlockInputStreams.cpp +++ b/dbms/src/DataStreams/narrowBlockInputStreams.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -20,7 +21,7 @@ BlockInputStreams narrowBlockInputStreams(BlockInputStreams & inputs, size_t wid for (size_t i = 0; i < size; ++i) distribution[i] = i % width; - std::mt19937 generator(randomSeed()); + pcg64 generator(randomSeed()); std::shuffle(distribution.begin(), distribution.end(), generator); for (size_t i = 0; i < size; ++i) diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index f69fbd607f4..b9d33f5ac16 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -68,7 +68,7 @@ CacheDictionary::CacheDictionary(const std::string & name, const DictionaryStruc size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))}, size_overlap_mask{this->size - 1}, cells{this->size}, - rnd_engine{randomSeed()} + rnd_engine(randomSeed()) { if (!this->source_ptr->supportsSelectiveLoad()) throw Exception{ diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index 45bfceb16ad..ac0854fc84e 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include @@ -260,7 +260,7 @@ private: Attribute * hierarchical_attribute = nullptr; std::unique_ptr string_arena; - mutable std::mt19937_64 rnd_engine; + mutable pcg64 rnd_engine; mutable size_t bytes_allocated = 0; mutable std::atomic element_count{0}; diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp index b8f5c8c1dfd..37cfc759e6c 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -57,7 +57,7 @@ ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const std::string & name, c : name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))}, size_overlap_mask{this->size - 1}, - rnd_engine{randomSeed()} + rnd_engine(randomSeed()) { if (!this->source_ptr->supportsSelectiveLoad()) throw Exception{ diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index 2b806b51316..09e6330efdd 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -19,6 +18,7 @@ #include #include #include +#include namespace ProfileEvents @@ -721,7 +721,7 @@ private: std::unique_ptr fixed_size_keys_pool = key_size_is_fixed ? std::make_unique(key_size) : nullptr; std::unique_ptr string_arena; - mutable std::mt19937_64 rnd_engine; + mutable pcg64 rnd_engine; mutable size_t bytes_allocated = 0; mutable std::atomic element_count{0}; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index a2812495c69..3663597f334 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -9,6 +8,7 @@ #include #include +#include #include #include @@ -175,7 +175,7 @@ struct ContextShared Context::ApplicationType application_type = Context::ApplicationType::SERVER; - std::mt19937_64 rng{randomSeed()}; + pcg64 rng{randomSeed()}; ContextShared() { diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index eda33e983ee..c2afea4a79f 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -34,6 +35,9 @@ #include #include +#include +#include + namespace DB { @@ -655,7 +659,7 @@ void DDLWorker::processTaskAlter( bool alter_executed_by_any_replica = false; { auto lock = createSimpleZooKeeperLock(zookeeper, shard_path, "lock", task.host_id_str); - std::mt19937 rng(StringRefHash{}(task.host_id_str) + reinterpret_cast(&rng)); + pcg64 rng(randomSeed()); for (int num_tries = 0; num_tries < 10; ++num_tries) { diff --git a/dbms/src/Interpreters/ExternalDictionaries.h b/dbms/src/Interpreters/ExternalDictionaries.h index 7419f44a627..84488118e19 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.h +++ b/dbms/src/Interpreters/ExternalDictionaries.h @@ -13,7 +13,8 @@ #include #include #include -#include +#include + namespace DB { @@ -74,7 +75,7 @@ private: */ std::unordered_map update_times; - std::mt19937_64 rnd_engine{randomSeed()}; + pcg64 rnd_engine{randomSeed()}; Context & context; diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp index 4a3d40aa7fa..65fdda86fa5 100644 --- a/dbms/src/Interpreters/Quota.cpp +++ b/dbms/src/Interpreters/Quota.cpp @@ -8,6 +8,7 @@ #include #include +#include namespace DB @@ -159,7 +160,7 @@ void QuotaForInterval::check( } -void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng) +void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, pcg64 & rng) { Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(config_elem, config_keys); @@ -250,7 +251,7 @@ String QuotaForIntervals::toString() const } -void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng) +void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, pcg64 & rng) { name = name_; @@ -308,7 +309,7 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config) { - std::mt19937 rng; + pcg64 rng; Poco::Util::AbstractConfiguration::Keys config_keys; config.keys("quotas", config_keys); diff --git a/dbms/src/Interpreters/Quota.h b/dbms/src/Interpreters/Quota.h index 3331af1cfa2..8df01fe9593 100644 --- a/dbms/src/Interpreters/Quota.h +++ b/dbms/src/Interpreters/Quota.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include @@ -191,7 +191,7 @@ public: return cont.empty(); } - void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng); + void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, pcg64 & rng); /// Set maximum values (limits) from passed argument. /// Remove intervals that does not exist in argument. Add intervals from argument, that we don't have. @@ -241,7 +241,7 @@ struct Quota bool keyed_by_ip = false; - void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng); + void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, pcg64 & rng); QuotaForIntervalsPtr get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip); }; diff --git a/dbms/src/Server/Benchmark.cpp b/dbms/src/Server/Benchmark.cpp index f3bd262ad6f..8d7f1057a2f 100644 --- a/dbms/src/Server/Benchmark.cpp +++ b/dbms/src/Server/Benchmark.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -230,7 +231,7 @@ private: void run() { - std::mt19937 generator(randomSeed()); + pcg64 generator(randomSeed()); std::uniform_int_distribution distribution(0, queries.size() - 1); for (size_t i = 0; i < concurrency; ++i) diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index c0a7c1c0214..12b7edf32dc 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -2,10 +2,12 @@ #include #include #include +#include #include #include #include +#include #include @@ -115,7 +117,7 @@ void BackgroundProcessingPool::threadFunction() memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); current_memory_tracker = &memory_tracker; - std::mt19937 rng(reinterpret_cast(&rng)); + pcg64 rng(randomSeed()); std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, sleep_seconds_random_part)(rng))); while (!shutdown) diff --git a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp index e0d0086c3fe..08b5ed51504 100644 --- a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp +++ b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp @@ -31,11 +31,14 @@ #include #include +#include + #include #include #include #include -#include +#include + namespace DB { @@ -1391,19 +1394,8 @@ void ReshardingWorker::executeAttach(LogRecord & log_record) /// Description of tasks for each replica of a shard. /// For fault tolerance purposes, some fields are provided /// to perform attempts on more than one replica if needed. - struct ShardTaskInfo + struct ShardTaskInfo : private boost::noncopyable { - ShardTaskInfo() - { - rng = std::mt19937(randomSeed()); - } - - ShardTaskInfo(const ShardTaskInfo &) = delete; - ShardTaskInfo & operator=(const ShardTaskInfo &) = delete; - - ShardTaskInfo(ShardTaskInfo &&) = default; - ShardTaskInfo & operator=(ShardTaskInfo &&) = default; - /// one task for each replica std::vector shard_tasks; /// index to the replica to be used @@ -1411,7 +1403,7 @@ void ReshardingWorker::executeAttach(LogRecord & log_record) /// result of the operation on the current replica bool is_success = false; /// For pseudo-random number generation. - std::mt19937 rng; + pcg64 rng{randomSeed()}; }; const WeightedZooKeeperPath & weighted_path = current_job.paths[log_record.shard_no]; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 0793231b16e..25562c778f1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -321,7 +321,7 @@ private: Logger * log; - std::mt19937 rng{randomSeed()}; + pcg64 rng{randomSeed()}; StorageReplicatedMergeTree( const String & zookeeper_path_, diff --git a/utils/iotest/iotest.cpp b/utils/iotest/iotest.cpp index 2c91e11cd41..439672fc25c 100644 --- a/utils/iotest/iotest.cpp +++ b/utils/iotest/iotest.cpp @@ -7,12 +7,14 @@ #include #include #include +#include #include #include #include #include +#include #include #include @@ -50,11 +52,7 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block else buf = &simple_buf[0]; - std::mt19937 rng; - - timespec times; - clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×); - rng.seed(times.tv_nsec); + pcg64 rng(randomSeed()); for (size_t i = 0; i < count; ++i) { diff --git a/utils/iotest/iotest_nonblock.cpp b/utils/iotest/iotest_nonblock.cpp index 5482e0ddaa7..fdd25860825 100644 --- a/utils/iotest/iotest_nonblock.cpp +++ b/utils/iotest/iotest_nonblock.cpp @@ -13,12 +13,14 @@ #include #include +#include #include #include #include #include +#include #include #include @@ -77,11 +79,7 @@ int mainImpl(int argc, char ** argv) std::vector buf(block_size); - std::mt19937 rng; - - timespec times; - clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×); - rng.seed(times.tv_nsec); + pcg64 rng(randomSeed()); Stopwatch watch; diff --git a/utils/test-data-generator/main.cpp b/utils/test-data-generator/main.cpp index 4d16195cb72..2a0db1e4b3d 100644 --- a/utils/test-data-generator/main.cpp +++ b/utils/test-data-generator/main.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -42,8 +43,8 @@ struct Models struct Generator { WriteBufferFromFileDescriptor out; - std::mt19937_64 random; - std::mt19937_64 random_with_seed; + pcg64 random; + pcg64 random_with_seed; Models models; // UInt64 WatchID = random(); diff --git a/utils/test-data-generator/markov-model.cpp b/utils/test-data-generator/markov-model.cpp index 08f7a3f72a9..87849f8f398 100644 --- a/utils/test-data-generator/markov-model.cpp +++ b/utils/test-data-generator/markov-model.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -47,7 +48,7 @@ try ReadBufferFromFileDescriptor in(STDIN_FILENO); WriteBufferFromFileDescriptor out(STDOUT_FILENO); - std::mt19937 random; + pcg64 random; if (options.count("seed")) random.seed(options["seed"].as()); From cf5fd27b9484ca301b38dd3818e818845f62c011 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 04:39:40 +0300 Subject: [PATCH 095/108] Clarifications [#CLICKHOUSE-2]. --- dbms/src/DataStreams/AsynchronousBlockInputStream.h | 2 +- dbms/src/DataStreams/IBlockInputStream.h | 2 ++ dbms/src/DataStreams/IProfilingBlockInputStream.h | 8 +++++--- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index 4ee17c2d30f..e72b2f6489f 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -115,7 +115,7 @@ protected: return res; /// Start the next block calculation - block = Block(); + block.clear(); next(); return res; diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index fccccbd67e6..8e4b6d4abc0 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -49,6 +49,8 @@ public: /** Read next block. * If there are no more blocks, return an empty block (for which operator `bool` returns false). + * NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously. + * This also applies for readPrefix, readSuffix. */ virtual Block read() = 0; diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h index 8fa9eb6731b..166560be154 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.h @@ -174,8 +174,6 @@ protected: ProgressCallback progress_callback; ProcessListElement * process_list_elem = nullptr; - bool enabled_extremes = false; - /// Additional information that can be generated during the work process. /// Total values during aggregation. @@ -184,6 +182,10 @@ protected: Block extremes; /// The approximate total number of rows to read. For progress bar. size_t total_rows_approx = 0; + +private: + bool enabled_extremes = false; + /// Information about the approximate total number of rows is collected in the parent source. bool collected_total_rows_approx = false; @@ -218,7 +220,7 @@ protected: void collectTotalRowsApprox(); /** Send information about the approximate total number of rows to the progress bar. - * It is done so that sending occurs only in the upper source. + * It is done so that sending occurs only in the upper stream. */ void collectAndSendTotalRowsApprox(); }; From dc5eca0bebf41bf91b88ec3bcc24e664e5d22f3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 05:06:16 +0300 Subject: [PATCH 096/108] Fixed awful translation [#CLICKHOUSE-2]. --- dbms/src/Client/MultiplexedConnections.h | 2 +- dbms/src/DataStreams/RemoteBlockInputStream.cpp | 2 +- dbms/src/Server/TCPHandler.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Client/MultiplexedConnections.h b/dbms/src/Client/MultiplexedConnections.h index 1765741c649..817a37b1d93 100644 --- a/dbms/src/Client/MultiplexedConnections.h +++ b/dbms/src/Client/MultiplexedConnections.h @@ -43,7 +43,7 @@ public: /// Get packet from any replica. Connection::Packet receivePacket(); - /// Get information about the last received package. + /// Get information about the last received packet. BlockExtraInfo getBlockExtraInfo() const; /// Break all active connections. diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index c20b0e7a60a..4d60d9ae5fd 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -236,7 +236,7 @@ void RemoteBlockInputStream::readSuffixImpl() /// Send the request to abort the execution of the request, if not already sent. tryCancel("Cancelling query because enough data has been read"); - /// Get the remaining packages so that there is no out of sync in the connections to the replicas. + /// Get the remaining packets so that there is no out of sync in the connections to the replicas. Connection::Packet packet = multiplexed_connections->drain(); switch (packet.type) { diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 011454a5159..7ca2a1cbf8b 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -315,7 +315,7 @@ void TCPHandler::processOrdinaryQuery() { if (isQueryCancelled()) { - /// A package was received requesting to stop execution of the request. + /// A packet was received requesting to stop execution of the request. async_in.cancel(); break; } From 98ad6a5db3239ac6567ee96e90973b935dcac39b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 05:30:45 +0300 Subject: [PATCH 097/108] Simplification [#CLICKHOUSE-2]. --- .../AsynchronousBlockInputStream.h | 42 ++++++++----------- 1 file changed, 17 insertions(+), 25 deletions(-) diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index e72b2f6489f..cd4116a8325 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -57,10 +57,7 @@ public: if (started) { pool.wait(); - if (exception) - std::rethrow_exception(exception); children.back()->readSuffix(); - started = false; } } @@ -82,18 +79,23 @@ public: ~AsynchronousBlockInputStream() override { - if (started) - pool.wait(); + try + { + pool.wait(); /// It's ok to call wait even if there is no active threads. + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } protected: - ThreadPool pool{1}; + ThreadPool pool{1}; /// Rethrows exceptions automatically on wait. Poco::Event ready; bool started = false; bool first = true; Block block; - std::exception_ptr exception; Block readImpl() override @@ -101,15 +103,12 @@ protected: /// If there were no calculations yet, calculate the first block synchronously if (!started) { - calculate(current_memory_tracker); started = true; + calculate(current_memory_tracker); } else /// If the calculations are already in progress - wait for the result pool.wait(); - if (exception) - std::rethrow_exception(exception); - Block res = block; if (!res) return res; @@ -134,22 +133,15 @@ protected: { CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - try + if (first) { - if (first) - { - first = false; - setThreadName("AsyncBlockInput"); - current_memory_tracker = memory_tracker; - children.back()->readPrefix(); - } + first = false; + setThreadName("AsyncBlockInput"); + current_memory_tracker = memory_tracker; + children.back()->readPrefix(); + } - block = children.back()->read(); - } - catch (...) - { - exception = std::current_exception(); - } + block = children.back()->read(); ready.set(); } From 2c424c7989dd9143ee460805ed949791b6a75d7e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 06:17:05 +0300 Subject: [PATCH 098/108] Added description of possible race condition [#CLICKHOUSE-2]. --- dbms/src/DataStreams/LazyBlockInputStream.h | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/dbms/src/DataStreams/LazyBlockInputStream.h b/dbms/src/DataStreams/LazyBlockInputStream.h index 8229bedcf65..8cf4f3a293d 100644 --- a/dbms/src/DataStreams/LazyBlockInputStream.h +++ b/dbms/src/DataStreams/LazyBlockInputStream.h @@ -60,6 +60,23 @@ protected: { std::lock_guard lock(cancel_mutex); + /** TODO Data race here. See IProfilingBlockInputStream::collectAndSendTotalRowsApprox. + Assume following pipeline: + + RemoteBlockInputStream + AsynchronousBlockInputStream + LazyBlockInputStream + + RemoteBlockInputStream calls AsynchronousBlockInputStream::readPrefix + and AsynchronousBlockInputStream spawns a thread and returns. + + The separate thread will call LazyBlockInputStream::read + LazyBlockInputStream::read will add more children to itself + + In the same moment, in main thread, RemoteBlockInputStream::read is called, + then IProfilingBlockInputStream::collectAndSendTotalRowsApprox is called + and iterates over set of children. + */ children.push_back(input); if (isCancelled() && p_input) From 02a0317db635c264d2df1228a4af39f963d34b5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 06:29:40 +0300 Subject: [PATCH 099/108] Updated instruction of installation of clang: added useful tool [#CLICKHOUSE-2]. --- utils/prepare-environment/install-clang.sh | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/utils/prepare-environment/install-clang.sh b/utils/prepare-environment/install-clang.sh index d1ca427ad4e..b58e0d8f20a 100755 --- a/utils/prepare-environment/install-clang.sh +++ b/utils/prepare-environment/install-clang.sh @@ -9,20 +9,29 @@ THREADS=$(grep -c ^processor /proc/cpuinfo) cd ~ sudo apt-get install -y subversion cmake3 + mkdir llvm cd llvm svn co "http://llvm.org/svn/llvm-project/llvm/${BRANCH}" llvm + cd llvm/tools svn co "http://llvm.org/svn/llvm-project/cfe/${BRANCH}" clang svn co "http://llvm.org/svn/llvm-project/lld/${BRANCH}" lld svn co "http://llvm.org/svn/llvm-project/polly/${BRANCH}" polly + cd clang/tools svn co "http://llvm.org/svn/llvm-project/clang-tools-extra/${BRANCH}" extra + +git clone https://github.com/include-what-you-use/include-what-you-use.git +echo 'add_subdirectory(include-what-you-use)' >> CMakeLists.txt +sudo apt-get install libncurses5-dev + cd ../../../.. cd llvm/projects/ svn co "http://llvm.org/svn/llvm-project/compiler-rt/${BRANCH}" compiler-rt svn co "http://llvm.org/svn/llvm-project/libcxx/${BRANCH}" libcxx svn co "http://llvm.org/svn/llvm-project/libcxxabi/${BRANCH}" libcxxabi + cd ../.. mkdir build cd build/ From 7405dd57423836580035be774b9578a79b1605e0 Mon Sep 17 00:00:00 2001 From: Bulat Gaifullin Date: Sun, 10 Sep 2017 00:26:02 +0300 Subject: [PATCH 100/108] Add ssl options for MySQL connection #1217 --- libs/libmysqlxx/include/mysqlxx/Connection.h | 28 +++++++++++- libs/libmysqlxx/include/mysqlxx/Pool.h | 3 ++ libs/libmysqlxx/src/Connection.cpp | 18 +++++++- libs/libmysqlxx/src/Pool.cpp | 46 ++++++++++++++------ 4 files changed, 78 insertions(+), 17 deletions(-) diff --git a/libs/libmysqlxx/include/mysqlxx/Connection.h b/libs/libmysqlxx/include/mysqlxx/Connection.h index c704faa3212..93efb7d7002 100644 --- a/libs/libmysqlxx/include/mysqlxx/Connection.h +++ b/libs/libmysqlxx/include/mysqlxx/Connection.h @@ -43,6 +43,12 @@ private: * Or using socket: * mysqlxx::Connection connection("Test", "localhost", "root", "qwerty", 0, "/path/to/socket/file.sock"); * + * Or using custom certificate authority file: + * mysqlxx::Connection connection("Test", "localhost", "root", "qwerty", 3306, "/path/to/ca/file.pem"); + * + * Or using custom certificate and key file: + * mysqlxx::Connection connection("Test", "localhost", "root", "qwerty", 3306, "", "/path/to/cert/file.pem", "/path/to/key/file.pem"); + * * Attention! It's strictly recommended to use connection in thread where it was created. * In order to use connection in other thread, you should call MySQL C API function mysql_thread_init() before and * mysql_thread_end() after working with it. @@ -62,6 +68,9 @@ public: const char * password = 0, unsigned port = 0, const char * socket = "", + const char * ssl_ca = "", + const char * ssl_cert = "", + const char * ssl_key = "", unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT); @@ -78,6 +87,9 @@ public: const char * password, unsigned port, const char * socket, + const char* ssl_ca, + const char* ssl_cert, + const char* ssl_key, unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT); @@ -91,6 +103,9 @@ public: std::string password = cfg.getString(config_name + ".password"); unsigned port = cfg.getInt(config_name + ".port", 0); std::string socket = cfg.getString(config_name + ".socket", ""); + std::string ssl_ca = cfg.getString(config_name + ".ssl_ca", ""); + std::string ssl_cert = cfg.getString(config_name + ".ssl_cert", ""); + std::string ssl_key = cfg.getString(config_name + ".ssl_key", ""); unsigned timeout = cfg.getInt(config_name + ".connect_timeout", @@ -102,7 +117,18 @@ public: cfg.getInt("mysql_rw_timeout", MYSQLXX_DEFAULT_RW_TIMEOUT)); - connect(db.c_str(), server.c_str(), user.c_str(), password.c_str(), port, socket.c_str(), timeout, rw_timeout); + connect( + db.c_str(), + server.c_str(), + user.c_str(), + password.c_str(), + port, + socket.c_str(), + ssl_ca.c_str(), + ssl_cert.c_str(), + ssl_key.c_str(), + timeout, + rw_timeout); } /// If MySQL connection was established. diff --git a/libs/libmysqlxx/include/mysqlxx/Pool.h b/libs/libmysqlxx/include/mysqlxx/Pool.h index 88921735355..d3ab7044cde 100644 --- a/libs/libmysqlxx/include/mysqlxx/Pool.h +++ b/libs/libmysqlxx/include/mysqlxx/Pool.h @@ -231,6 +231,9 @@ private: std::string socket; unsigned connect_timeout; unsigned rw_timeout; + std::string ssl_ca; + std::string ssl_cert; + std::string ssl_key; /// True if connection was established at least once. bool was_successful{false}; diff --git a/libs/libmysqlxx/src/Connection.cpp b/libs/libmysqlxx/src/Connection.cpp index f76ce3d473a..8d8bcb57fc3 100644 --- a/libs/libmysqlxx/src/Connection.cpp +++ b/libs/libmysqlxx/src/Connection.cpp @@ -3,6 +3,10 @@ #include #include +static inline const char* ifNotEmpty(const char* s) +{ + return s && *s ? s : nullptr; +} namespace mysqlxx { @@ -35,12 +39,15 @@ Connection::Connection( const char* password, unsigned port, const char * socket, + const char* ssl_ca, + const char* ssl_cert, + const char* ssl_key, unsigned timeout, unsigned rw_timeout) : driver(std::make_unique()) { is_connected = false; - connect(db, server, user, password, port, socket, timeout, rw_timeout); + connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout); } Connection::Connection(const std::string & config_name) @@ -62,6 +69,9 @@ void Connection::connect(const char* db, const char* password, unsigned port, const char * socket, + const char* ssl_ca, + const char* ssl_cert, + const char* ssl_key, unsigned timeout, unsigned rw_timeout) { @@ -88,7 +98,11 @@ void Connection::connect(const char* db, if (mysql_options(driver.get(), MYSQL_OPT_LOCAL_INFILE, nullptr)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); - if (!mysql_real_connect(driver.get(), server, user, password, db, port, *socket ? socket : nullptr, driver->client_flag)) + /// Specifies particular ssl key and certificate if it needs + if (mysql_ssl_set(driver.get(), ifNotEmpty(ssl_key), ifNotEmpty(ssl_cert), ifNotEmpty(ssl_ca), nullptr, nullptr)) + throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); + + if (!mysql_real_connect(driver.get(), server, user, password, db, port, ifNotEmpty(socket), driver->client_flag)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); /// Sets UTF-8 as default encoding. diff --git a/libs/libmysqlxx/src/Pool.cpp b/libs/libmysqlxx/src/Pool.cpp index b2217caf6fa..23d231e1aae 100644 --- a/libs/libmysqlxx/src/Pool.cpp +++ b/libs/libmysqlxx/src/Pool.cpp @@ -56,6 +56,15 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co socket = cfg.has(config_name + ".socket") ? cfg.getString(config_name + ".socket") : cfg.getString(parent_config_name + ".socket", ""); + ssl_ca = cfg.has(config_name + ".ssl_ca") + ? cfg.getString(config_name + ".ssl_ca") + : cfg.getString(parent_config_name + ".ssl_ca", ""); + ssl_cert = cfg.has(config_name + ".ssl_cert") + ? cfg.getString(config_name + ".ssl_cert") + : cfg.getString(parent_config_name + ".ssl_cert", ""); + ssl_key = cfg.has(config_name + ".ssl_key") + ? cfg.getString(config_name + ".ssl_key") + : cfg.getString(parent_config_name + ".ssl_key", ""); } else { @@ -68,6 +77,9 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co port = cfg.getInt(config_name + ".port", 0); socket = cfg.getString(config_name + ".socket", ""); + ssl_ca = cfg.getString(config_name + ".ssl_ca", ""); + ssl_cert = cfg.getString(config_name + ".ssl_cert", ""); + ssl_key = cfg.getString(config_name + ".ssl_key", ""); } connect_timeout = cfg.getInt(config_name + ".connect_timeout", @@ -167,13 +179,16 @@ void Pool::Entry::forceConnected() const app.logger().information("MYSQL: Reconnecting to " + pool->description); data->conn.connect( pool->db.c_str(), - pool->server.c_str(), - pool->user.c_str(), - pool->password.c_str(), - pool->port, - pool->socket.c_str(), - pool->connect_timeout, - pool->rw_timeout); + pool->server.c_str(), + pool->user.c_str(), + pool->password.c_str(), + pool->port, + pool->socket.c_str(), + pool->ssl_ca.c_str(), + pool->ssl_cert.c_str(), + pool->ssl_key.c_str(), + pool->connect_timeout, + pool->rw_timeout); } while (!data->conn.ping()); } @@ -205,13 +220,16 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) conn->conn.connect( db.c_str(), - server.c_str(), - user.c_str(), - password.c_str(), - port, - socket.c_str(), - connect_timeout, - rw_timeout); + server.c_str(), + user.c_str(), + password.c_str(), + port, + socket.c_str(), + ssl_ca.c_str(), + ssl_cert.c_str(), + ssl_key.c_str(), + connect_timeout, + rw_timeout); } catch (mysqlxx::ConnectionFailed & e) { From fabfed6f22811084b041bbf06dc1428655c4f7f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Sep 2017 09:51:27 +0300 Subject: [PATCH 101/108] Added support for "include-what-you-use" tool [#CLICKHOUSE-2]. --- CMakeLists.txt | 12 ++++++++++++ dbms/CMakeLists.txt | 4 ++++ 2 files changed, 16 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 87c4ef63e65..69b20377459 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -156,6 +156,18 @@ set (CMAKE_C_FLAGS_MSAN "${CMAKE_C_FLAGS_MSAN} ${SAN_FLAGS} set (CMAKE_CXX_FLAGS_TSAN "${CMAKE_CXX_FLAGS_TSAN} ${SAN_FLAGS} -fsanitize=thread") set (CMAKE_C_FLAGS_TSAN "${CMAKE_C_FLAGS_TSAN} ${SAN_FLAGS} -fsanitize=thread") +# Using "include-what-you-use" tool. +option (USE_INCLUDE_WHAT_YOU_USE "Use 'include-what-you-use' tool" OFF) +if (USE_INCLUDE_WHAT_YOU_USE) + find_program(IWYU_PATH NAMES include-what-you-use iwyu) + if (NOT IWYU_PATH) + message(FATAL_ERROR "Could not find the program include-what-you-use") + endif() + if (${CMAKE_VERSION} VERSION_LESS "3.3.0") + message(FATAL_ERROR "include-what-you-use requires CMake version at least 3.3.") + endif() +endif () + # Flags for test coverage if (TEST_COVERAGE) set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -fprofile-arcs -ftest-coverage -DIS_DEBUG") diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index dbf23edc479..6ae88e0df3e 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -1,3 +1,7 @@ +if (USE_INCLUDE_WHAT_YOU_USE) + set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH}) +endif () + include(${CMAKE_CURRENT_SOURCE_DIR}/cmake/find_vectorclass.cmake) set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_version.h) From 104c6d9ddf41ba0c8c6d16353ec7ccc469992380 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Sep 2017 08:33:36 +0300 Subject: [PATCH 102/108] Fixed warning in Poco [#CLICKHOUSE-2]. --- contrib/libpoco/MongoDB/include/Poco/MongoDB/Element.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libpoco/MongoDB/include/Poco/MongoDB/Element.h b/contrib/libpoco/MongoDB/include/Poco/MongoDB/Element.h index d2aff4a1caa..258e0e8b494 100644 --- a/contrib/libpoco/MongoDB/include/Poco/MongoDB/Element.h +++ b/contrib/libpoco/MongoDB/include/Poco/MongoDB/Element.h @@ -79,7 +79,7 @@ inline std::string Element::name() const class ElementComparator { public: - bool operator()(const Element::Ptr& s1, const Element::Ptr& s2) + bool operator()(const Element::Ptr& s1, const Element::Ptr& s2) const { return s1->name() < s2->name(); } From 9cafeb9e85ce094969e79ce46b85233bc682de4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Sep 2017 08:51:15 +0300 Subject: [PATCH 103/108] Fixed not-an-issue found by Coverity [#CLICKHOUSE-2]. --- dbms/src/IO/WriteBufferFromString.h | 2 +- dbms/src/IO/WriteBufferFromVector.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromString.h b/dbms/src/IO/WriteBufferFromString.h index 9545562669f..fe1be32266f 100644 --- a/dbms/src/IO/WriteBufferFromString.h +++ b/dbms/src/IO/WriteBufferFromString.h @@ -22,7 +22,7 @@ private: { size_t old_size = s.size(); s.resize(old_size * 2); - internal_buffer = Buffer(reinterpret_cast(&s[old_size]), reinterpret_cast(&*s.end())); + internal_buffer = Buffer(reinterpret_cast(&s[old_size]), reinterpret_cast(&s[s.size()])); working_buffer = internal_buffer; } diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index bea92121bfe..3c5ed53889c 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -24,7 +24,7 @@ private: { size_t old_size = vector.size(); vector.resize(old_size * 2); - internal_buffer = Buffer(reinterpret_cast(&vector[old_size]), reinterpret_cast(&*vector.end())); + internal_buffer = Buffer(reinterpret_cast(&vector[old_size]), reinterpret_cast(vector.data() + vector.size())); working_buffer = internal_buffer; } From 7576cb2c186c2d47b895185d621d2154953d238d Mon Sep 17 00:00:00 2001 From: Vadim Skipin Date: Mon, 11 Sep 2017 13:23:13 +0300 Subject: [PATCH 104/108] fix LRU cache definition --- dbms/src/Common/LRUCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/LRUCache.h b/dbms/src/Common/LRUCache.h index 4899d8f35e7..5b296da1959 100644 --- a/dbms/src/Common/LRUCache.h +++ b/dbms/src/Common/LRUCache.h @@ -29,7 +29,7 @@ struct TrivialWeightFunction /// Cache starts to evict entries when their total weight exceeds max_size and when expiration time of these /// entries is due. /// Value weight should not change after insertion. -template , typename WeightFunction = TrivialWeightFunction> +template , typename WeightFunction = TrivialWeightFunction> class LRUCache { public: From 06c7d4315fa8492de260a3b359bdc3abcf9efab0 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 11 Sep 2017 18:51:32 +0300 Subject: [PATCH 105/108] Cmake fixes (#CLICKHOUSE-3303) (#1229) * Cmake fixes (#CLICKHOUSE-3303) --- CMakeLists.txt | 5 +++-- cmake/find_ccache.cmake | 3 +-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 69b20377459..3e0fe38e84a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -36,6 +36,7 @@ if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") message (STATUS "CMAKE_BUILD_TYPE is not set, set to default = RELWITHDEBINFO") set (CMAKE_BUILD_TYPE "RELWITHDEBINFO") endif () +string(TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) message (STATUS "CMAKE_BUILD_TYPE: " ${CMAKE_BUILD_TYPE} ) # ASan - build type with address sanitizer @@ -234,8 +235,8 @@ include (libs/libmysqlxx/cmake/find_mysqlclient.cmake) include (libs/libdaemon/cmake/find_unwind.cmake) -set (FULL_C_FLAGS "${CMAKE_C_FLAGS} ${CMAKE_C_FLAGS_${CMAKE_BUILD_TYPE}}") -set (FULL_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CMAKE_CXX_FLAGS_${CMAKE_BUILD_TYPE}}") +set (FULL_C_FLAGS "${CMAKE_C_FLAGS} ${CMAKE_C_FLAGS_${CMAKE_BUILD_TYPE_UC}}") +set (FULL_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CMAKE_CXX_FLAGS_${CMAKE_BUILD_TYPE_UC}}") message (STATUS "C_FLAGS = ${FULL_C_FLAGS}") message (STATUS "CXX_FLAGS = ${FULL_CXX_FLAGS}") message (STATUS "LINK_FLAGS = ${CMAKE_EXE_LINKER_FLAGS}") diff --git a/cmake/find_ccache.cmake b/cmake/find_ccache.cmake index 926002a44ae..22372c3cbfe 100644 --- a/cmake/find_ccache.cmake +++ b/cmake/find_ccache.cmake @@ -3,9 +3,8 @@ find_program (CCACHE_FOUND ccache) if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CMAKE_CXX_COMPILER MATCHES "ccache") execute_process(COMMAND ${CCACHE_FOUND} "-V" OUTPUT_VARIABLE CCACHE_VERSION) string(REGEX REPLACE "ccache version ([0-9\\.]+).*" "\\1" CCACHE_VERSION ${CCACHE_VERSION} ) - string(COMPARE GREATER ${CCACHE_VERSION} "3.2.0" CCACHE_VERSION_OK) - if (CCACHE_VERSION_OK OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") + if (CCACHE_VERSION VERSION_GREATER "3.2.0" OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") #message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}") set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) From 9ab64b4706c4f1abc2c9273a4da788026614f5ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Sep 2017 21:03:32 +0300 Subject: [PATCH 106/108] Added link to Go library by Denis Korolev [#CLICKHOUSE-3]. --- docs/en/interfaces/third-party_client_libraries.rst | 3 ++- docs/ru/interfaces/third-party_client_libraries.rst | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/third-party_client_libraries.rst b/docs/en/interfaces/third-party_client_libraries.rst index 6d32eab73da..9cacf91f555 100644 --- a/docs/en/interfaces/third-party_client_libraries.rst +++ b/docs/en/interfaces/third-party_client_libraries.rst @@ -12,9 +12,10 @@ There exist third-party client libraries for ClickHouse: - `PhpClickHouseClient `_ - `phpClickHouse `_ * Go - - `clickhouse (Go) `_ + - `clickhouse `_ - `go-clickhouse `_ - `mailru\go-clickhouse `_ + - `golang-clickhouse `_ * NodeJs - `clickhouse (NodeJs) `_ - `node-clickhouse `_ diff --git a/docs/ru/interfaces/third-party_client_libraries.rst b/docs/ru/interfaces/third-party_client_libraries.rst index 6c8a0f09359..9757af0f712 100644 --- a/docs/ru/interfaces/third-party_client_libraries.rst +++ b/docs/ru/interfaces/third-party_client_libraries.rst @@ -12,9 +12,10 @@ - `PhpClickHouseClient `_ - `phpClickHouse `_ * Go - - `clickhouse (Go) `_ + - `clickhouse `_ - `go-clickhouse `_ - `mailru\go-clickhouse `_ + - `golang-clickhouse `_ * NodeJs - `clickhouse (NodeJs) `_ - `node-clickhouse `_ From 5054d503070a04dc172053516da2f26eb82121c1 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 11 Sep 2017 20:22:43 +0300 Subject: [PATCH 107/108] Remove too noisy message (#CLICKHOUSE-3304) --- dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index dc24c934baf..9f0a4498762 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -532,8 +532,6 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & new_ if (future_part.contains(result_part)) { - out_reason = "Not executing log entry for part " + new_part_name - + " because another log entry for covering part " + future_part_name + " is being processed."; return false; } } From e0a1e6982b9f1962b1443186974a1164b0f143d6 Mon Sep 17 00:00:00 2001 From: Vadim Skipin Date: Mon, 11 Sep 2017 15:39:01 +0300 Subject: [PATCH 108/108] Context parameter added to the IDatabase methods --- dbms/src/Databases/DatabaseDictionary.cpp | 69 +++++++------ dbms/src/Databases/DatabaseDictionary.h | 70 ++++++++----- dbms/src/Databases/DatabaseMemory.cpp | 99 ++++++++++++------- dbms/src/Databases/DatabaseMemory.h | 51 +++++++--- dbms/src/Databases/DatabaseOrdinary.cpp | 47 ++++++--- dbms/src/Databases/DatabaseOrdinary.h | 38 ++++--- dbms/src/Databases/IDatabase.h | 44 ++++++--- dbms/src/Interpreters/AsynchronousMetrics.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 12 +-- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 6 +- .../Interpreters/InterpreterRenameQuery.cpp | 2 +- dbms/src/Server/ReplicasStatusHandler.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 4 +- .../Storages/System/StorageSystemColumns.cpp | 2 +- .../Storages/System/StorageSystemParts.cpp | 2 +- .../Storages/System/StorageSystemReplicas.cpp | 2 +- .../System/StorageSystemReplicationQueue.cpp | 2 +- .../Storages/System/StorageSystemTables.cpp | 4 +- .../src/TableFunctions/TableFunctionMerge.cpp | 2 +- 20 files changed, 301 insertions(+), 161 deletions(-) diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index 5c32988845e..f0ca52ad6fc 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -46,13 +46,17 @@ Tables DatabaseDictionary::loadTables() return tables; } -bool DatabaseDictionary::isTableExist(const String & table_name) const +bool DatabaseDictionary::isTableExist( + const Context & context, + const String & table_name) const { const std::lock_guard lock_dictionaries {external_dictionaries.dictionaries_mutex}; return external_dictionaries.dictionaries.count(table_name) && !deleted_tables.count(table_name); } -StoragePtr DatabaseDictionary::tryGetTable(const String & table_name) +StoragePtr DatabaseDictionary::tryGetTable( + const Context & context, + const String & table_name) { const std::lock_guard lock_dictionaries {external_dictionaries.dictionaries_mutex}; @@ -75,12 +79,12 @@ StoragePtr DatabaseDictionary::tryGetTable(const String & table_name) return {}; } -DatabaseIteratorPtr DatabaseDictionary::getIterator() +DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context) { return std::make_unique(loadTables()); } -bool DatabaseDictionary::empty() const +bool DatabaseDictionary::empty(const Context & context) const { const std::lock_guard lock_dictionaries {external_dictionaries.dictionaries_mutex}; for (const auto & pair : external_dictionaries.dictionaries) @@ -99,39 +103,58 @@ void DatabaseDictionary::attachTable(const String & table_name, const StoragePtr throw Exception("DatabaseDictionary: attachTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } -void DatabaseDictionary::createTable(const String & table_name, - const StoragePtr & table, - const ASTPtr & query, - const String & engine, - const Settings & settings) +void DatabaseDictionary::createTable( + const Context & context, + const String & table_name, + const StoragePtr & table, + const ASTPtr & query, + const String & engine) { throw Exception("DatabaseDictionary: createTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } -void DatabaseDictionary::removeTable(const String & table_name) +void DatabaseDictionary::removeTable( + const Context & context, + const String & table_name) { - if (!isTableExist(table_name)) + if (!isTableExist(context, table_name)) throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); const std::lock_guard lock_dictionaries {external_dictionaries.dictionaries_mutex}; deleted_tables.insert(table_name); } -void DatabaseDictionary::renameTable(const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name, - const Settings & settings) +void DatabaseDictionary::renameTable( + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name) { throw Exception("DatabaseDictionary: renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } -time_t DatabaseDictionary::getTableMetadataModificationTime(const String & table_name) +void DatabaseDictionary::alterTable( + const Context & context, + const String & name, + const NamesAndTypesList & columns, + const NamesAndTypesList & materialized_columns, + const NamesAndTypesList & alias_columns, + const ColumnDefaults & column_defaults, + const ASTModifier & engine_modifier) +{ + throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); +} + +time_t DatabaseDictionary::getTableMetadataModificationTime( + const Context & context, + const String & table_name) { return static_cast(0); } -ASTPtr DatabaseDictionary::getCreateQuery(const String & table_name) const +ASTPtr DatabaseDictionary::getCreateQuery( + const Context & context, + const String & table_name) const { throw Exception("DatabaseDictionary: getCreateQuery() is not supported", ErrorCodes::NOT_IMPLEMENTED); return nullptr; @@ -146,14 +169,4 @@ void DatabaseDictionary::drop() /// Additional actions to delete database are not required. } -void DatabaseDictionary::alterTable(const Context & context, - const String & name, - const NamesAndTypesList & columns, - const NamesAndTypesList & materialized_columns, - const NamesAndTypesList & alias_columns, - const ColumnDefaults & column_defaults, - const ASTModifier & engine_modifier) -{ - throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); -} } diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index 6dfe4aeed06..fe0e2ffeb93 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -39,45 +39,63 @@ public: { return "Dictionary"; } - void loadTables(Context & context, ThreadPool * thread_pool, bool has_force_restore_data_flag) override; - bool isTableExist(const String & table_name) const override; - StoragePtr tryGetTable(const String & table_name) override; + void loadTables( + Context & context, + ThreadPool * thread_pool, + bool has_force_restore_data_flag) override; - DatabaseIteratorPtr getIterator() override; + bool isTableExist( + const Context & context, + const String & table_name) const override; - bool empty() const override; + StoragePtr tryGetTable( + const Context & context, + const String & table_name) override; - void createTable(const String & table_name, - const StoragePtr & table, - const ASTPtr & query, - const String & engine, - const Settings & settings) override; + DatabaseIteratorPtr getIterator(const Context & context) override; - void removeTable(const String & table_name) override; + bool empty(const Context & context) const override; + + void createTable( + const Context & context, + const String & table_name, + const StoragePtr & table, + const ASTPtr & query, + const String & engine) override; + + void removeTable( + const Context & context, + const String & table_name) override; void attachTable(const String & table_name, const StoragePtr & table) override; StoragePtr detachTable(const String & table_name) override; - void renameTable(const Context & context, - const String & table_name, - IDatabase & to_database, - const String & to_table_name, - const Settings & settings) override; + void renameTable( + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name) override; - time_t getTableMetadataModificationTime(const String & table_name) override; + void alterTable( + const Context & context, + const String & name, + const NamesAndTypesList & columns, + const NamesAndTypesList & materialized_columns, + const NamesAndTypesList & alias_columns, + const ColumnDefaults & column_defaults, + const ASTModifier & engine_modifier) override; - ASTPtr getCreateQuery(const String & table_name) const override; + time_t getTableMetadataModificationTime( + const Context & context, + const String & table_name) override; + + ASTPtr getCreateQuery( + const Context & context, + const String & table_name) const override; void shutdown() override; void drop() override; - - void alterTable(const Context & context, - const String & name, - const NamesAndTypesList & columns, - const NamesAndTypesList & materialized_columns, - const NamesAndTypesList & alias_columns, - const ColumnDefaults & column_defaults, - const ASTModifier & engine_modifier) override; }; + } diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 2d9ee679d58..aae8f781cde 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -13,20 +13,27 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void DatabaseMemory::loadTables(Context & context, ThreadPool * thread_pool, bool has_force_restore_data_flag) +void DatabaseMemory::loadTables( + Context & context, + ThreadPool * thread_pool, + bool has_force_restore_data_flag) { log = &Logger::get("DatabaseMemory(" + name + ")"); /// Nothing to load. } -bool DatabaseMemory::isTableExist(const String & table_name) const +bool DatabaseMemory::isTableExist( + const Context & context, + const String & table_name) const { std::lock_guard lock(mutex); return tables.count(table_name); } -StoragePtr DatabaseMemory::tryGetTable(const String & table_name) +StoragePtr DatabaseMemory::tryGetTable( + const Context & context, + const String & table_name) { std::lock_guard lock(mutex); auto it = tables.find(table_name); @@ -35,13 +42,13 @@ StoragePtr DatabaseMemory::tryGetTable(const String & table_name) return it->second; } -DatabaseIteratorPtr DatabaseMemory::getIterator() +DatabaseIteratorPtr DatabaseMemory::getIterator(const Context & context) { std::lock_guard lock(mutex); return std::make_unique(tables); } -bool DatabaseMemory::empty() const +bool DatabaseMemory::empty(const Context & context) const { std::lock_guard lock(mutex); return tables.empty(); @@ -70,50 +77,31 @@ void DatabaseMemory::attachTable(const String & table_name, const StoragePtr & t } void DatabaseMemory::createTable( - const String & table_name, const StoragePtr & table, const ASTPtr & query, const String & engine, const Settings & settings) + const Context & context, + const String & table_name, + const StoragePtr & table, + const ASTPtr & query, + const String & engine) { attachTable(table_name, table); } -void DatabaseMemory::removeTable(const String & table_name) +void DatabaseMemory::removeTable( + const Context & context, + const String & table_name) { detachTable(table_name); } void DatabaseMemory::renameTable( - const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name, const Settings & settings) + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name) { throw Exception("DatabaseMemory: renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } -time_t DatabaseMemory::getTableMetadataModificationTime(const String & table_name) -{ - return static_cast(0); -} - -ASTPtr DatabaseMemory::getCreateQuery(const String & table_name) const -{ - throw Exception("DatabaseMemory: getCreateQuery() is not supported", ErrorCodes::NOT_IMPLEMENTED); - return nullptr; -} - -void DatabaseMemory::shutdown() -{ - /// You can not hold a lock during shutdown. - /// Because inside `shutdown` function tables can work with database, and mutex is not recursive. - - for (auto iterator = getIterator(); iterator->isValid(); iterator->next()) - iterator->table()->shutdown(); - - std::lock_guard lock(mutex); - tables.clear(); -} - -void DatabaseMemory::drop() -{ - /// Additional actions to delete database are not required. -} - void DatabaseMemory::alterTable( const Context & context, const String & name, @@ -126,4 +114,43 @@ void DatabaseMemory::alterTable( throw Exception("DatabaseMemory: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } +time_t DatabaseMemory::getTableMetadataModificationTime( + const Context & context, + const String & table_name) +{ + return static_cast(0); +} + +ASTPtr DatabaseMemory::getCreateQuery( + const Context & context, + const String & table_name) const +{ + throw Exception("DatabaseMemory: getCreateQuery() is not supported", ErrorCodes::NOT_IMPLEMENTED); + return nullptr; +} + +void DatabaseMemory::shutdown() +{ + /// You can not hold a lock during shutdown. + /// Because inside `shutdown` function tables can work with database, and mutex is not recursive. + + Tables tables_snapshot; + { + std::lock_guard lock(mutex); + tables_snapshot = tables; + } + + for (const auto & kv: tables_snapshot) { + kv.second->shutdown(); + } + + std::lock_guard lock(mutex); + tables.clear(); +} + +void DatabaseMemory::drop() +{ + /// Additional actions to delete database are not required. +} + } diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 4dc75c7dfff..d4d15ad61e0 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -31,32 +31,42 @@ public: String getEngineName() const override { return "Memory"; } - void loadTables(Context & context, ThreadPool * thread_pool, bool has_force_restore_data_flag) override; + void loadTables( + Context & context, + ThreadPool * thread_pool, + bool has_force_restore_data_flag) override; - bool isTableExist(const String & table_name) const override; - StoragePtr tryGetTable(const String & table_name) override; + bool empty(const Context & context) const override; - DatabaseIteratorPtr getIterator() override; + DatabaseIteratorPtr getIterator(const Context & context) override; - bool empty() const override; + bool isTableExist( + const Context & context, + const String & table_name) const override; + + StoragePtr tryGetTable( + const Context & context, + const String & table_name) override; void createTable( - const String & table_name, const StoragePtr & table, const ASTPtr & query, const String & engine, const Settings & settings) override; + const Context & context, + const String & table_name, + const StoragePtr & table, + const ASTPtr & query, + const String & engine) override; - void removeTable(const String & table_name) override; + void removeTable( + const Context & context, + const String & table_name) override; void attachTable(const String & table_name, const StoragePtr & table) override; StoragePtr detachTable(const String & table_name) override; void renameTable( - const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name, const Settings & settings) override; - - time_t getTableMetadataModificationTime(const String & table_name) override; - - ASTPtr getCreateQuery(const String & table_name) const override; - - void shutdown() override; - void drop() override; + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name) override; void alterTable( const Context & context, @@ -66,6 +76,17 @@ public: const NamesAndTypesList & alias_columns, const ColumnDefaults & column_defaults, const ASTModifier & engine_modifier) override; + + time_t getTableMetadataModificationTime( + const Context & context, + const String & table_name) override; + + ASTPtr getCreateQuery( + const Context & context, + const String & table_name) const override; + + void shutdown() override; + void drop() override; }; } diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 609b20c6383..67b4a4e986a 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -97,7 +97,10 @@ DatabaseOrdinary::DatabaseOrdinary( } -void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, bool has_force_restore_data_flag) +void DatabaseOrdinary::loadTables( + Context & context, + ThreadPool * thread_pool, + bool has_force_restore_data_flag) { log = &Logger::get("DatabaseOrdinary (" + name + ")"); @@ -241,8 +244,14 @@ void DatabaseOrdinary::startupTables(ThreadPool * thread_pool) void DatabaseOrdinary::createTable( - const String & table_name, const StoragePtr & table, const ASTPtr & query, const String & engine, const Settings & settings) + const Context & context, + const String & table_name, + const StoragePtr & table, + const ASTPtr & query, + const String & engine) { + const auto & settings = context.getSettingsRef(); + /// Create a file with metadata if necessary - if the query is not ATTACH. /// Write the query of `ATTACH table` to it. @@ -298,7 +307,9 @@ void DatabaseOrdinary::createTable( } -void DatabaseOrdinary::removeTable(const String & table_name) +void DatabaseOrdinary::removeTable( + const Context & context, + const String & table_name) { StoragePtr res = detachTable(table_name); @@ -332,14 +343,17 @@ static ASTPtr getCreateQueryImpl(const String & path, const String & table_name) void DatabaseOrdinary::renameTable( - const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name, const Settings & settings) + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name) { DatabaseOrdinary * to_database_concrete = typeid_cast(&to_database); if (!to_database_concrete) throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); - StoragePtr table = tryGetTable(table_name); + StoragePtr table = tryGetTable(context, table_name); if (!table) throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -366,12 +380,14 @@ void DatabaseOrdinary::renameTable( ast_create_query.table = to_table_name; /// NOTE Non-atomic. - to_database_concrete->createTable(to_table_name, table, ast, table->getName(), settings); - removeTable(table_name); + to_database_concrete->createTable(context, to_table_name, table, ast, table->getName()); + removeTable(context, table_name); } -time_t DatabaseOrdinary::getTableMetadataModificationTime(const String & table_name) +time_t DatabaseOrdinary::getTableMetadataModificationTime( + const Context & context, + const String & table_name) { String table_metadata_path = getTableMetadataPath(path, table_name); Poco::File meta_file(table_metadata_path); @@ -387,7 +403,9 @@ time_t DatabaseOrdinary::getTableMetadataModificationTime(const String & table_n } -ASTPtr DatabaseOrdinary::getCreateQuery(const String & table_name) const +ASTPtr DatabaseOrdinary::getCreateQuery( + const Context & context, + const String & table_name) const { ASTPtr ast = getCreateQueryImpl(path, table_name); @@ -404,8 +422,15 @@ void DatabaseOrdinary::shutdown() /// You can not hold a lock during shutdown. /// Because inside `shutdown` function the tables can work with database, and mutex is not recursive. - for (auto iterator = getIterator(); iterator->isValid(); iterator->next()) - iterator->table()->shutdown(); + Tables tables_snapshot; + { + std::lock_guard lock(mutex); + tables_snapshot = tables; + } + + for (const auto & kv: tables_snapshot) { + kv.second->shutdown(); + } std::lock_guard lock(mutex); tables.clear(); diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index 6e04bd57d5d..5666beb007f 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -20,22 +20,27 @@ public: String getEngineName() const override { return "Ordinary"; } - void loadTables(Context & context, ThreadPool * thread_pool, bool has_force_restore_data_flag) override; + void loadTables( + Context & context, + ThreadPool * thread_pool, + bool has_force_restore_data_flag) override; void createTable( - const String & table_name, const StoragePtr & table, const ASTPtr & query, const String & engine, const Settings & settings) override; + const Context & context, + const String & table_name, + const StoragePtr & table, + const ASTPtr & query, + const String & engine) override; - void removeTable(const String & table_name) override; + void removeTable( + const Context & context, + const String & table_name) override; void renameTable( - const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name, const Settings & settings) override; - - time_t getTableMetadataModificationTime(const String & table_name) override; - - ASTPtr getCreateQuery(const String & table_name) const override; - - void shutdown() override; - void drop() override; + const Context & context, + const String & table_name, + IDatabase & to_database, + const String & to_table_name) override; void alterTable( const Context & context, @@ -46,6 +51,17 @@ public: const ColumnDefaults & column_defaults, const ASTModifier & engine_modifier) override; + time_t getTableMetadataModificationTime( + const Context & context, + const String & table_name) override; + + ASTPtr getCreateQuery( + const Context & context, + const String & table_name) const override; + + void shutdown() override; + void drop() override; + private: void startupTables(ThreadPool * thread_pool); }; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 027c11c3cd6..33f0b637224 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -59,27 +59,40 @@ public: /// Load a set of existing tables. If thread_pool is specified, use it. /// You can call only once, right after the object is created. - virtual void loadTables(Context & context, ThreadPool * thread_pool, bool has_force_restore_data_flag) = 0; + virtual void loadTables( + Context & context, + ThreadPool * thread_pool, + bool has_force_restore_data_flag) = 0; /// Check the existence of the table. - virtual bool isTableExist(const String & name) const = 0; + virtual bool isTableExist( + const Context & context, + const String & name) const = 0; /// Get the table for work. Return nullptr if there is no table. - virtual StoragePtr tryGetTable(const String & name) = 0; + virtual StoragePtr tryGetTable( + const Context & context, + const String & name) = 0; /// Get an iterator that allows you to pass through all the tables. /// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above. - virtual DatabaseIteratorPtr getIterator() = 0; + virtual DatabaseIteratorPtr getIterator(const Context & context) = 0; /// Is the database empty. - virtual bool empty() const = 0; + virtual bool empty(const Context & context) const = 0; /// Add the table to the database. Record its presence in the metadata. virtual void createTable( - const String & name, const StoragePtr & table, const ASTPtr & query, const String & engine, const Settings & settings) = 0; + const Context & context, + const String & name, + const StoragePtr & table, + const ASTPtr & query, + const String & engine) = 0; /// Delete the table from the database and return it. Delete the metadata. - virtual void removeTable(const String & name) = 0; + virtual void removeTable( + const Context & context, + const String & name) = 0; /// Add a table to the database, but do not add it to the metadata. The database may not support this method. virtual void attachTable(const String & name, const StoragePtr & table) = 0; @@ -89,10 +102,10 @@ public: /// Rename the table and possibly move the table to another database. virtual void renameTable( - const Context & context, const String & name, IDatabase & to_database, const String & to_name, const Settings & settings) = 0; - - /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. - virtual time_t getTableMetadataModificationTime(const String & name) = 0; + const Context & context, + const String & name, + IDatabase & to_database, + const String & to_name) = 0; using ASTModifier = std::function; @@ -107,8 +120,15 @@ public: const ColumnDefaults & column_defaults, const ASTModifier & engine_modifier) = 0; + /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. + virtual time_t getTableMetadataModificationTime( + const Context & context, + const String & name) = 0; + /// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata. - virtual ASTPtr getCreateQuery(const String & name) const = 0; + virtual ASTPtr getCreateQuery( + const Context & context, + const String & name) const = 0; /// Ask all tables to complete the background threads they are using and delete all table objects. virtual void shutdown() = 0; diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index 823e6eab2e3..fcc3394c3d0 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -146,7 +146,7 @@ void AsynchronousMetrics::update() for (const auto & db : databases) { - for (auto iterator = db.second->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) { auto & table = iterator->table(); StorageMergeTree * table_merge_tree = typeid_cast(table.get()); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 3663597f334..1e0023a1b3e 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -621,7 +621,7 @@ bool Context::isTableExist(const String & database_name, const String & table_na Databases::const_iterator it = shared->databases.find(db); return shared->databases.end() != it - && it->second->isTableExist(table_name); + && it->second->isTableExist(*this, table_name); } @@ -645,7 +645,7 @@ void Context::assertTableExists(const String & database_name, const String & tab if (shared->databases.end() == it) throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); - if (!it->second->isTableExist(table_name)) + if (!it->second->isTableExist(*this, table_name)) throw Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); } @@ -659,7 +659,7 @@ void Context::assertTableDoesntExist(const String & database_name, const String checkDatabaseAccessRights(db); Databases::const_iterator it = shared->databases.find(db); - if (shared->databases.end() != it && it->second->isTableExist(table_name)) + if (shared->databases.end() != it && it->second->isTableExist(*this, table_name)) throw Exception("Table " + db + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } @@ -758,7 +758,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta return {}; } - auto table = it->second->tryGetTable(table_name); + auto table = it->second->tryGetTable(*this, table_name); if (!table) { if (exception) @@ -826,7 +826,7 @@ std::unique_ptr Context::getDDLGuardIfTableDoesntExist(const String & auto lock = getLock(); Databases::const_iterator it = shared->databases.find(database); - if (shared->databases.end() != it && it->second->isTableExist(table)) + if (shared->databases.end() != it && it->second->isTableExist(*this, table)) return {}; return getDDLGuard(database, table, message); @@ -859,7 +859,7 @@ ASTPtr Context::getCreateQuery(const String & database_name, const String & tabl String db = resolveDatabase(database_name, current_database); assertDatabaseExists(db); - return shared->databases[db]->getCreateQuery(table_name); + return shared->databases[db]->getCreateQuery(*this, table_name); } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 3be60614bd2..6d570ad09d4 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -549,7 +549,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.is_temporary) context.getSessionContext().addExternalTable(table_name, res); else - context.getDatabase(database_name)->createTable(table_name, res, query_ptr, storage_name, context.getSettingsRef()); + context.getDatabase(database_name)->createTable(context, table_name, res, query_ptr, storage_name); } res->startup(); diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index cefb092820f..3d769b3e583 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -99,7 +99,7 @@ BlockIO InterpreterDropQuery::execute() return {}; } - for (auto iterator = database->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next()) tables_to_drop.emplace_back(iterator->table(), context.getDDLGuard(database_name, iterator->name(), "Table " + database_name + "." + iterator->name() + " is dropping or detaching right now")); } @@ -128,7 +128,7 @@ BlockIO InterpreterDropQuery::execute() else { /// Delete table metdata and table itself from memory - database->removeTable(current_table_name); + database->removeTable(context, current_table_name); /// Delete table data table.first->drop(); @@ -151,7 +151,7 @@ BlockIO InterpreterDropQuery::execute() context.assertDatabaseExists(database_name); /// Someone could have time to create a table in the database to be deleted while we deleted the tables without the context lock. - if (!context.getDatabase(database_name)->empty()) + if (!context.getDatabase(database_name)->empty(context)) throw Exception("New table appeared in database being dropped. Try dropping it again.", ErrorCodes::DATABASE_NOT_EMPTY); /// Delete database information from the RAM diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index d626e80a06b..8068463a845 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -118,7 +118,7 @@ BlockIO InterpreterRenameQuery::execute() context.assertTableDoesntExist(elem.to_database_name, elem.to_table_name); context.getDatabase(elem.from_database_name)->renameTable( - context, elem.from_table_name, *context.getDatabase(elem.to_database_name), elem.to_table_name, context.getSettingsRef()); + context, elem.from_table_name, *context.getDatabase(elem.to_database_name), elem.to_table_name); } return {}; diff --git a/dbms/src/Server/ReplicasStatusHandler.cpp b/dbms/src/Server/ReplicasStatusHandler.cpp index dfbd6cfc765..f1709be880c 100644 --- a/dbms/src/Server/ReplicasStatusHandler.cpp +++ b/dbms/src/Server/ReplicasStatusHandler.cpp @@ -40,7 +40,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request /// Iterate through all the replicated tables. for (const auto & db : databases) { - for (auto iterator = db.second->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) { auto & table = iterator->table(); StorageReplicatedMergeTree * table_replicated = typeid_cast(table.get()); diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index d5a33ba1323..6f83e98ff59 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -58,7 +58,7 @@ StorageMerge::StorageMerge( bool StorageMerge::isRemote() const { auto database = context.getDatabase(source_database); - auto iterator = database->getIterator(); + auto iterator = database->getIterator(context); while (iterator->isValid()) { @@ -327,7 +327,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables() const { StorageListWithLocks selected_tables; auto database = context.getDatabase(source_database); - auto iterator = database->getIterator(); + auto iterator = database->getIterator(context); while (iterator->isValid()) { diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index b0df8444fcb..58b30d6455f 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -74,7 +74,7 @@ BlockInputStreams StorageSystemColumns::read( const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next()) { const String & table_name = iterator->name(); storages.emplace(std::piecewise_construct, diff --git a/dbms/src/Storages/System/StorageSystemParts.cpp b/dbms/src/Storages/System/StorageSystemParts.cpp index 3b053aed166..b3835740f33 100644 --- a/dbms/src/Storages/System/StorageSystemParts.cpp +++ b/dbms/src/Storages/System/StorageSystemParts.cpp @@ -93,7 +93,7 @@ BlockInputStreams StorageSystemParts::read( const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next()) { String table_name = iterator->name(); StoragePtr storage = iterator->table(); diff --git a/dbms/src/Storages/System/StorageSystemReplicas.cpp b/dbms/src/Storages/System/StorageSystemReplicas.cpp index 9a1074154ae..e907a48d762 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicas.cpp @@ -62,7 +62,7 @@ BlockInputStreams StorageSystemReplicas::read( /// We collect a set of replicated tables. std::map> replicated_tables; for (const auto & db : context.getDatabases()) - for (auto iterator = db.second->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) if (typeid_cast(iterator->table().get())) replicated_tables[db.first][iterator->name()] = iterator->table(); diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp index 5c3c7f5d931..ddea933578d 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -60,7 +60,7 @@ BlockInputStreams StorageSystemReplicationQueue::read( std::map> replicated_tables; for (const auto & db : context.getDatabases()) - for (auto iterator = db.second->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) if (typeid_cast(iterator->table().get())) replicated_tables[db.first][iterator->name()] = iterator->table(); diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 3f91703c507..f39186487ac 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -100,13 +100,13 @@ BlockInputStreams StorageSystemTables::read( continue; } - for (auto iterator = database->getIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next()) { auto table_name = iterator->name(); col_db.column->insert(database_name); col_name.column->insert(table_name); col_engine.column->insert(iterator->table()->getName()); - col_meta_mod_time.column->insert(static_cast(database->getTableMetadataModificationTime(table_name))); + col_meta_mod_time.column->insert(static_cast(database->getTableMetadataModificationTime(context, table_name))); } } diff --git a/dbms/src/TableFunctions/TableFunctionMerge.cpp b/dbms/src/TableFunctions/TableFunctionMerge.cpp index d6e2c7d4dc7..cf9f96f2f8d 100644 --- a/dbms/src/TableFunctions/TableFunctionMerge.cpp +++ b/dbms/src/TableFunctions/TableFunctionMerge.cpp @@ -33,7 +33,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str { auto database = context.getDatabase(source_database); - auto iterator = database->getIterator(); + auto iterator = database->getIterator(context); while (iterator->isValid()) {