From 61bef2adf1d0b49489629fa35f6d11dd5384ca51 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Oct 2017 13:31:06 +0300 Subject: [PATCH 01/32] added ExternalLoader [#CLICKHOUSE-3305] --- dbms/src/Databases/DatabaseDictionary.cpp | 35 +- dbms/src/Dictionaries/DictionaryFactory.h | 2 +- .../Dictionaries/DictionarySourceFactory.cpp | 2 +- .../Dictionaries/DictionarySourceFactory.h | 2 +- dbms/src/Dictionaries/DictionaryStructure.cpp | 10 - dbms/src/Dictionaries/DictionaryStructure.h | 9 +- dbms/src/Dictionaries/IDictionary.h | 26 +- dbms/src/Interpreters/DictionaryFactory.cpp | 2 +- .../src/Interpreters/ExternalDictionaries.cpp | 428 ++---------------- dbms/src/Interpreters/ExternalDictionaries.h | 119 ++--- dbms/src/Interpreters/ExternalLoader.cpp | 417 +++++++++++++++++ dbms/src/Interpreters/ExternalLoader.h | 158 +++++++ dbms/src/Interpreters/IExternalLoadable.h | 39 ++ .../System/StorageSystemDictionaries.cpp | 9 +- 14 files changed, 718 insertions(+), 540 deletions(-) create mode 100644 dbms/src/Interpreters/ExternalLoader.cpp create mode 100644 dbms/src/Interpreters/ExternalLoader.h create mode 100644 dbms/src/Interpreters/IExternalLoadable.h diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index f0ca52ad6fc..0dc471c9e0a 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -26,18 +26,19 @@ void DatabaseDictionary::loadTables(Context & context, ThreadPool * thread_pool, Tables DatabaseDictionary::loadTables() { - const std::lock_guard lock_dictionaries {external_dictionaries.dictionaries_mutex}; + auto objects_map = external_dictionaries.getObjectsMap(); + const auto & dictionaries = std::get<1>(objects_map); Tables tables; - for (const auto & pair : external_dictionaries.dictionaries) + for (const auto & pair : dictionaries) { const std::string & name = pair.first; if (deleted_tables.count(name)) continue; - auto dict_ptr = pair.second.dict; + auto dict_ptr = std::static_pointer_cast(pair.second.loadable); if (dict_ptr) { - const DictionaryStructure & dictionary_structure = dict_ptr->get()->getStructure(); + const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); tables[name] = StorageDictionary::create(name, columns, {}, {}, {}, dictionary_structure, name); } @@ -50,26 +51,28 @@ 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); + auto objects_map = external_dictionaries.getObjectsMap(); + const auto & dictionaries = std::get<1>(objects_map); + return dictionaries.count(table_name) && !deleted_tables.count(table_name); } StoragePtr DatabaseDictionary::tryGetTable( const Context & context, const String & table_name) { - const std::lock_guard lock_dictionaries {external_dictionaries.dictionaries_mutex}; + auto objects_map = external_dictionaries.getObjectsMap(); + const auto & dictionaries = std::get<1>(objects_map); if (deleted_tables.count(table_name)) return {}; { - auto it = external_dictionaries.dictionaries.find(table_name); - if (it != external_dictionaries.dictionaries.end()) + auto it = dictionaries.find(table_name); + if (it != dictionaries.end()) { - const auto & dict_ptr = it->second.dict; + const auto & dict_ptr = std::static_pointer_cast(it->second.loadable); if (dict_ptr) { - const DictionaryStructure & dictionary_structure = dict_ptr->get()->getStructure(); + const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); return StorageDictionary::create(table_name, columns, {}, {}, {}, dictionary_structure, table_name); } @@ -86,9 +89,10 @@ DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context) bool DatabaseDictionary::empty(const Context & context) const { - const std::lock_guard lock_dictionaries {external_dictionaries.dictionaries_mutex}; - for (const auto & pair : external_dictionaries.dictionaries) - if (pair.second.dict && !deleted_tables.count(pair.first)) + auto objects_map = external_dictionaries.getObjectsMap(); + const auto & dictionaries = std::get<1>(objects_map); + for (const auto & pair : dictionaries) + if (pair.second.loadable && !deleted_tables.count(pair.first)) return false; return true; } @@ -120,7 +124,7 @@ void DatabaseDictionary::removeTable( 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}; + auto objects_map = external_dictionaries.getObjectsMap(); deleted_tables.insert(table_name); } @@ -157,7 +161,6 @@ ASTPtr DatabaseDictionary::getCreateQuery( const String & table_name) const { throw Exception("DatabaseDictionary: getCreateQuery() is not supported", ErrorCodes::NOT_IMPLEMENTED); - return nullptr; } void DatabaseDictionary::shutdown() diff --git a/dbms/src/Dictionaries/DictionaryFactory.h b/dbms/src/Dictionaries/DictionaryFactory.h index 0b5e5004265..bd8f6d18af2 100644 --- a/dbms/src/Dictionaries/DictionaryFactory.h +++ b/dbms/src/Dictionaries/DictionaryFactory.h @@ -13,7 +13,7 @@ class Context; class DictionaryFactory : public ext::singleton { public: - DictionaryPtr create(const std::string & name, Poco::Util::AbstractConfiguration & config, + DictionaryPtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const; }; diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.cpp b/dbms/src/Dictionaries/DictionarySourceFactory.cpp index e8dcb9e0d3d..75ab62c183d 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.cpp +++ b/dbms/src/Dictionaries/DictionarySourceFactory.cpp @@ -90,7 +90,7 @@ DictionarySourceFactory::DictionarySourceFactory() DictionarySourcePtr DictionarySourceFactory::create( - const std::string & name, Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, Context & context) const { Poco::Util::AbstractConfiguration::Keys keys; diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.h b/dbms/src/Dictionaries/DictionarySourceFactory.h index f4ea6ebb6ad..5792e9ba44f 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.h +++ b/dbms/src/Dictionaries/DictionarySourceFactory.h @@ -25,7 +25,7 @@ public: DictionarySourceFactory(); DictionarySourcePtr create( - const std::string & name, Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, Context & context) const; }; diff --git a/dbms/src/Dictionaries/DictionaryStructure.cpp b/dbms/src/Dictionaries/DictionaryStructure.cpp index f96c0f9262b..9fdad17c1d0 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.cpp +++ b/dbms/src/Dictionaries/DictionaryStructure.cpp @@ -105,16 +105,6 @@ std::string toString(const AttributeUnderlyingType type) } -DictionaryLifetime::DictionaryLifetime(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) -{ - const auto & lifetime_min_key = config_prefix + ".min"; - const auto has_min = config.has(lifetime_min_key); - - this->min_sec = has_min ? config.getInt(lifetime_min_key) : config.getInt(config_prefix); - this->max_sec = has_min ? config.getInt(config_prefix + ".max") : this->min_sec; -} - - DictionarySpecialAttribute::DictionarySpecialAttribute(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) : name{config.getString(config_prefix + ".name", "")}, expression{config.getString(config_prefix + ".expression", "")} diff --git a/dbms/src/Dictionaries/DictionaryStructure.h b/dbms/src/Dictionaries/DictionaryStructure.h index 7fc54b3139c..81107cf8f18 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.h +++ b/dbms/src/Dictionaries/DictionaryStructure.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -42,13 +43,7 @@ std::string toString(const AttributeUnderlyingType type); /// Min and max lifetimes for a dictionary or it's entry -struct DictionaryLifetime final -{ - UInt64 min_sec; - UInt64 max_sec; - - DictionaryLifetime(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); -}; +using DictionaryLifetime = ExternalLoadableLifetime; /** Holds the description of a single dictionary attribute: diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index 7ca6526f60c..f378a9a7b07 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -24,14 +25,10 @@ class IBlockInputStream; using BlockInputStreamPtr = std::shared_ptr; -struct IDictionaryBase : public std::enable_shared_from_this +struct IDictionaryBase : public IExternalLoadable { using Key = UInt64; - virtual std::exception_ptr getCreationException() const = 0; - - virtual std::string getName() const = 0; - virtual std::string getTypeName() const = 0; virtual size_t getBytesAllocated() const = 0; @@ -45,12 +42,11 @@ struct IDictionaryBase : public std::enable_shared_from_this virtual double getLoadFactor() const = 0; virtual bool isCached() const = 0; + virtual DictionaryPtr clone() const = 0; virtual const IDictionarySource * getSource() const = 0; - virtual const DictionaryLifetime & getLifetime() const = 0; - virtual const DictionaryStructure & getStructure() const = 0; virtual std::chrono::time_point getCreationTime() const = 0; @@ -59,7 +55,21 @@ struct IDictionaryBase : public std::enable_shared_from_this virtual BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const = 0; - virtual ~IDictionaryBase() = default; + bool supportUpdates() const override { return !isCached(); } + + virtual std::shared_ptr cloneObject() const override + { + return std::static_pointer_cast(clone()); + }; + + std::shared_ptr shared_from_this() + { + return std::static_pointer_cast(IExternalLoadable::shared_from_this()); + } + std::shared_ptr shared_from_this() const + { + return std::static_pointer_cast(IExternalLoadable::shared_from_this()); + } }; diff --git a/dbms/src/Interpreters/DictionaryFactory.cpp b/dbms/src/Interpreters/DictionaryFactory.cpp index a160c763971..b1db641ba85 100644 --- a/dbms/src/Interpreters/DictionaryFactory.cpp +++ b/dbms/src/Interpreters/DictionaryFactory.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes } -DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::AbstractConfiguration & config, +DictionaryPtr DictionaryFactory::create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const { Poco::Util::AbstractConfiguration::Keys keys; diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 9fcb8510404..b42d703d090 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -1,427 +1,59 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace -{ - const auto check_period_sec = 5; - const auto backoff_initial_sec = 5; - /// 10 minutes - const auto backoff_max_sec = 10 * 60; -} namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; -} - - -void ExternalDictionaries::reloadPeriodically() -{ - setThreadName("ExterDictReload"); - - while (true) - { - if (destroy.tryWait(check_period_sec * 1000)) - return; - - reloadAndUpdate(); - } -} - - -ExternalDictionaries::ExternalDictionaries(Context & context, const bool throw_on_error) - : context(context), log(&Logger::get("ExternalDictionaries")) -{ - { - /** During synchronous loading of external dictionaries at moment of query execution, - * we should not use per query memory limit. - */ - TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker; - - reloadAndUpdate(throw_on_error); - } - - reloading_thread = std::thread{&ExternalDictionaries::reloadPeriodically, this}; -} - - -ExternalDictionaries::~ExternalDictionaries() -{ - destroy.set(); - reloading_thread.join(); -} - namespace { -std::set getDictionariesConfigPaths(const Poco::Util::AbstractConfiguration & config) -{ - std::set files; - auto patterns = getMultipleValuesFromConfig(config, "", "dictionaries_config"); - for (auto & pattern : patterns) + const ExternalLoaderUpdateSettings & getExternalDictionariesUpdateSettings() { - if (pattern.empty()) - continue; + static ExternalLoaderUpdateSettings settings; + static std::once_flag flag; - if (pattern[0] != '/') - { - const auto app_config_path = config.getString("config-file", "config.xml"); - const auto config_dir = Poco::Path{app_config_path}.parent().toString(); - const auto absolute_path = config_dir + pattern; - Poco::Glob::glob(absolute_path, files, 0); - if (!files.empty()) - continue; - } + std::call_once(flag, [] { + settings.check_period_sec = 5; + settings.backoff_initial_sec = 5; + /// 10 minutes + settings.backoff_max_sec = 10 * 60; + }); - Poco::Glob::glob(pattern, files, 0); + return settings; } - return files; -} -} - -void ExternalDictionaries::reloadAndUpdate(bool throw_on_error) -{ - reloadFromConfigFiles(throw_on_error); - - /// list of recreated dictionaries to perform delayed removal from unordered_map - std::list recreated_failed_dictionaries; - - std::unique_lock all_lock(all_mutex); - - /// retry loading failed dictionaries - for (auto & failed_dictionary : failed_dictionaries) + const ExternalLoaderConfigSettings & getExternalDictionariesConfigSettings() { - if (std::chrono::system_clock::now() < failed_dictionary.second.next_attempt_time) - continue; + static ExternalLoaderConfigSettings settings; + static std::once_flag flag; - const auto & name = failed_dictionary.first; + std::call_once(flag, [] { + settings.external_config = "dictionary"; + settings.external_name = "name"; - try - { - auto dict_ptr = failed_dictionary.second.dict->clone(); - if (const auto exception_ptr = dict_ptr->getCreationException()) - { - /// recalculate next attempt time - std::uniform_int_distribution distribution( - 0, static_cast(std::exp2(failed_dictionary.second.error_count))); + settings.path_setting_name = "dictionaries_config"; + }); - failed_dictionary.second.next_attempt_time = std::chrono::system_clock::now() + - std::chrono::seconds{ - std::min(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine))}; - - ++failed_dictionary.second.error_count; - - std::rethrow_exception(exception_ptr); - } - else - { - const std::lock_guard lock{dictionaries_mutex}; - - const auto & lifetime = dict_ptr->getLifetime(); - std::uniform_int_distribution distribution{lifetime.min_sec, lifetime.max_sec}; - update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}; - - const auto dict_it = dictionaries.find(name); - if (dict_it->second.dict) - dict_it->second.dict->set(dict_ptr.release()); - else - dict_it->second.dict = std::make_shared>(dict_ptr.release()); - - /// clear stored exception on success - dict_it->second.exception = std::exception_ptr{}; - - recreated_failed_dictionaries.push_back(name); - } - } - catch (...) - { - tryLogCurrentException(log, "Failed reloading '" + name + "' dictionary"); - - if (throw_on_error) - throw; - } - } - - /// do not undertake further attempts to recreate these dictionaries - for (const auto & name : recreated_failed_dictionaries) - failed_dictionaries.erase(name); - - /// periodic update - for (auto & dictionary : dictionaries) - { - const auto & name = dictionary.first; - - try - { - /// If the dictionary failed to load or even failed to initialize from the config. - if (!dictionary.second.dict) - continue; - - auto current = dictionary.second.dict->get(); - const auto & lifetime = current->getLifetime(); - - /// do not update dictionaries with zero as lifetime - if (lifetime.min_sec == 0 || lifetime.max_sec == 0) - continue; - - /// update only non-cached dictionaries - if (!current->isCached()) - { - auto & update_time = update_times[current->getName()]; - - /// check that timeout has passed - if (std::chrono::system_clock::now() < update_time) - continue; - - SCOPE_EXIT({ - /// calculate next update time - std::uniform_int_distribution distribution{lifetime.min_sec, lifetime.max_sec}; - update_time = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}; - }); - - /// check source modified - if (current->getSource()->isModified()) - { - /// create new version of dictionary - auto new_version = current->clone(); - - if (const auto exception_ptr = new_version->getCreationException()) - std::rethrow_exception(exception_ptr); - - dictionary.second.dict->set(new_version.release()); - } - } - - /// erase stored exception on success - dictionary.second.exception = std::exception_ptr{}; - } - catch (...) - { - dictionary.second.exception = std::current_exception(); - - tryLogCurrentException(log, "Cannot update external dictionary '" + name + "', leaving old version"); - - if (throw_on_error) - throw; - } + return settings; } } -void ExternalDictionaries::reloadFromConfigFiles(const bool throw_on_error, const bool force_reload, const std::string & only_dictionary) + +ExternalDictionaries::ExternalDictionaries(Context & context, bool throw_on_error) + : ExternalLoader(context.getConfigRef(), + getExternalDictionariesUpdateSettings(), + getExternalDictionariesConfigSettings(), + &Logger::get("ExternalDictionaries"), + "external dictionary", throw_on_error), + context(context) { - const auto config_paths = getDictionariesConfigPaths(context.getConfigRef()); - for (const auto & config_path : config_paths) - { - try - { - reloadFromConfigFile(config_path, throw_on_error, force_reload, only_dictionary); - } - catch (...) - { - tryLogCurrentException(log, "reloadFromConfigFile has thrown while reading from " + config_path); - - if (throw_on_error) - throw; - } - } } -void ExternalDictionaries::reloadFromConfigFile(const std::string & config_path, const bool throw_on_error, const bool force_reload, - const std::string & only_dictionary) +ExternalDictionaries::LoadablePtr ExternalDictionaries::create( + const std::string & name, const Configuration & config, const std::string & config_prefix) { - const Poco::File config_file{config_path}; - - if (config_path.empty() || !config_file.exists()) - { - LOG_WARNING(log, "config file '" + config_path + "' does not exist"); - } - else - { - std::unique_lock all_lock(all_mutex); - - auto modification_time_it = last_modification_times.find(config_path); - if (modification_time_it == std::end(last_modification_times)) - modification_time_it = last_modification_times.emplace(config_path, Poco::Timestamp{0}).first; - auto & config_last_modified = modification_time_it->second; - - const auto last_modified = config_file.getLastModified(); - if (force_reload || last_modified > config_last_modified) - { - Poco::AutoPtr config = new Poco::Util::XMLConfiguration(config_path); - - /// Definitions of dictionaries may have changed, recreate all of them - - /// If we need update only one dictionary, don't update modification time: might be other dictionaries in the config file - if (only_dictionary.empty()) - config_last_modified = last_modified; - - /// get all dictionaries' definitions - Poco::Util::AbstractConfiguration::Keys keys; - config->keys(keys); - - /// for each dictionary defined in xml config - for (const auto & key : keys) - { - std::string name; - - if (!startsWith(key, "dictionary")) - { - if (!startsWith(key.data(), "comment")) - LOG_WARNING(log, - config_path << ": unknown node in dictionaries file: '" << key + "', 'dictionary'"); - - continue; - } - - try - { - name = config->getString(key + ".name"); - if (name.empty()) - { - LOG_WARNING(log, config_path << ": dictionary name cannot be empty"); - continue; - } - - if (!only_dictionary.empty() && name != only_dictionary) - continue; - - decltype(dictionaries.begin()) dict_it; - { - std::lock_guard lock{dictionaries_mutex}; - dict_it = dictionaries.find(name); - } - - if (dict_it != std::end(dictionaries) && dict_it->second.origin != config_path) - throw std::runtime_error{"Overriding dictionary from file " + dict_it->second.origin}; - - auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context); - - /// If the dictionary could not be loaded. - if (const auto exception_ptr = dict_ptr->getCreationException()) - { - const auto failed_dict_it = failed_dictionaries.find(name); - if (failed_dict_it != std::end(failed_dictionaries)) - { - failed_dict_it->second = FailedDictionaryInfo{ - std::move(dict_ptr), - 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::rethrow_exception(exception_ptr); - } - else if (!dict_ptr->isCached()) - { - const auto & lifetime = dict_ptr->getLifetime(); - if (lifetime.min_sec != 0 && lifetime.max_sec != 0) - { - std::uniform_int_distribution distribution{ - lifetime.min_sec, - lifetime.max_sec - }; - update_times[name] = std::chrono::system_clock::now() + - std::chrono::seconds{distribution(rnd_engine)}; - } - } - - const std::lock_guard lock{dictionaries_mutex}; - - /// add new dictionary or update an existing version - if (dict_it == std::end(dictionaries)) - dictionaries.emplace(name, DictionaryInfo{ - std::make_shared>(dict_ptr.release()), - config_path - }); - else - { - if (dict_it->second.dict) - dict_it->second.dict->set(dict_ptr.release()); - else - dict_it->second.dict = std::make_shared>(dict_ptr.release()); - - /// erase stored exception on success - dict_it->second.exception = std::exception_ptr{}; - failed_dictionaries.erase(name); - } - } - catch (...) - { - if (!name.empty()) - { - /// If the dictionary could not load data or even failed to initialize from the config. - /// - all the same we insert information into the `dictionaries`, with the zero pointer `dict`. - - const std::lock_guard lock{dictionaries_mutex}; - - const auto exception_ptr = std::current_exception(); - const auto dict_it = dictionaries.find(name); - if (dict_it == std::end(dictionaries)) - dictionaries.emplace(name, DictionaryInfo{nullptr, config_path, exception_ptr}); - else - dict_it->second.exception = exception_ptr; - } - - tryLogCurrentException(log, "Cannot create external dictionary '" + name + "' from config path " + config_path); - - /// propagate exception - if (throw_on_error) - throw; - } - } - } - } -} - -void ExternalDictionaries::reload() -{ - reloadFromConfigFiles(true, true); -} - -void ExternalDictionaries::reloadDictionary(const std::string & name) -{ - reloadFromConfigFiles(true, true, name); - - /// Check that specified dict was loaded - const std::lock_guard lock{dictionaries_mutex}; - if (!dictionaries.count(name)) - throw Exception("Dictionary " + name + " wasn't loaded during the reload process", ErrorCodes::BAD_ARGUMENTS); -} - -MultiVersion::Version ExternalDictionaries::getDictionary(const std::string & name) const -{ - const std::lock_guard lock{dictionaries_mutex}; - - const auto it = dictionaries.find(name); - if (it == std::end(dictionaries)) - throw Exception{ - "No such dictionary: " + name, - ErrorCodes::BAD_ARGUMENTS - }; - - if (!it->second.dict) - it->second.exception ? std::rethrow_exception(it->second.exception) : - throw Exception{"No dictionary", ErrorCodes::LOGICAL_ERROR}; - - return it->second.dict->get(); + return DictionaryFactory::instance().create(name, config, config_prefix, context); } } diff --git a/dbms/src/Interpreters/ExternalDictionaries.h b/dbms/src/Interpreters/ExternalDictionaries.h index 84488118e19..31ff6ad5341 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.h +++ b/dbms/src/Interpreters/ExternalDictionaries.h @@ -1,19 +1,9 @@ #pragma once #include -#include -#include -#include -#include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include +#include namespace DB @@ -21,93 +11,36 @@ namespace DB class Context; -/** Manages user-defined dictionaries. -* Monitors configuration file and automatically reloads dictionaries in a separate thread. -* The monitoring thread wakes up every @check_period_sec seconds and checks -* modification time of dictionaries' configuration file. If said time is greater than -* @config_last_modified, the dictionaries are created from scratch using configuration file, -* possibly overriding currently existing dictionaries with the same name (previous versions of -* overridden dictionaries will live as long as there are any users retaining them). -* -* Apart from checking configuration file for modifications, each non-cached dictionary -* has a lifetime of its own and may be updated if it's source reports that it has been -* modified. The time of next update is calculated by choosing uniformly a random number -* distributed between lifetime.min_sec and lifetime.max_sec. -* If either of lifetime.min_sec and lifetime.max_sec is zero, such dictionary is never updated. -*/ -class ExternalDictionaries +/// Manages user-defined dictionaries. +class ExternalDictionaries : public ExternalLoader { -private: +public: + using DictPtr = std::shared_ptr; + + /// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. + ExternalDictionaries(Context & context, bool throw_on_error); + + /// Forcibly reloads specified dictionary. + void reloadDictionary(const std::string & name) { reload(name); } + + DictPtr getDictionary(const std::string & name) const + { + return std::static_pointer_cast(getLoadable(name)); + } + +protected: + + LoadablePtr create(const std::string & name, const Configuration & config, + const std::string & config_prefix) override; + + using ExternalLoader::getObjectsMap; + friend class StorageSystemDictionaries; friend class DatabaseDictionary; - /// Protects only dictionaries map. - mutable std::mutex dictionaries_mutex; - - /// Protects all data, currently used to avoid races between updating thread and SYSTEM queries - mutable std::mutex all_mutex; - - using DictionaryPtr = std::shared_ptr>; - struct DictionaryInfo final - { - DictionaryPtr dict; - std::string origin; - std::exception_ptr exception; - }; - - struct FailedDictionaryInfo final - { - std::unique_ptr dict; - std::chrono::system_clock::time_point next_attempt_time; - UInt64 error_count; - }; - - /** name -> dictionary. - */ - std::unordered_map dictionaries; - - /** Here are dictionaries, that has been never loaded successfully. - * They are also in 'dictionaries', but with nullptr as 'dict'. - */ - std::unordered_map failed_dictionaries; - - /** Both for dictionaries and failed_dictionaries. - */ - std::unordered_map update_times; - - pcg64 rnd_engine{randomSeed()}; +private: Context & context; - - std::thread reloading_thread; - Poco::Event destroy; - - Logger * log; - - std::unordered_map last_modification_times; - - /// Check dictionaries definitions in config files and reload or/and add new ones if the definition is changed - void reloadFromConfigFiles(const bool throw_on_error, const bool force_reload = false, const std::string & only_dictionary = ""); - void reloadFromConfigFile(const std::string & config_path, const bool throw_on_error, const bool force_reload, - const std::string & only_dictionary); - - /// Check config files and update expired dictionaries - void reloadAndUpdate(bool throw_on_error = false); - - void reloadPeriodically(); - -public: - /// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. - ExternalDictionaries(Context & context, const bool throw_on_error); - ~ExternalDictionaries(); - - /// Forcibly reloads all dictionaries. - void reload(); - - /// Forcibly reloads specified dictionary. - void reloadDictionary(const std::string & name); - - MultiVersion::Version getDictionary(const std::string & name) const; }; } diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp new file mode 100644 index 00000000000..42967f541c8 --- /dev/null +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -0,0 +1,417 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +} + + +ExternalLoadableLifetime::ExternalLoadableLifetime(const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix) +{ + const auto & lifetime_min_key = config_prefix + ".min"; + const auto has_min = config.has(lifetime_min_key); + + min_sec = has_min ? config.getUInt64(lifetime_min_key) : config.getUInt64(config_prefix); + max_sec = has_min ? config.getUInt64(config_prefix + ".max") : min_sec; +} + +void ExternalLoader::reloadPeriodically() +{ + setThreadName("ExterLdrReload"); + + while (true) + { + if (destroy.tryWait(update_settings.check_period_sec * 1000)) + return; + + reloadAndUpdate(); + } +} + + +ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & config, + const ExternalLoaderUpdateSettings & update_settings, + const ExternalLoaderConfigSettings & config_settings, + Logger * log, const std::string & loadable_object_name, bool throw_on_error) + : config(config), update_settings(update_settings), config_settings(config_settings), + log(log), object_name(loadable_object_name) +{ + { + /// During synchronous loading of external dictionaries at moment of query execution, + /// we should not use per query memory limit. + TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker; + + reloadAndUpdate(throw_on_error); + } + + reloading_thread = std::thread{&ExternalLoader::reloadPeriodically, this}; +} + + +ExternalLoader::~ExternalLoader() +{ + destroy.set(); + reloading_thread.join(); +} + +namespace +{ +std::set getConfigPaths(const Poco::Util::AbstractConfiguration & config, + const std::string & external_config_paths_setting) +{ + std::set files; + auto patterns = getMultipleValuesFromConfig(config, "", external_config_paths_setting); + for (auto & pattern : patterns) + { + if (pattern.empty()) + continue; + + if (pattern[0] != '/') + { + const auto app_config_path = config.getString("config-file", "config.xml"); + const auto config_dir = Poco::Path{app_config_path}.parent().toString(); + const auto absolute_path = config_dir + pattern; + Poco::Glob::glob(absolute_path, files, 0); + if (!files.empty()) + continue; + } + + Poco::Glob::glob(pattern, files, 0); + } + + return files; +} +} + +void ExternalLoader::reloadAndUpdate(bool throw_on_error) +{ + reloadFromConfigFiles(throw_on_error); + + /// list of recreated loadable objects to perform delayed removal from unordered_map + std::list recreated_failed_loadable_objects; + + std::unique_lock all_lock(all_mutex); + + /// retry loading failed loadable objects + for (auto & failed_loadable_object : failed_loadable_objects) + { + if (std::chrono::system_clock::now() < failed_loadable_object.second.next_attempt_time) + continue; + + const auto & name = failed_loadable_object.first; + + try + { + auto loadable_ptr = failed_loadable_object.second.loadable->clone(); + if (const auto exception_ptr = loadable_ptr->getCreationException()) + { + /// recalculate next attempt time + std::uniform_int_distribution distribution( + 0, static_cast(std::exp2(failed_loadable_object.second.error_count))); + + std::chrono::seconds delay(std::min( + update_settings.backoff_max_sec, + update_settings.backoff_initial_sec + distribution(rnd_engine))); + failed_loadable_object.second.next_attempt_time = std::chrono::system_clock::now() + delay; + + ++failed_loadable_object.second.error_count; + + std::rethrow_exception(exception_ptr); + } + else + { + const std::lock_guard lock{map_mutex}; + + const auto & lifetime = loadable_ptr->getLifetime(); + std::uniform_int_distribution distribution{lifetime.min_sec, lifetime.max_sec}; + update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}; + + const auto dict_it = loadable_objects.find(name); + + dict_it->second.loadable.reset(); + dict_it->second.loadable = loadable_ptr; + + /// clear stored exception on success + dict_it->second.exception = std::exception_ptr{}; + + recreated_failed_loadable_objects.push_back(name); + } + } + catch (...) + { + tryLogCurrentException(log, "Failed reloading '" + name + "' " + object_name); + + if (throw_on_error) + throw; + } + } + + /// do not undertake further attempts to recreate these loadable objects + for (const auto & name : recreated_failed_loadable_objects) + failed_loadable_objects.erase(name); + + /// periodic update + for (auto & loadable_object : loadable_objects) + { + const auto & name = loadable_object.first; + + try + { + /// If the loadable objects failed to load or even failed to initialize from the config. + if (!loadable_object.second.loadable) + continue; + + auto current = loadable_object.second.loadable; + const auto & lifetime = current->getLifetime(); + + /// do not update loadable objects with zero as lifetime + if (lifetime.min_sec == 0 || lifetime.max_sec == 0) + continue; + + if (current->supportUpdates()) + { + auto & update_time = update_times[current->getName()]; + + /// check that timeout has passed + if (std::chrono::system_clock::now() < update_time) + continue; + + SCOPE_EXIT({ + /// calculate next update time + std::uniform_int_distribution distribution{lifetime.min_sec, lifetime.max_sec}; + update_time = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}; + }); + + /// check source modified + if (current->isModified()) + { + /// create new version of loadable object + auto new_version = current->clone(); + + if (const auto exception_ptr = new_version->getCreationException()) + std::rethrow_exception(exception_ptr); + + loadable_object.second.loadable.reset(); + loadable_object.second.loadable = new_version; + } + } + + /// erase stored exception on success + loadable_object.second.exception = std::exception_ptr{}; + } + catch (...) + { + loadable_object.second.exception = std::current_exception(); + + tryLogCurrentException(log, "Cannot update " + object_name + " '" + name + "', leaving old version"); + + if (throw_on_error) + throw; + } + } +} + +void ExternalLoader::reloadFromConfigFiles(const bool throw_on_error, const bool force_reload, const std::string & only_dictionary) +{ + const auto config_paths = getConfigPaths(config, config_settings.path_setting_name); + + for (const auto & config_path : config_paths) + { + try + { + reloadFromConfigFile(config_path, throw_on_error, force_reload, only_dictionary); + } + catch (...) + { + tryLogCurrentException(log, "reloadFromConfigFile has thrown while reading from " + config_path); + + if (throw_on_error) + throw; + } + } +} + +void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const bool throw_on_error, + const bool force_reload, const std::string & loadable_name) +{ + const Poco::File config_file{config_path}; + + if (config_path.empty() || !config_file.exists()) + { + LOG_WARNING(log, "config file '" + config_path + "' does not exist"); + } + else + { + std::unique_lock all_lock(all_mutex); + + auto modification_time_it = last_modification_times.find(config_path); + if (modification_time_it == std::end(last_modification_times)) + modification_time_it = last_modification_times.emplace(config_path, Poco::Timestamp{0}).first; + auto & config_last_modified = modification_time_it->second; + + const auto last_modified = config_file.getLastModified(); + if (force_reload || last_modified > config_last_modified) + { + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(config_path); + + /// Definitions of loadable objects may have changed, recreate all of them + + /// If we need update only one object, don't update modification time: might be other objects in the config file + if (loadable_name.empty()) + config_last_modified = last_modified; + + /// get all objects' definitions + Poco::Util::AbstractConfiguration::Keys keys; + config->keys(keys); + + /// for each loadable object defined in xml config + for (const auto & key : keys) + { + std::string name; + + if (!startsWith(key, config_settings.external_config)) + { + if (!startsWith(key, "comment")) + LOG_WARNING(log, config_path << ": unknown node in file: '" << key + << "', expected '" << config_settings.external_config << "'"); + continue; + } + + try + { + name = config->getString(key + "." + config_settings.external_name); + if (name.empty()) + { + LOG_WARNING(log, config_path << ": " + config_settings.external_name + " name cannot be empty"); + continue; + } + + if (!loadable_name.empty() && name != loadable_name) + continue; + + decltype(loadable_objects.begin()) object_it; + { + std::lock_guard lock{map_mutex}; + object_it = loadable_objects.find(name); + } + + if (object_it != std::end(loadable_objects) && object_it->second.origin != config_path) + throw std::runtime_error{"Overriding " + object_name + " from file " + object_it->second.origin}; + + auto object_ptr = create(name, *config, key); + + /// If the object could not be loaded. + if (const auto exception_ptr = object_ptr->getCreationException()) + { + std::chrono::seconds delay(update_settings.backoff_initial_sec); + const auto failed_dict_it = failed_loadable_objects.find(name); + FailedLoadableInfo info{std::move(object_ptr), std::chrono::system_clock::now() + delay, 0}; + if (failed_dict_it != std::end(failed_loadable_objects)) + failed_dict_it->second = info; + else + failed_loadable_objects.emplace(name, info); + + std::rethrow_exception(exception_ptr); + } + else if (object_ptr->supportUpdates()) + { + const auto & lifetime = object_ptr->getLifetime(); + if (lifetime.min_sec != 0 && lifetime.max_sec != 0) + { + std::uniform_int_distribution distribution(lifetime.min_sec, lifetime.max_sec); + + update_times[name] = std::chrono::system_clock::now() + + std::chrono::seconds{distribution(rnd_engine)}; + } + } + + const std::lock_guard lock{map_mutex}; + + /// add new loadable object or update an existing version + if (object_it == std::end(loadable_objects)) + loadable_objects.emplace(name, LoadableInfo{object_ptr, config_path}); + else + { + if (object_it->second.loadable) + object_it->second.loadable.reset(); + object_it->second.loadable = object_ptr; + + /// erase stored exception on success + object_it->second.exception = std::exception_ptr{}; + failed_loadable_objects.erase(name); + } + } + catch (...) + { + if (!name.empty()) + { + /// If the loadable object could not load data or even failed to initialize from the config. + /// - all the same we insert information into the `loadable_objects`, with the zero pointer `loadable`. + + const std::lock_guard lock{map_mutex}; + + const auto exception_ptr = std::current_exception(); + const auto loadable_it = loadable_objects.find(name); + if (loadable_it == std::end(loadable_objects)) + loadable_objects.emplace(name, LoadableInfo{nullptr, config_path, exception_ptr}); + else + loadable_it->second.exception = exception_ptr; + } + + tryLogCurrentException(log, "Cannot create " + object_name + " '" + + name + "' from config path " + config_path); + + /// propagate exception + if (throw_on_error) + throw; + } + } + } + } +} + +void ExternalLoader::reload() +{ + reloadFromConfigFiles(true, true); +} + +void ExternalLoader::reload(const std::string & name) +{ + reloadFromConfigFiles(true, true, name); + + /// Check that specified object was loaded + const std::lock_guard lock{map_mutex}; + if (!loadable_objects.count(name)) + throw Exception("Failed to load " + object_name + " '" + name + "' during the reload process", ErrorCodes::BAD_ARGUMENTS); +} + +ExternalLoader::LoadablePtr ExternalLoader::getLoadable(const std::string & name) const +{ + const std::lock_guard lock{map_mutex}; + + const auto it = loadable_objects.find(name); + if (it == std::end(loadable_objects)) + throw Exception("No such " + object_name + ": " + name, ErrorCodes::BAD_ARGUMENTS); + + if (!it->second.loadable) + it->second.exception ? std::rethrow_exception(it->second.exception) : + throw Exception{object_name + " '" + name + "' is not loaded", ErrorCodes::LOGICAL_ERROR}; + + return it->second.loadable; +} + +} diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h new file mode 100644 index 00000000000..d330bae2633 --- /dev/null +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -0,0 +1,158 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class Context; + +struct ExternalLoaderUpdateSettings +{ + UInt64 check_period_sec = 5; + UInt64 backoff_initial_sec = 5; + /// 10 minutes + UInt64 backoff_max_sec = 10 * 60; +}; + + +/* External configuration structure. + * + * + * + * name + * .... + * + * + */ +struct ExternalLoaderConfigSettings +{ + std::string external_config; + std::string external_name; + + std::string path_setting_name; +}; + +/** Manages user-defined objects. +* Monitors configuration file and automatically reloads objects in a separate thread. +* The monitoring thread wakes up every @check_period_sec seconds and checks +* modification time of objects' configuration file. If said time is greater than +* @config_last_modified, the objects are created from scratch using configuration file, +* possibly overriding currently existing objects with the same name (previous versions of +* overridden objects will live as long as there are any users retaining them). +* +* Apart from checking configuration file for modifications, each object +* has a lifetime of its own and may be updated if it supportUpdates. +* The time of next update is calculated by choosing uniformly a random number +* distributed between lifetime.min_sec and lifetime.max_sec. +* If either of lifetime.min_sec and lifetime.max_sec is zero, such object is never updated. +*/ +class ExternalLoader +{ +public: + using Configuration = Poco::Util::AbstractConfiguration; + using ObjectsMap = std::unordered_map; + + /// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. + ExternalLoader(const Configuration & config, + const ExternalLoaderUpdateSettings & update_settings, + const ExternalLoaderConfigSettings & config_settings, + Logger * log, const std::string & loadable_object_name, bool throw_on_error); + ~ExternalLoader(); + + /// Forcibly reloads all loadable objects. + void reload(); + + /// Forcibly reloads specified loadable object. + void reload(const std::string & name); + + LoadablePtr getLoadable(const std::string & name) const; + +protected: + virtual LoadablePtr create(const std::string & name, const Configuration & config, + const std::string & config_prefix) = 0; + + /// Direct access to objects. + std::tuple, const ObjectsMap &> getObjectsMap() + { + return std::make_tuple(std::lock_guard(map_mutex), std::cref(loadable_objects)); + } + +private: + + /// Protects only dictionaries map. + mutable std::mutex map_mutex; + + /// Protects all data, currently used to avoid races between updating thread and SYSTEM queries + mutable std::mutex all_mutex; + + using LoadablePtr = std::shared_ptr; + struct LoadableInfo final + { + LoadablePtr loadable; + std::string origin; + std::exception_ptr exception; + }; + + struct FailedLoadableInfo final + { + std::unique_ptr loadable; + std::chrono::system_clock::time_point next_attempt_time; + UInt64 error_count; + }; + + /// name -> loadable. + ObjectsMap loadable_objects; + + /** Here are loadable objects, that has been never loaded successfully. + * They are also in 'loadable_objects', but with nullptr as 'loadable'. + */ + std::unordered_map failed_loadable_objects; + + /** Both for dictionaries and failed_dictionaries. + */ + std::unordered_map update_times; + + pcg64 rnd_engine{randomSeed()}; + + const Configuration & config; + const ExternalLoaderUpdateSettings & update_settings; + const ExternalLoaderConfigSettings & config_settings; + + std::thread reloading_thread; + Poco::Event destroy; + + Logger * log; + /// Loadable object name to use in log messages. + std::string object_name; + + std::unordered_map last_modification_times; + + /// Check dictionaries definitions in config files and reload or/and add new ones if the definition is changed + /// If loadable_name is not empty, load only loadable object with name loadable_name + void reloadFromConfigFiles(bool throw_on_error, bool force_reload = false, const std::string & loadable_name = ""); + void reloadFromConfigFile(const std::string & config_path, bool throw_on_error, bool force_reload, + const std::string & loadable_name); + + /// Check config files and update expired loadable objects + void reloadAndUpdate(bool throw_on_error = false); + + void reloadPeriodically(); +}; + +} diff --git a/dbms/src/Interpreters/IExternalLoadable.h b/dbms/src/Interpreters/IExternalLoadable.h new file mode 100644 index 00000000000..841aba86dfa --- /dev/null +++ b/dbms/src/Interpreters/IExternalLoadable.h @@ -0,0 +1,39 @@ +#include +#include + +namespace DB +{ + +namespace Poco::Util +{ +class AbstractConfiguration; +} + +/// Min and max lifetimes for a loadable object or it's entry +struct ExternalLoadableLifetime final +{ + UInt64 min_sec; + UInt64 max_sec; + + ExternalLoadableLifetime(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); +}; + +class IExternalLoadable : public std::enable_shared_from_this +{ +public: + virtual ~IExternalLoadable() = default; + + virtual const ExternalLoadableLifetime & getLifetime() const = 0; + + virtual std::string getName() const = 0; + + virtual bool supportUpdates() const = 0; + + virtual bool isModified() const = 0; + + virtual std::shared_ptr cloneObject() const = 0; + + virtual std::exception_ptr getCreationException() const = 0; +}; + +} diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index 62f00b00323..02a8c7a44aa 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -76,16 +76,17 @@ BlockInputStreams StorageSystemDictionaries::read( ColumnWithTypeAndName col_source{std::make_shared(), std::make_shared(), "source"}; const auto & external_dictionaries = context.getExternalDictionaries(); - const std::lock_guard lock{external_dictionaries.dictionaries_mutex}; + auto objects_map = external_dictionaries.getObjectsMap(); + const auto & dictionaries = std::get<1>(objects_map); - for (const auto & dict_info : external_dictionaries.dictionaries) + for (const auto & dict_info : dictionaries) { col_name.column->insert(dict_info.first); col_origin.column->insert(dict_info.second.origin); - if (dict_info.second.dict) + if (dict_info.second.loadable) { - const auto dict_ptr = dict_info.second.dict->get(); + const auto dict_ptr = std::static_pointer_cast(dict_info.second.loadable); col_type.column->insert(dict_ptr->getTypeName()); From 99e9c0c4864894de5986d4c627769c18c6d0fb9c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Oct 2017 14:10:01 +0300 Subject: [PATCH 02/32] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/IDictionary.h | 13 ++- .../src/Interpreters/ExternalDictionaries.cpp | 6 +- dbms/src/Interpreters/ExternalDictionaries.h | 4 +- dbms/src/Interpreters/ExternalLoader.cpp | 34 +++++-- dbms/src/Interpreters/ExternalLoader.h | 97 +++++++++---------- dbms/src/Interpreters/IExternalLoadable.h | 12 ++- 6 files changed, 94 insertions(+), 72 deletions(-) diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index f378a9a7b07..48b8206fbca 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -8,16 +8,14 @@ #include #include #include +#include namespace DB { -class IDictionarySource; - struct IDictionaryBase; using DictionaryPtr = std::unique_ptr; -struct DictionaryLifetime; struct DictionaryStructure; class ColumnString; @@ -57,9 +55,14 @@ struct IDictionaryBase : public IExternalLoadable bool supportUpdates() const override { return !isCached(); } - virtual std::shared_ptr cloneObject() const override + bool isModified() const override + { auto source = getSource(); + return source && source->isModified(); + } + + std::unique_ptr cloneObject() const override { - return std::static_pointer_cast(clone()); + return clone(); }; std::shared_ptr shared_from_this() diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index b42d703d090..4c1e563e1f2 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -44,13 +44,13 @@ ExternalDictionaries::ExternalDictionaries(Context & context, bool throw_on_erro getExternalDictionariesUpdateSettings(), getExternalDictionariesConfigSettings(), &Logger::get("ExternalDictionaries"), - "external dictionary", throw_on_error), + "external dictionary"), context(context) { - + init(throw_on_error); } -ExternalDictionaries::LoadablePtr ExternalDictionaries::create( +std::unique_ptr ExternalDictionaries::create( const std::string & name, const Configuration & config, const std::string & config_prefix) { return DictionaryFactory::instance().create(name, config, config_prefix, context); diff --git a/dbms/src/Interpreters/ExternalDictionaries.h b/dbms/src/Interpreters/ExternalDictionaries.h index 31ff6ad5341..9ed41f89d50 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.h +++ b/dbms/src/Interpreters/ExternalDictionaries.h @@ -30,8 +30,8 @@ public: protected: - LoadablePtr create(const std::string & name, const Configuration & config, - const std::string & config_prefix) override; + std::unique_ptr create(const std::string & name, const Configuration & config, + const std::string & config_prefix) override; using ExternalLoader::getObjectsMap; diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 42967f541c8..9b784639366 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -1,7 +1,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -46,10 +48,19 @@ void ExternalLoader::reloadPeriodically() ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & config, const ExternalLoaderUpdateSettings & update_settings, const ExternalLoaderConfigSettings & config_settings, - Logger * log, const std::string & loadable_object_name, bool throw_on_error) + Logger * log, const std::string & loadable_object_name) : config(config), update_settings(update_settings), config_settings(config_settings), log(log), object_name(loadable_object_name) { +} + +void ExternalLoader::init(bool throw_on_error) +{ + if (is_initialized) + return; + + is_initialized = true; + { /// During synchronous loading of external dictionaries at moment of query execution, /// we should not use per query memory limit. @@ -116,7 +127,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error) try { - auto loadable_ptr = failed_loadable_object.second.loadable->clone(); + auto loadable_ptr = failed_loadable_object.second.loadable->cloneObject(); if (const auto exception_ptr = loadable_ptr->getCreationException()) { /// recalculate next attempt time @@ -143,7 +154,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error) const auto dict_it = loadable_objects.find(name); dict_it->second.loadable.reset(); - dict_it->second.loadable = loadable_ptr; + dict_it->second.loadable = std::move(loadable_ptr); /// clear stored exception on success dict_it->second.exception = std::exception_ptr{}; @@ -200,13 +211,13 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error) if (current->isModified()) { /// create new version of loadable object - auto new_version = current->clone(); + auto new_version = current->cloneObject(); if (const auto exception_ptr = new_version->getCreationException()) std::rethrow_exception(exception_ptr); loadable_object.second.loadable.reset(); - loadable_object.second.loadable = new_version; + loadable_object.second.loadable = std::move(new_version); } } @@ -321,9 +332,9 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const const auto failed_dict_it = failed_loadable_objects.find(name); FailedLoadableInfo info{std::move(object_ptr), std::chrono::system_clock::now() + delay, 0}; if (failed_dict_it != std::end(failed_loadable_objects)) - failed_dict_it->second = info; + (*failed_dict_it).second = std::move(info); else - failed_loadable_objects.emplace(name, info); + failed_loadable_objects.emplace(name, std::move(info)); std::rethrow_exception(exception_ptr); } @@ -343,12 +354,12 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const /// add new loadable object or update an existing version if (object_it == std::end(loadable_objects)) - loadable_objects.emplace(name, LoadableInfo{object_ptr, config_path}); + loadable_objects.emplace(name, LoadableInfo{std::move(object_ptr), config_path}); else { if (object_it->second.loadable) object_it->second.loadable.reset(); - object_it->second.loadable = object_ptr; + object_it->second.loadable = std::move(object_ptr); /// erase stored exception on success object_it->second.exception = std::exception_ptr{}; @@ -414,4 +425,9 @@ ExternalLoader::LoadablePtr ExternalLoader::getLoadable(const std::string & name return it->second.loadable; } +std::tuple, const ExternalLoader::ObjectsMap &> ExternalLoader::getObjectsMap() const +{ + return std::make_tuple(std::unique_lock(map_mutex), std::cref(loadable_objects)); +} + } diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index d330bae2633..672c801424e 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -1,20 +1,16 @@ #pragma once -#include -#include -#include -#include #include #include -#include -#include #include #include #include #include -#include -#include +#include #include +#include +#include +#include namespace DB @@ -65,43 +61,9 @@ struct ExternalLoaderConfigSettings class ExternalLoader { public: - using Configuration = Poco::Util::AbstractConfiguration; - using ObjectsMap = std::unordered_map; - - /// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. - ExternalLoader(const Configuration & config, - const ExternalLoaderUpdateSettings & update_settings, - const ExternalLoaderConfigSettings & config_settings, - Logger * log, const std::string & loadable_object_name, bool throw_on_error); - ~ExternalLoader(); - - /// Forcibly reloads all loadable objects. - void reload(); - - /// Forcibly reloads specified loadable object. - void reload(const std::string & name); - - LoadablePtr getLoadable(const std::string & name) const; - -protected: - virtual LoadablePtr create(const std::string & name, const Configuration & config, - const std::string & config_prefix) = 0; - - /// Direct access to objects. - std::tuple, const ObjectsMap &> getObjectsMap() - { - return std::make_tuple(std::lock_guard(map_mutex), std::cref(loadable_objects)); - } + using LoadablePtr = std::shared_ptr; private: - - /// Protects only dictionaries map. - mutable std::mutex map_mutex; - - /// Protects all data, currently used to avoid races between updating thread and SYSTEM queries - mutable std::mutex all_mutex; - - using LoadablePtr = std::shared_ptr; struct LoadableInfo final { LoadablePtr loadable; @@ -116,16 +78,53 @@ private: UInt64 error_count; }; +public: + using Configuration = Poco::Util::AbstractConfiguration; + using ObjectsMap = std::unordered_map; + + /// Objects will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. + ExternalLoader(const Configuration & config, + const ExternalLoaderUpdateSettings & update_settings, + const ExternalLoaderConfigSettings & config_settings, + Logger * log, const std::string & loadable_object_name); + virtual ~ExternalLoader(); + + /// Forcibly reloads all loadable objects. + void reload(); + + /// Forcibly reloads specified loadable object. + void reload(const std::string & name); + + LoadablePtr getLoadable(const std::string & name) const; + +protected: + virtual std::unique_ptr create(const std::string & name, const Configuration & config, + const std::string & config_prefix) = 0; + + /// Direct access to objects. + std::tuple, const ObjectsMap &> getObjectsMap() const; + + /// Should be called in derived constructor (to avoid pure virtual call). + void init(bool throw_on_error); + +private: + + bool is_initialized = false; + + /// Protects only objects map. + mutable std::mutex map_mutex; + + /// Protects all data, currently used to avoid races between updating thread and SYSTEM queries + mutable std::mutex all_mutex; + /// name -> loadable. ObjectsMap loadable_objects; - /** Here are loadable objects, that has been never loaded successfully. - * They are also in 'loadable_objects', but with nullptr as 'loadable'. - */ + /// Here are loadable objects, that has been never loaded successfully. + /// They are also in 'loadable_objects', but with nullptr as 'loadable'. std::unordered_map failed_loadable_objects; - /** Both for dictionaries and failed_dictionaries. - */ + /// Both for loadable_objects and failed_loadable_objects. std::unordered_map update_times; pcg64 rnd_engine{randomSeed()}; @@ -143,7 +142,7 @@ private: std::unordered_map last_modification_times; - /// Check dictionaries definitions in config files and reload or/and add new ones if the definition is changed + /// Check objects definitions in config files and reload or/and add new ones if the definition is changed /// If loadable_name is not empty, load only loadable object with name loadable_name void reloadFromConfigFiles(bool throw_on_error, bool force_reload = false, const std::string & loadable_name = ""); void reloadFromConfigFile(const std::string & config_path, bool throw_on_error, bool force_reload, diff --git a/dbms/src/Interpreters/IExternalLoadable.h b/dbms/src/Interpreters/IExternalLoadable.h index 841aba86dfa..9e52b9e6864 100644 --- a/dbms/src/Interpreters/IExternalLoadable.h +++ b/dbms/src/Interpreters/IExternalLoadable.h @@ -1,14 +1,18 @@ +#pragma once #include #include +#include -namespace DB -{ namespace Poco::Util { -class AbstractConfiguration; + class AbstractConfiguration; } + +namespace DB +{ + /// Min and max lifetimes for a loadable object or it's entry struct ExternalLoadableLifetime final { @@ -31,7 +35,7 @@ public: virtual bool isModified() const = 0; - virtual std::shared_ptr cloneObject() const = 0; + virtual std::unique_ptr cloneObject() const = 0; virtual std::exception_ptr getCreationException() const = 0; }; From 24fc661f004e9416351152d715f59bf5c363b51a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Oct 2017 17:48:33 +0300 Subject: [PATCH 03/32] added boost::dll [#CLICKHOUSE-3305] --- .../libboost/boost_1_65_0/boost/dll/alias.hpp | 264 +++++++++ .../boost/dll/detail/aggressive_ptr_cast.hpp | 135 +++++ .../boost/dll/detail/ctor_dtor.hpp | 192 ++++++ .../dll/detail/demangling/demangle_symbol.hpp | 108 ++++ .../boost/dll/detail/demangling/itanium.hpp | 326 ++++++++++ .../demangling/mangled_storage_base.hpp | 120 ++++ .../boost/dll/detail/demangling/msvc.hpp | 439 ++++++++++++++ .../boost/dll/detail/elf_info.hpp | 285 +++++++++ .../boost/dll/detail/get_mem_fn_type.hpp | 40 ++ .../dll/detail/import_mangled_helpers.hpp | 290 +++++++++ .../boost/dll/detail/macho_info.hpp | 321 ++++++++++ .../boost_1_65_0/boost/dll/detail/pe_info.hpp | 430 ++++++++++++++ .../dll/detail/posix/path_from_handle.hpp | 169 ++++++ .../detail/posix/program_location_impl.hpp | 140 +++++ .../dll/detail/posix/shared_library_impl.hpp | 215 +++++++ .../boost/dll/detail/system_error.hpp | 56 ++ .../boost/dll/detail/type_info.hpp | 83 +++ .../dll/detail/windows/path_from_handle.hpp | 62 ++ .../detail/windows/shared_library_impl.hpp | 177 ++++++ .../boost/dll/detail/x_info_interface.hpp | 32 + .../boost_1_65_0/boost/dll/import.hpp | 277 +++++++++ .../boost_1_65_0/boost/dll/import_class.hpp | 558 ++++++++++++++++++ .../boost_1_65_0/boost/dll/import_mangled.hpp | 309 ++++++++++ .../boost_1_65_0/boost/dll/library_info.hpp | 181 ++++++ .../boost/dll/runtime_symbol_info.hpp | 237 ++++++++ .../boost_1_65_0/boost/dll/shared_library.hpp | 550 +++++++++++++++++ .../boost/dll/shared_library_load_mode.hpp | 249 ++++++++ .../boost_1_65_0/boost/dll/smart_library.hpp | 462 +++++++++++++++ dbms/src/Dictionaries/CatBoostModel.cpp | 42 ++ dbms/src/Dictionaries/CatBoostModel.h | 39 ++ 30 files changed, 6788 insertions(+) create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/alias.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/import.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp create mode 100644 contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp create mode 100644 dbms/src/Dictionaries/CatBoostModel.cpp create mode 100644 dbms/src/Dictionaries/CatBoostModel.h diff --git a/contrib/libboost/boost_1_65_0/boost/dll/alias.hpp b/contrib/libboost/boost_1_65_0/boost/dll/alias.hpp new file mode 100644 index 00000000000..b8ee45ffa33 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/alias.hpp @@ -0,0 +1,264 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_ALIAS_HPP +#define BOOST_DLL_ALIAS_HPP + +#include +#include +#include +#include +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +/// \file boost/dll/alias.hpp +/// \brief Includes alias methods and macro. You can include this header or +/// boost/dll/shared_library.hpp to reduce dependencies +/// in case you do not use the refcountable functions. + +namespace boost { namespace dll { + +#ifdef BOOST_DLL_DOXYGEN +/// Define this macro to explicitly specify translation unit in which alias must be instantiated. +/// See section 'Limitations' for more info. You may find usage examples in source codes of almost each tutorial. +/// Must be used in code, when \forcedmacrolink{BOOST_DLL_FORCE_NO_WEAK_EXPORTS} is defined +#define BOOST_DLL_FORCE_ALIAS_INSTANTIATION + +/// Define this macro to disable exporting weak symbols and start using the \forcedmacrolink{BOOST_DLL_FORCE_ALIAS_INSTANTIATION}. +/// This may be usefull for working around linker problems or to test your program for compatability with linkers that do not support export of weak symbols. +#define BOOST_DLL_FORCE_NO_WEAK_EXPORTS +#endif + +#if BOOST_COMP_MSVC || (BOOST_COMP_INTEL && BOOST_OS_WINDOWS) + +#define BOOST_DLL_SELECTANY __declspec(selectany) + +#define BOOST_DLL_SECTION(SectionName, Permissions) \ + BOOST_STATIC_ASSERT_MSG( \ + sizeof(#SectionName) < 10, \ + "Some platforms require section names to be at most 8 bytest" \ + ); \ + __pragma(section(#SectionName, Permissions)) __declspec(allocate(#SectionName)) \ + /**/ + +#else // #if BOOST_COMP_MSVC + + +#if BOOST_OS_WINDOWS || BOOST_OS_ANDROID || BOOST_COMP_IBM +// There are some problems with mixing `__dllexport__` and `weak` using MinGW +// See https://sourceware.org/bugzilla/show_bug.cgi?id=17480 +// +// Android had an issue with exporting weak symbols +// https://code.google.com/p/android/issues/detail?id=70206 +#define BOOST_DLL_SELECTANY +#else // #if BOOST_OS_WINDOWS +/*! +* \brief Macro that allows linker to select any occurrence of this symbol instead of +* failing with 'multiple definitions' error at linktime. +* +* This macro does not work on Android, IBM XL C/C++ and MinGW+Windows +* because of linker problems with exporting weak symbols +* (See https://code.google.com/p/android/issues/detail?id=70206, https://sourceware.org/bugzilla/show_bug.cgi?id=17480) +*/ +#define BOOST_DLL_SELECTANY __attribute__((weak)) +#endif // #if BOOST_OS_WINDOWS + +// TODO: improve section permissions using following info: +// http://stackoverflow.com/questions/6252812/what-does-the-aw-flag-in-the-section-attribute-mean + +#if !BOOST_OS_MACOS && !BOOST_OS_IOS +/*! +* \brief Macro that puts symbol to a specific section. On MacOS all the sections are put into "__DATA" segment. +* \param SectionName Name of the section. Must be a valid C identifier without quotes not longer than 8 bytes. +* \param Permissions Can be "read" or "write" (without quotes!). +*/ +#define BOOST_DLL_SECTION(SectionName, Permissions) \ + BOOST_STATIC_ASSERT_MSG( \ + sizeof(#SectionName) < 10, \ + "Some platforms require section names to be at most 8 bytest" \ + ); \ + __attribute__ ((section (#SectionName))) \ + /**/ +#else // #if !BOOST_OS_MACOS && !BOOST_OS_IOS + +#define BOOST_DLL_SECTION(SectionName, Permissions) \ + BOOST_STATIC_ASSERT_MSG( \ + sizeof(#SectionName) < 10, \ + "Some platforms require section names to be at most 8 bytest" \ + ); \ + __attribute__ ((section ( "__DATA," #SectionName))) \ + /**/ + +#endif // #if #if !BOOST_OS_MACOS && !BOOST_OS_IOS + +#endif // #if BOOST_COMP_MSVC + + +// Alias - is just a variable that pointers to original data +// +// A few attempts were made to avoid additional indirection: +// 1) +// // Does not work on Windows, work on Linux +// extern "C" BOOST_SYMBOL_EXPORT void AliasName() { +// reinterpret_cast(Function)(); +// } +// +// 2) +// // Does not work on Linux (changes permissions of .text section and produces incorrect DSO) +// extern "C" BOOST_SYMBOL_EXPORT void* __attribute__ ((section(".text#"))) +// func_ptr = *reinterpret_cast(&foo::bar); +// +// 3) // requires mangled name of `Function` +// // AliasName() __attribute__ ((weak, alias ("Function"))) +// +// // hard to use +// `#pragma comment(linker, "/alternatename:_pWeakValue=_pDefaultWeakValue")` + +/*! +* \brief Makes an alias name for exported function or variable. +* +* This macro is useful in cases of long mangled C++ names. For example some `void boost::foo(std::sting)` +* function name will change to something like `N5boostN3foosE` after mangling. +* Importing function by `N5boostN3foosE` name does not looks user friendly, especially assuming the fact +* that different compilers have different mangling schemes. AliasName is the name that won't be mangled +* and can be used as a portable import name. +* +* +* Can be used in any namespace, including global. FunctionOrVar must be fully qualified, +* so that address of it could be taken. Multiple different aliases for a single variable/function +* are allowed. +* +* Make sure that AliasNames are unique per library/executable. Functions or variables +* in global namespace must not have names same as AliasNames. +* +* Same AliasName in different translation units must point to the same FunctionOrVar. +* +* Puts all the aliases into the \b "boostdll" read only section of the binary. Equal to +* \forcedmacrolink{BOOST_DLL_ALIAS_SECTIONED}(FunctionOrVar, AliasName, boostdll). +* +* \param FunctionOrVar Function or variable for which an alias must be made. +* \param AliasName Name of the alias. Must be a valid C identifier. +* +* \b Example: +* \code +* namespace foo { +* void bar(std::string&); +* +* BOOST_DLL_ALIAS(foo::bar, foo_bar) +* } +* +* BOOST_DLL_ALIAS(foo::bar, foo_bar_another_alias_name) +* \endcode +* +* \b See: \forcedmacrolink{BOOST_DLL_ALIAS_SECTIONED} for making alias in a specific section. +*/ +#define BOOST_DLL_ALIAS(FunctionOrVar, AliasName) \ + BOOST_DLL_ALIAS_SECTIONED(FunctionOrVar, AliasName, boostdll) \ + /**/ + + +#if ((BOOST_COMP_GNUC && BOOST_OS_WINDOWS) || BOOST_OS_ANDROID || BOOST_COMP_IBM || defined(BOOST_DLL_FORCE_NO_WEAK_EXPORTS)) \ + && !defined(BOOST_DLL_FORCE_ALIAS_INSTANTIATION) && !defined(BOOST_DLL_DOXYGEN) + +#define BOOST_DLL_ALIAS_SECTIONED(FunctionOrVar, AliasName, SectionName) \ + namespace _autoaliases { \ + extern "C" BOOST_SYMBOL_EXPORT const void *AliasName; \ + } /* namespace _autoaliases */ \ + /**/ + +#define BOOST_DLL_AUTO_ALIAS(FunctionOrVar) \ + namespace _autoaliases { \ + extern "C" BOOST_SYMBOL_EXPORT const void *FunctionOrVar; \ + } /* namespace _autoaliases */ \ + /**/ +#else +// Note: we can not use `aggressive_ptr_cast` here, because in that case GCC applies +// different permissions to the section and it causes Segmentation fault. +// Note: we can not use `boost::addressof()` here, because in that case GCC +// may optimize away the FunctionOrVar instance and we'll get a pointer to unexisting symbol. +/*! +* \brief Same as \forcedmacrolink{BOOST_DLL_ALIAS} but puts alias name into the user specified section. +* +* \param FunctionOrVar Function or variable for which an alias must be made. +* \param AliasName Name of the alias. Must be a valid C identifier. +* \param SectionName Name of the section. Must be a valid C identifier without quotes not longer than 8 bytes. +* +* \b Example: +* \code +* namespace foo { +* void bar(std::string&); +* +* BOOST_DLL_ALIAS_SECTIONED(foo::bar, foo_bar, sect_1) // section "sect_1" now exports "foo_bar" +* } +* \endcode +* +*/ +#define BOOST_DLL_ALIAS_SECTIONED(FunctionOrVar, AliasName, SectionName) \ + namespace _autoaliases { \ + extern "C" BOOST_SYMBOL_EXPORT const void *AliasName; \ + BOOST_DLL_SECTION(SectionName, read) BOOST_DLL_SELECTANY \ + const void * AliasName = reinterpret_cast(reinterpret_cast( \ + &FunctionOrVar \ + )); \ + } /* namespace _autoaliases */ \ + /**/ + +/*! +* \brief Exports variable or function with unmangled alias name. +* +* This macro is useful in cases of long mangled C++ names. For example some `void boost::foo(std::sting)` +* function name will change to something like `N5boostN3foosE` after mangling. +* Importing function by `N5boostN3foosE` name does not looks user friendly, especially assuming the fact +* that different compilers have different mangling schemes.* +* +* Must be used in scope where FunctionOrVar declared. FunctionOrVar must be a valid C name, which means that +* it must not contain `::`. +* +* Functions or variables +* in global namespace must not have names same as FunctionOrVar. +* +* Puts all the aliases into the \b "boostdll" read only section of the binary. Almost same as +* \forcedmacrolink{BOOST_DLL_ALIAS}(FunctionOrVar, FunctionOrVar). +* +* \param FunctionOrVar Function or variable for which an unmangled alias must be made. +* +* \b Example: +* \code +* namespace foo { +* void bar(std::string&); +* BOOST_DLL_AUTO_ALIAS(bar) +* } +* +* \endcode +* +* \b See: \forcedmacrolink{BOOST_DLL_ALIAS} for making an alias with different names. +*/ + +#define BOOST_DLL_AUTO_ALIAS(FunctionOrVar) \ + namespace _autoaliases { \ + BOOST_DLL_SELECTANY const void * dummy_ ## FunctionOrVar \ + = reinterpret_cast(reinterpret_cast( \ + &FunctionOrVar \ + )); \ + extern "C" BOOST_SYMBOL_EXPORT const void *FunctionOrVar; \ + BOOST_DLL_SECTION(boostdll, read) BOOST_DLL_SELECTANY \ + const void * FunctionOrVar = dummy_ ## FunctionOrVar; \ + } /* namespace _autoaliases */ \ + /**/ + + +#endif + + +}} // namespace boost::dll + + +#endif // BOOST_DLL_ALIAS_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp new file mode 100644 index 00000000000..8d1ecc112bc --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp @@ -0,0 +1,135 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015-2017 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_AGGRESSIVE_PTR_CAST_HPP +#define BOOST_DLL_DETAIL_AGGRESSIVE_PTR_CAST_HPP + +#include +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include // std::memcpy + +#if defined(__GNUC__) && defined(__GNUC_MINOR__) && (__GNUC__ * 100 + __GNUC_MINOR__ > 301) +# pragma GCC system_header +#endif + +namespace boost { namespace dll { namespace detail { + +// GCC warns when reinterpret_cast between function pointer and object pointer occur. +// This method suppress the warnings and ensures that such casts are safe. +template +BOOST_FORCEINLINE typename boost::disable_if_c::value || boost::is_reference::value || boost::is_member_pointer::value, To>::type + aggressive_ptr_cast(From v) BOOST_NOEXCEPT +{ + BOOST_STATIC_ASSERT_MSG( + boost::is_pointer::value && boost::is_pointer::value, + "`agressive_ptr_cast` function must be used only for pointer casting." + ); + + BOOST_STATIC_ASSERT_MSG( + boost::is_void< typename boost::remove_pointer::type >::value + || boost::is_void< typename boost::remove_pointer::type >::value, + "`agressive_ptr_cast` function must be used only for casting to or from void pointers." + ); + + BOOST_STATIC_ASSERT_MSG( + sizeof(v) == sizeof(To), + "Pointer to function and pointer to object differ in size on your platform." + ); + + return reinterpret_cast(v); +} + +#ifdef BOOST_MSVC +# pragma warning(push) +# pragma warning(disable: 4172) // "returning address of local variable or temporary" but **v is not local! +#endif + +template +BOOST_FORCEINLINE typename boost::disable_if_c::value || boost::is_member_pointer::value, To>::type + aggressive_ptr_cast(From v) BOOST_NOEXCEPT +{ + BOOST_STATIC_ASSERT_MSG( + boost::is_pointer::value, + "`agressive_ptr_cast` function must be used only for pointer casting." + ); + + BOOST_STATIC_ASSERT_MSG( + boost::is_void< typename boost::remove_pointer::type >::value, + "`agressive_ptr_cast` function must be used only for casting to or from void pointers." + ); + + BOOST_STATIC_ASSERT_MSG( + sizeof(v) == sizeof(typename boost::remove_reference::type*), + "Pointer to function and pointer to object differ in size on your platform." + ); + return static_cast( + **reinterpret_cast::type**>( + v + ) + ); +} + +#ifdef BOOST_MSVC +# pragma warning(pop) +#endif + +template +BOOST_FORCEINLINE typename boost::disable_if_c::value || boost::is_member_pointer::value, To>::type + aggressive_ptr_cast(From v) BOOST_NOEXCEPT +{ + BOOST_STATIC_ASSERT_MSG( + boost::is_pointer::value, + "`agressive_ptr_cast` function must be used only for pointer casting." + ); + + BOOST_STATIC_ASSERT_MSG( + boost::is_void< typename boost::remove_pointer::type >::value, + "`agressive_ptr_cast` function must be used only for casting to or from void pointers." + ); + + To res = 0; + std::memcpy(&res, &v, sizeof(From)); + return res; +} + +template +BOOST_FORCEINLINE typename boost::disable_if_c::value || !boost::is_member_pointer::value, To>::type + aggressive_ptr_cast(From /* v */) BOOST_NOEXCEPT +{ + BOOST_STATIC_ASSERT_MSG( + boost::is_pointer::value, + "`agressive_ptr_cast` function must be used only for pointer casting." + ); + + BOOST_STATIC_ASSERT_MSG( + boost::is_void< typename boost::remove_pointer::type >::value, + "`agressive_ptr_cast` function must be used only for casting to or from void pointers." + ); + + BOOST_STATIC_ASSERT_MSG( + !sizeof(From), + "Casting from member pointers to void pointer is not implemnted in `agressive_ptr_cast`." + ); + + return 0; +} + +}}} // boost::dll::detail + +#endif // BOOST_DLL_DETAIL_AGGRESSIVE_PTR_CAST_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp new file mode 100644 index 00000000000..3da8395ff5a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp @@ -0,0 +1,192 @@ +// Copyright 2016 Klemens Morgenstern, Antony Polukhin +// +// 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) + +// For more information, see http://www.boost.org + +#ifndef BOOST_DLL_DETAIL_CTOR_DTOR_HPP_ +#define BOOST_DLL_DETAIL_CTOR_DTOR_HPP_ + +#include +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#include +#include + +#if defined(BOOST_MSVC) || defined(BOOST_MSVC_VER) +# include +#else +# include +#endif + + +namespace boost { namespace dll { namespace detail { + +/*! + * This class stores a constructor. + * + * In some compilers there are several constructors in code, which may include an allocating one. + * This can be used if the imported class shall be put on the heap, which is why the class provied both types. + */ +template +struct constructor; + +template +struct constructor { + typedef typename detail::get_mem_fn_type::mem_fn standard_t; + typedef Class*(*allocating_t)(Args...); + + + //! The standard, i.e. not allocating constructor. @warning May differ with the compiler. Use @ref constructor::call_standard instead. + standard_t standard; + //! The allocating constructor. @warning May differ with the compiler. Use @ref constructor::call_allocating instead. + allocating_t allocating; + + //! Call the standard contructor + void call_standard (Class * const ptr, Args...args){ (ptr->*standard)(static_cast(args)...); } + + //! Call the deleting destructor + Class * call_allocating(Args...args){ return allocating(static_cast(args)...); } + + + //! True if a allocating constructor could be loaded. + bool has_allocating() const { return allocating != nullptr; } + + //! True if a standard constructor could be loaded. + bool has_standard() const { return standard != nullptr; } + + //! False if neither the allocating nor the standard constructor is available. + bool is_empty() const { return (allocating == nullptr) && (standard == nullptr) ; } + + constructor() = delete; + constructor(const constructor &) = default; + + explicit constructor(standard_t standard, allocating_t allocating = nullptr) + : standard(standard) + , allocating(allocating) + {} +}; + + + +template +struct destructor { +#if !defined(_WIN32) + typedef void(*type)(Class* const); +#elif !defined(_WIN64) + typedef void(__thiscall * type)(Class* const); +#else + typedef void(__cdecl * type)(Class* const); +#endif + + typedef type standard_t; + typedef type deleting_t; + + //! The standard, i.e. not deleting destructor. @warning May differ with the compiler. Use @ref destructor::call_standard instead. + standard_t standard; + //! The deleting destructor. @warning May differ with the compiler. Use @ref destructor::call_deallocating instead. + deleting_t deleting; + + //! Call the standard contructor + void call_standard(Class * const ptr){ standard(ptr); } + + //! Call the deleting destructor + void call_deleting(Class * const ptr){ deleting(ptr); } + + //! True if a deleting destructor could be loaded. + bool has_deleting() const { return deleting != nullptr; } + + //! True if a standard destructor could be loaded. + bool has_standard() const { return standard != nullptr; } + + //! False if neither the deleting nor the standard destructor is available. + bool is_empty() const { return (deleting == nullptr) && (standard == nullptr) ; } + destructor() = delete; + + //! Copy destructor. + destructor(const destructor &) = default; + + //! Construct it from both the standard destructor and the allocating destructor + explicit destructor(const standard_t &standard, const deleting_t &deleting = nullptr) + : standard(standard) + , deleting(deleting) + {} +}; + +#if defined(BOOST_MSVC) || defined(BOOST_MSVC_VER) +template +constructor load_ctor(Lib & lib, const mangled_storage_impl::ctor_sym & ct) { + typedef typename constructor::standard_t standard_t; + standard_t ctor = lib.template get(ct); + return constructor(ctor); +} + +template +destructor load_dtor(Lib & lib, const mangled_storage_impl::dtor_sym & dt) { + typedef typename destructor::standard_t standard_t; + //@apolukhin That does NOT work this way with MSVC-14 x32 via memcpy. The x64 is different. + //standard_t dtor = &lib.template get< typename boost::remove_pointer::type >(dt); + void * buf = &lib.template get(dt); + standard_t dtor; + std::memcpy(&dtor, &buf, sizeof(dtor)); + return destructor(dtor); +} + +#else + +template +constructor load_ctor(Lib & lib, const mangled_storage_impl::ctor_sym & ct) { + typedef typename constructor::standard_t stand; + typedef typename constructor::allocating_t alloc; + + stand s = nullptr; + alloc a = nullptr; + + //see here for the abi http://mentorembedded.github.io/cxx-abi/abi.html#mangling-special-ctor-dtor + + if (!ct.C1.empty()) + { + //the only way this works on mingw/win. + //For some reason there is always an 0xA in the following poniter, which screws with the this pointer. + void *buf = &lib.template get(ct.C1); + std::memcpy(&s, &buf, sizeof(void*)); + } + if (!ct.C3.empty()) + { + void *buf = &lib.template get(ct.C3); + std::memcpy(&a, &buf, sizeof(void*)); + } + + return constructor(s,a); +} + +template +destructor load_dtor(Lib & lib, const mangled_storage_impl::dtor_sym & dt) { + typedef typename destructor::standard_t stand; + typedef typename destructor::deleting_t delet; + + stand s = nullptr; + delet d = nullptr; + + //see here for the abi http://mentorembedded.github.io/cxx-abi/abi.html#mangling-special-ctor-dtor + if (!dt.D1.empty()) { + s = &lib.template get< typename boost::remove_pointer::type >(dt.D1); + } + + if (!dt.D0.empty()) { + d = &lib.template get< typename boost::remove_pointer::type >(dt.D0); + } + + return destructor(s,d); + +} + +#endif + +}}} // namespace boost::dll::detail + +#endif /* BOOST_DLL_DETAIL_CTOR_DTOR_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp new file mode 100644 index 00000000000..30525f6b317 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp @@ -0,0 +1,108 @@ +// Copyright 2015 Klemens Morgenstern +// +// This file provides a demangling for function names, i.e. entry points of a dll. +// +// Distributed under the Boost Software License, Version 1.0. +// See http://www.boost.org/LICENSE_1_0.txt + +#ifndef BOOST_DLL_DEMANGLE_SYMBOL_HPP_ +#define BOOST_DLL_DEMANGLE_SYMBOL_HPP_ + +#include +#include +#include + +#if defined(BOOST_MSVC) || defined(BOOST_MSVC_FULL_VER) + +namespace boost +{ +namespace dll +{ +namespace detail +{ + +typedef void * (__cdecl * allocation_function)(std::size_t); +typedef void (__cdecl * free_function)(void *); + +extern "C" char* __unDName( char* outputString, + const char* name, + int maxStringLength, // Note, COMMA is leading following optional arguments + allocation_function pAlloc, + free_function pFree, + unsigned short disableFlags + ); + + +inline std::string demangle_symbol(const char *mangled_name) +{ + + allocation_function alloc = [](std::size_t size){return static_cast(new char[size]);}; + free_function free_f = [](void* p){delete [] static_cast(p);}; + + + + std::unique_ptr name { __unDName( + nullptr, + mangled_name, + 0, + alloc, + free_f, + static_cast(0))}; + + return std::string(name.get()); +} +inline std::string demangle_symbol(const std::string& mangled_name) +{ + return demangle_symbol(mangled_name.c_str()); +} + + +}}} +#else + +#include + +namespace boost +{ +namespace dll +{ +namespace detail +{ + +inline std::string demangle_symbol(const char *mangled_name) +{ + + if (*mangled_name == '_') + { + //because it start's with an underline _ + auto dm = boost::core::demangle(mangled_name); + if (!dm.empty()) + return dm; + else + return (mangled_name); + } + + //could not demangled + return ""; + + +} + +//for my personal convinience +inline std::string demangle_symbol(const std::string& mangled_name) +{ + return demangle_symbol(mangled_name.c_str()); +} + + +} +namespace experimental +{ +using ::boost::dll::detail::demangle_symbol; +} + +}} + +#endif + +#endif /* BOOST_DEMANGLE_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp new file mode 100644 index 00000000000..2701a0390b8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp @@ -0,0 +1,326 @@ +// Copyright 2016 Klemens Morgenstern +// +// 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) + +#ifndef BOOST_DLL_DETAIL_DEMANGLING_ITANIUM_HPP_ +#define BOOST_DLL_DETAIL_DEMANGLING_ITANIUM_HPP_ + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace boost { namespace dll { namespace detail { + + + +class mangled_storage_impl : public mangled_storage_base +{ + template + struct dummy {}; + + template + std::vector get_func_params(dummy) const + { + return {get_name()...}; + } + template + std::string get_return_type(dummy) const + { + return get_name(); + } +public: + using mangled_storage_base::mangled_storage_base; + struct ctor_sym + { + std::string C1; + std::string C2; + std::string C3; + + bool empty() const + { + return C1.empty() && C2.empty() && C3.empty(); + } + }; + + struct dtor_sym + { + std::string D0; + std::string D1; + std::string D2; + bool empty() const + { + return D0.empty() && D1.empty() && D2.empty(); + } + }; + + template + std::string get_variable(const std::string &name) const; + + template + std::string get_function(const std::string &name) const; + + template + std::string get_mem_fn(const std::string &name) const; + + template + ctor_sym get_constructor() const; + + template + dtor_sym get_destructor() const; + + template + std::string get_type_info() const; + + template + std::vector get_related() const; + +}; + + + +namespace parser +{ + + inline std::string const_rule_impl(true_type ) {return " const";} + inline std::string const_rule_impl(false_type) {return "";} + template + std::string const_rule() {using t = is_const::type>; return const_rule_impl(t());} + + inline std::string volatile_rule_impl(true_type ) {return " volatile";} + inline std::string volatile_rule_impl(false_type) {return "";} + template + std::string volatile_rule() {using t = is_volatile::type>; return volatile_rule_impl(t());} + + inline std::string reference_rule_impl(false_type, false_type) {return "";} + inline std::string reference_rule_impl(true_type, false_type) {return "&" ;} + inline std::string reference_rule_impl(false_type, true_type ) {return "&&";} + + + template + std::string reference_rule() {using t_l = is_lvalue_reference; using t_r = is_rvalue_reference; return reference_rule_impl(t_l(), t_r());} + + //it takes a string, because it may be overloaded. + template + std::string type_rule(const std::string & type_name) + { + using namespace std; + + return type_name + + const_rule() + + volatile_rule() + + reference_rule(); + } + + + template + std::string arg_list(const mangled_storage_impl & ms, Return (*)(Arg)) + { + using namespace std; + auto str = ms.get_name(); + return type_rule(str); + } + + template + std::string arg_list(const mangled_storage_impl & ms, Return (*)(First, Second, Args...)) + { + auto st = ms.get_name(); + + using next_type = Return (*)(Second, Args...); + return type_rule(st) + ", " + arg_list(ms, next_type()); + } + + template + std::string arg_list(const mangled_storage_impl &, Return (*)()) + { + return ""; + } +} + + + +template std::string mangled_storage_impl::get_variable(const std::string &name) const +{ + auto found = std::find_if(storage_.begin(), storage_.end(), + [&](const entry& e) {return e.demangled == name;}); + + if (found != storage_.end()) + return found->mangled; + else + return ""; +} + +template std::string mangled_storage_impl::get_function(const std::string &name) const +{ + using func_type = Func*; + + auto matcher = name + '(' + parser::arg_list(*this, func_type()) + ')'; + + auto found = std::find_if(storage_.begin(), storage_.end(), [&](const entry& e) {return e.demangled == matcher;}); + if (found != storage_.end()) + return found->mangled; + else + return ""; + +} + +template +std::string mangled_storage_impl::get_mem_fn(const std::string &name) const +{ + using namespace parser; + + using func_type = Func*; + + std::string cname = get_name(); + + auto matcher = cname + "::" + name + + '(' + parser::arg_list(*this, func_type()) + ')' + + const_rule() + volatile_rule(); + + auto found = std::find_if(storage_.begin(), storage_.end(), [&](const entry& e) {return e.demangled == matcher;}); + + if (found != storage_.end()) + return found->mangled; + else + return ""; + +} + + +template +auto mangled_storage_impl::get_constructor() const -> ctor_sym +{ + using namespace parser; + + using func_type = Signature*; + + std::string ctor_name; // = class_name + "::" + name; + std::string unscoped_cname; //the unscoped class-name + { + auto class_name = get_return_type(dummy()); + auto pos = class_name.rfind("::"); + if (pos == std::string::npos) + { + ctor_name = class_name+ "::" +class_name ; + unscoped_cname = class_name; + } + else + { + unscoped_cname = class_name.substr(pos+2) ; + ctor_name = class_name+ "::" + unscoped_cname; + } + } + + auto matcher = + ctor_name + '(' + parser::arg_list(*this, func_type()) + ')'; + + + std::vector findings; + std::copy_if(storage_.begin(), storage_.end(), + std::back_inserter(findings), [&](const entry& e) {return e.demangled == matcher;}); + + ctor_sym ct; + + for (auto & e : findings) + { + + if (e.mangled.find(unscoped_cname +"C1E") != std::string::npos) + ct.C1 = e.mangled; + else if (e.mangled.find(unscoped_cname +"C2E") != std::string::npos) + ct.C2 = e.mangled; + else if (e.mangled.find(unscoped_cname +"C3E") != std::string::npos) + ct.C3 = e.mangled; + } + return ct; +} + +template +auto mangled_storage_impl::get_destructor() const -> dtor_sym +{ + std::string dtor_name; // = class_name + "::" + name; + std::string unscoped_cname; //the unscoped class-name + { + auto class_name = get_name(); + auto pos = class_name.rfind("::"); + if (pos == std::string::npos) + { + dtor_name = class_name+ "::~" + class_name + "()"; + unscoped_cname = class_name; + } + else + { + unscoped_cname = class_name.substr(pos+2) ; + dtor_name = class_name+ "::~" + unscoped_cname + "()"; + } + } + + auto d0 = unscoped_cname + "D0Ev"; + auto d1 = unscoped_cname + "D1Ev"; + auto d2 = unscoped_cname + "D2Ev"; + + dtor_sym dt; + //this is so simple, i don#t need a predicate + for (auto & s : storage_) + { + //alright, name fits + if (s.demangled == dtor_name) + { + if (s.mangled.find(d0) != std::string::npos) + dt.D0 = s.mangled; + else if (s.mangled.find(d1) != std::string::npos) + dt.D1 = s.mangled; + else if (s.mangled.find(d2) != std::string::npos) + dt.D2 = s.mangled; + + } + } + return dt; + +} + +template +std::string mangled_storage_impl::get_type_info() const +{ + std::string id = "typeinfo for " + get_name(); + + + auto predicate = [&](const mangled_storage_base::entry & e) + { + return e.demangled == id; + }; + + auto found = std::find_if(storage_.begin(), storage_.end(), predicate); + + + if (found != storage_.end()) + return found->mangled; + else + return ""; +} + +template +std::vector mangled_storage_impl::get_related() const +{ + std::vector ret; + auto name = get_name(); + + for (auto & c : storage_) + { + if (c.demangled.find(name) != std::string::npos) + ret.push_back(c.demangled); + } + + return ret; +} + +}}} + + +#endif /* BOOST_DLL_DETAIL_DEMANGLING_ITANIUM_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp new file mode 100644 index 00000000000..b9cfd64fa26 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp @@ -0,0 +1,120 @@ +// Copyright 2016 Klemens Morgenstern +// +// 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) + +#ifndef BOOST_DLL_DETAIL_MANGLE_STORAGE_BASE_HPP_ +#define BOOST_DLL_DETAIL_MANGLE_STORAGE_BASE_HPP_ + +#include +#include +#include +#include +#include +#include +#include + +namespace boost { namespace dll { namespace detail { + +///stores the mangled names with the demangled name. +struct mangled_storage_base +{ + struct entry + { + std::string mangled; + std::string demangled; + entry() = default; + entry(const std::string & m, const std::string &d) : mangled(m), demangled(d) {} + entry(const entry&) = default; + entry(entry&&) = default; + entry &operator= (const entry&) = default; + entry &operator= (entry&&) = default; + }; +protected: + std::vector storage_; + ///if a unknown class is imported it can be overloaded by this type + std::map aliases_; +public: + void assign(const mangled_storage_base & storage) + { + aliases_ = storage.aliases_; + storage_ = storage.storage_; + } + void swap( mangled_storage_base & storage) + { + aliases_.swap(storage.aliases_); + storage_.swap(storage.storage_); + } + void clear() + { + storage_.clear(); + aliases_.clear(); + } + std::vector & get_storage() {return storage_;}; + template + std::string get_name() const + { + using boost::typeindex::ctti_type_index; + auto tx = ctti_type_index::type_id(); + auto val = (aliases_.count(tx) > 0) ? aliases_.at(tx) : tx.pretty_name(); + return val; + } + + mangled_storage_base() = default; + mangled_storage_base(mangled_storage_base&&) = default; + mangled_storage_base(const mangled_storage_base&) = default; + + mangled_storage_base(const std::vector & symbols) { add_symbols(symbols);} + + explicit mangled_storage_base(library_info & li) : mangled_storage_base(li.symbols()) {} + + explicit mangled_storage_base( + const boost::filesystem::path& library_path, + bool throw_if_not_native_format = true) + : mangled_storage_base(library_info(library_path, throw_if_not_native_format).symbols()) + { + + } + + void load(library_info & li) { storage_.clear(); add_symbols(li.symbols()); }; + void load(const boost::filesystem::path& library_path, + bool throw_if_not_native_format = true) + { + storage_.clear(); + add_symbols(library_info(library_path, throw_if_not_native_format).symbols()); + }; + + /*! Allows do add a class as alias, if the class imported is not known + * in this binary. + * @tparam Alias The Alias type + * @param The name to create the alias for. + * + * @note There can be multiple aliases, this is on purpose. + */ + template void add_alias(const std::string& name) + { + aliases_.emplace( + boost::typeindex::ctti_type_index::type_id(), + name + ); + } + void add_symbols(const std::vector & symbols) + { + for (auto & sym : symbols) + { + auto dm = demangle_symbol(sym); + if (!dm.empty()) + storage_.emplace_back(sym, dm); + else + storage_.emplace_back(sym, sym); + } + } + + +}; + + +}}} + +#endif /* BOOST_DLL_DETAIL_MANGLE_STORAGE_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp new file mode 100644 index 00000000000..c1abd6a185c --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp @@ -0,0 +1,439 @@ +// Copyright 2016 Klemens Morgenstern +// +// 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) + +#ifndef BOOST_DLL_DETAIL_DEMANGLING_MSVC_HPP_ +#define BOOST_DLL_DETAIL_DEMANGLING_MSVC_HPP_ + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace boost { namespace dll { namespace detail { + +class mangled_storage_impl : public mangled_storage_base +{ + template + struct dummy {}; + + template + std::vector get_func_params(dummy) const + { + return {get_name()...}; + } + template + std::string get_return_type(dummy) const + { + return get_name(); + } + //function to remove preceeding 'class ' or 'struct ' if the are given in this format. + + inline static void trim_typename(std::string & val); +public: + using ctor_sym = std::string; + using dtor_sym = std::string; + + using mangled_storage_base::mangled_storage_base; + + template + std::string get_variable(const std::string &name) const; + + template + std::string get_function(const std::string &name) const; + + template + std::string get_mem_fn(const std::string &name) const; + + template + ctor_sym get_constructor() const; + + template + dtor_sym get_destructor() const; + + template //overload, does not need to virtual. + std::string get_name() const + { + auto nm = mangled_storage_base::get_name(); + trim_typename(nm); + return nm; + } + + template + std::string get_vtable() const; + + template + std::vector get_related() const; + +}; + +void mangled_storage_impl::trim_typename(std::string & val) +{ + //remove preceeding class or struct, because you might want to use a struct as class, et vice versa + if (val.size() >= 6) + { + using namespace std; + static constexpr char class_ [7] = "class "; + static constexpr char struct_[8] = "struct "; + + if (equal(begin(class_), end(class_)-1, val.begin())) //aklright, starts with 'class ' + val.erase(0, 6); + else if (val.size() >= 7) + if (equal(begin(struct_), end(struct_)-1, val.begin())) + val.erase(0, 7); + } +} + + +namespace parser +{ + namespace x3 = spirit::x3; + + auto ptr_rule_impl(std::integral_constant) + { + return -((-x3::space) >> "__ptr32"); + } + auto ptr_rule_impl(std::integral_constant) + { + return -((-x3::space) >> "__ptr64"); + } + + auto ptr_rule() { return ptr_rule_impl(std::integral_constant());} + + auto const visibility = ("public:" | x3::lit("protected:") | "private:"); + auto const virtual_ = x3::space >> "virtual"; + auto const static_ = x3::space >> x3::lit("static") ; + + auto const_rule_impl(true_type ) {return x3::space >> "const";}; + auto const_rule_impl(false_type) {return x3::eps;}; + template + auto const_rule() {using t = is_const::type>; return const_rule_impl(t());} + + auto volatile_rule_impl(true_type ) {return x3::space >> "volatile";}; + auto volatile_rule_impl(false_type) {return x3::eps;}; + template + auto volatile_rule() {using t = is_volatile::type>; return volatile_rule_impl(t());} + + + auto inv_const_rule_impl(true_type ) {return "const" >> x3::space ;}; + auto inv_const_rule_impl(false_type) {return x3::eps;}; + template + auto inv_const_rule() {using t = is_const::type>; return inv_const_rule_impl(t());} + + auto inv_volatile_rule_impl(true_type ) {return "volatile" >> x3::space;}; + auto inv_volatile_rule_impl(false_type) {return x3::eps;}; + template + auto inv_volatile_rule() {using t = is_volatile::type>; return inv_volatile_rule_impl(t());} + + + auto reference_rule_impl(false_type, false_type) {return x3::eps;} + auto reference_rule_impl(true_type, false_type) {return x3::space >>"&" ;} + auto reference_rule_impl(false_type, true_type ) {return x3::space >>"&&" ;} + + + template + auto reference_rule() {using t_l = is_lvalue_reference; using t_r = is_rvalue_reference; return reference_rule_impl(t_l(), t_r());} + + auto const class_ = ("class" | x3::lit("struct")); + + //it takes a string, because it may be overloaded. + template + auto type_rule(const std::string & type_name) + { + using namespace std; + + return -(class_ >> x3::space)>> x3::string(type_name) >> + const_rule() >> + volatile_rule() >> + reference_rule() >> + ptr_rule(); + } + template<> + auto type_rule(const std::string &) { return x3::string("void"); }; + + auto const cdecl_ = "__cdecl" >> x3::space; + auto const stdcall = "__stdcall" >> x3::space; +#if defined(_WIN64)//seems to be necessary by msvc 14-x64 + auto const thiscall = "__cdecl" >> x3::space; +#else + auto const thiscall = "__thiscall" >> x3::space; +#endif + + template + auto arg_list(const mangled_storage_impl & ms, Return (*)(Arg)) + { + using namespace std; + + return type_rule(ms.get_name()); + } + + template + auto arg_list(const mangled_storage_impl & ms, Return (*)(First, Second, Args...)) + { + + using next_type = Return (*)(Second, Args...); + return type_rule(ms.get_name()) >> x3::char_(',') >> arg_list(ms, next_type()); + } + + template + auto arg_list(const mangled_storage_impl& /*ms*/, Return (*)()) + { + return x3::string("void"); + } +} + + +template std::string mangled_storage_impl::get_variable(const std::string &name) const +{ + using namespace std; + using namespace boost; + + namespace x3 = spirit::x3; + using namespace parser; + + auto type_name = get_name(); + + auto matcher = + -(visibility >> static_ >> x3::space) >> //it may be a static class-member + parser::type_rule(type_name) >> x3::space >> + name; + + auto predicate = [&](const mangled_storage_base::entry & e) + { + if (e.demangled == name)//maybe not mangled, + return true; + + auto itr = e.demangled.begin(); + auto end = e.demangled.end(); + auto res = x3::parse(itr, end, matcher); + return res && (itr == end); + }; + + auto found = std::find_if(storage_.begin(), storage_.end(), predicate); + + if (found != storage_.end()) + return found->mangled; + else + return ""; +} + +template std::string mangled_storage_impl::get_function(const std::string &name) const +{ + namespace x3 = spirit::x3; + using namespace parser; + using func_type = Func*; + using return_type = typename function_traits::result_type; + std::string return_type_name = get_name(); + + + auto matcher = + -(visibility >> static_ >> x3::space) >> //it may be a static class-member, which does however not have the static attribute. + parser::type_rule(return_type_name) >> x3::space >> + cdecl_ >> //cdecl declaration for methods. stdcall cannot be + name >> x3::lit('(') >> parser::arg_list(*this, func_type()) >> x3::lit(')') >> parser::ptr_rule(); + + + auto predicate = [&](const mangled_storage_base::entry & e) + { + if (e.demangled == name)//maybe not mangled, + return true; + + auto itr = e.demangled.begin(); + auto end = e.demangled.end(); + auto res = x3::parse(itr, end, matcher); + + return res && (itr == end); + }; + + auto found = std::find_if(storage_.begin(), storage_.end(), predicate); + + if (found != storage_.end()) + return found->mangled; + else + return ""; + +} + +template +std::string mangled_storage_impl::get_mem_fn(const std::string &name) const +{ + namespace x3 = spirit::x3; + using namespace parser; + using func_type = Func*; + using return_type = typename function_traits::result_type; + auto return_type_name = get_name(); + + + auto cname = get_name(); + + auto matcher = + visibility >> -virtual_ >> x3::space >> + parser::type_rule(return_type_name) >> x3::space >> + thiscall >> //cdecl declaration for methods. stdcall cannot be + cname >> "::" >> name >> + x3::lit('(') >> parser::arg_list(*this, func_type()) >> x3::lit(')') >> + inv_const_rule() >> inv_volatile_rule() >> parser::ptr_rule(); + + auto predicate = [&](const mangled_storage_base::entry & e) + { + auto itr = e.demangled.begin(); + auto end = e.demangled.end(); + auto res = x3::parse(itr, end, matcher); + + return res && (itr == end); + }; + + auto found = std::find_if(storage_.begin(), storage_.end(), predicate); + + if (found != storage_.end()) + return found->mangled; + else + return ""; +} + + +template +auto mangled_storage_impl::get_constructor() const -> ctor_sym +{ + namespace x3 = spirit::x3; + using namespace parser; + + using func_type = Signature*; + + + std::string ctor_name; // = class_name + "::" + name; + std::string unscoped_cname; //the unscoped class-name + { + auto class_name = get_return_type(dummy()); + auto pos = class_name.rfind("::"); + if (pos == std::string::npos) + { + ctor_name = class_name+ "::" + class_name ; + unscoped_cname = class_name; + } + else + { + unscoped_cname = class_name.substr(pos+2) ; + ctor_name = class_name+ "::" + unscoped_cname; + } + } + + auto matcher = + visibility >> x3::space >> + thiscall >> //cdecl declaration for methods. stdcall cannot be + ctor_name >> + x3::lit('(') >> parser::arg_list(*this, func_type()) >> x3::lit(')') >> parser::ptr_rule(); + + + auto predicate = [&](const mangled_storage_base::entry & e) + { + auto itr = e.demangled.begin(); + auto end = e.demangled.end(); + auto res = x3::parse(itr, end, matcher); + + return res && (itr == end); + }; + + auto f = std::find_if(storage_.begin(), storage_.end(), predicate); + + if (f != storage_.end()) + return f->mangled; + else + return ""; +} + +template +auto mangled_storage_impl::get_destructor() const -> dtor_sym +{ + namespace x3 = spirit::x3; + using namespace parser; + std::string dtor_name; // = class_name + "::" + name; + std::string unscoped_cname; //the unscoped class-name + { + auto class_name = get_name(); + auto pos = class_name.rfind("::"); + if (pos == std::string::npos) + { + dtor_name = class_name+ "::~" + class_name + "(void)"; + unscoped_cname = class_name; + } + else + { + unscoped_cname = class_name.substr(pos+2) ; + dtor_name = class_name+ "::~" + unscoped_cname + "(void)"; + } + } + + auto matcher = + visibility >> -virtual_ >> x3::space >> + thiscall >> //cdecl declaration for methods. stdcall cannot be + dtor_name >> parser::ptr_rule(); + + + auto predicate = [&](const mangled_storage_base::entry & e) + { + auto itr = e.demangled.begin(); + auto end = e.demangled.end(); + auto res = x3::parse(itr, end, matcher); + + return res && (itr == end); + }; + + auto found = std::find_if(storage_.begin(), storage_.end(), predicate); + + + if (found != storage_.end()) + return found->mangled; + else + return ""; +} + +template +std::string mangled_storage_impl::get_vtable() const +{ + std::string id = "const " + get_name() + "::`vftable'"; + + auto predicate = [&](const mangled_storage_base::entry & e) + { + return e.demangled == id; + }; + + auto found = std::find_if(storage_.begin(), storage_.end(), predicate); + + + if (found != storage_.end()) + return found->mangled; + else + return ""; +} + +template +std::vector mangled_storage_impl::get_related() const +{ + std::vector ret; + auto name = get_name(); + + for (auto & c : storage_) + { + if (c.demangled.find(name) != std::string::npos) + ret.push_back(c.demangled); + } + + return ret; +} + + +}}} + + + +#endif /* BOOST_DLL_DETAIL_DEMANGLING_MSVC_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp new file mode 100644 index 00000000000..325df9cb10f --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp @@ -0,0 +1,285 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_POSIX_ELF_INFO_HPP +#define BOOST_DLL_DETAIL_POSIX_ELF_INFO_HPP + +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#include +#include +#include + +namespace boost { namespace dll { namespace detail { + +template +struct Elf_Ehdr_template { + unsigned char e_ident[16]; /* Magic number and other info */ + boost::uint16_t e_type; /* Object file type */ + boost::uint16_t e_machine; /* Architecture */ + boost::uint32_t e_version; /* Object file version */ + AddressOffsetT e_entry; /* Entry point virtual address */ + AddressOffsetT e_phoff; /* Program header table file offset */ + AddressOffsetT e_shoff; /* Section header table file offset */ + boost::uint32_t e_flags; /* Processor-specific flags */ + boost::uint16_t e_ehsize; /* ELF header size in bytes */ + boost::uint16_t e_phentsize; /* Program header table entry size */ + boost::uint16_t e_phnum; /* Program header table entry count */ + boost::uint16_t e_shentsize; /* Section header table entry size */ + boost::uint16_t e_shnum; /* Section header table entry count */ + boost::uint16_t e_shstrndx; /* Section header string table index */ +}; + +typedef Elf_Ehdr_template Elf32_Ehdr_; +typedef Elf_Ehdr_template Elf64_Ehdr_; + +template +struct Elf_Shdr_template { + boost::uint32_t sh_name; /* Section name (string tbl index) */ + boost::uint32_t sh_type; /* Section type */ + AddressOffsetT sh_flags; /* Section flags */ + AddressOffsetT sh_addr; /* Section virtual addr at execution */ + AddressOffsetT sh_offset; /* Section file offset */ + AddressOffsetT sh_size; /* Section size in bytes */ + boost::uint32_t sh_link; /* Link to another section */ + boost::uint32_t sh_info; /* Additional section information */ + AddressOffsetT sh_addralign; /* Section alignment */ + AddressOffsetT sh_entsize; /* Entry size if section holds table */ +}; + +typedef Elf_Shdr_template Elf32_Shdr_; +typedef Elf_Shdr_template Elf64_Shdr_; + +template +struct Elf_Sym_template; + +template <> +struct Elf_Sym_template { + typedef boost::uint32_t AddressOffsetT; + + boost::uint32_t st_name; /* Symbol name (string tbl index) */ + AddressOffsetT st_value; /* Symbol value */ + AddressOffsetT st_size; /* Symbol size */ + unsigned char st_info; /* Symbol type and binding */ + unsigned char st_other; /* Symbol visibility */ + boost::uint16_t st_shndx; /* Section index */ +}; + +template <> +struct Elf_Sym_template { + typedef boost::uint64_t AddressOffsetT; + + boost::uint32_t st_name; /* Symbol name (string tbl index) */ + unsigned char st_info; /* Symbol type and binding */ + unsigned char st_other; /* Symbol visibility */ + boost::uint16_t st_shndx; /* Section index */ + AddressOffsetT st_value; /* Symbol value */ + AddressOffsetT st_size; /* Symbol size */ +}; + + +typedef Elf_Sym_template Elf32_Sym_; +typedef Elf_Sym_template Elf64_Sym_; + +template +class elf_info: public x_info_interface { + boost::filesystem::ifstream& f_; + + typedef boost::dll::detail::Elf_Ehdr_template header_t; + typedef boost::dll::detail::Elf_Shdr_template section_t; + typedef boost::dll::detail::Elf_Sym_template symbol_t; + + BOOST_STATIC_CONSTANT(boost::uint32_t, SHT_SYMTAB_ = 2); + BOOST_STATIC_CONSTANT(boost::uint32_t, SHT_STRTAB_ = 3); + + BOOST_STATIC_CONSTANT(unsigned char, STB_LOCAL_ = 0); /* Local symbol */ + BOOST_STATIC_CONSTANT(unsigned char, STB_GLOBAL_ = 1); /* Global symbol */ + BOOST_STATIC_CONSTANT(unsigned char, STB_WEAK_ = 2); /* Weak symbol */ + + /* Symbol visibility specification encoded in the st_other field. */ + BOOST_STATIC_CONSTANT(unsigned char, STV_DEFAULT_ = 0); /* Default symbol visibility rules */ + BOOST_STATIC_CONSTANT(unsigned char, STV_INTERNAL_ = 1); /* Processor specific hidden class */ + BOOST_STATIC_CONSTANT(unsigned char, STV_HIDDEN_ = 2); /* Sym unavailable in other modules */ + BOOST_STATIC_CONSTANT(unsigned char, STV_PROTECTED_ = 3); /* Not preemptible, not exported */ + +public: + static bool parsing_supported(boost::filesystem::ifstream& f) { + const unsigned char magic_bytes[5] = { + 0x7f, 'E', 'L', 'F', sizeof(boost::uint32_t) == sizeof(AddressOffsetT) ? 1 : 2 + }; + + unsigned char ch; + f.seekg(0); + for (std::size_t i = 0; i < sizeof(magic_bytes); ++i) { + f >> ch; + if (ch != magic_bytes[i]) { + return false; + } + } + + return true; + } + + explicit elf_info(boost::filesystem::ifstream& f) BOOST_NOEXCEPT + : f_(f) + {} + + std::vector sections() { + std::vector ret; + std::vector names; + sections_names_raw(names); + + const char* name_begin = &names[0]; + const char* const name_end = name_begin + names.size(); + ret.reserve(header().e_shnum); + do { + ret.push_back(name_begin); + name_begin += ret.back().size() + 1; + } while (name_begin != name_end); + + return ret; + } + +private: + template + inline void read_raw(T& value, std::size_t size = sizeof(T)) const { + f_.read(reinterpret_cast(&value), size); + } + + inline header_t header() { + header_t elf; + + f_.seekg(0); + read_raw(elf); + + return elf; + } + + void sections_names_raw(std::vector& sections) { + const header_t elf = header(); + + section_t section_names_section; + f_.seekg(elf.e_shoff + elf.e_shstrndx * sizeof(section_t)); + read_raw(section_names_section); + + sections.resize(static_cast(section_names_section.sh_size)); + f_.seekg(section_names_section.sh_offset); + read_raw(sections[0], static_cast(section_names_section.sh_size)); + } + + void symbols_text(std::vector& symbols, std::vector& text) { + const header_t elf = header(); + f_.seekg(elf.e_shoff); + + for (std::size_t i = 0; i < elf.e_shnum; ++i) { + section_t section; + read_raw(section); + + if (section.sh_type == SHT_SYMTAB_) { + symbols.resize(static_cast(section.sh_size / sizeof(symbol_t))); + + const boost::filesystem::ifstream::pos_type pos = f_.tellg(); + f_.seekg(section.sh_offset); + read_raw(symbols[0], static_cast(section.sh_size - (section.sh_size % sizeof(symbol_t))) ); + f_.seekg(pos); + } else if (section.sh_type == SHT_STRTAB_) { + text.resize(static_cast(section.sh_size)); + + const boost::filesystem::ifstream::pos_type pos = f_.tellg(); + f_.seekg(section.sh_offset); + read_raw(text[0], static_cast(section.sh_size)); + f_.seekg(pos); + } + } + } + + static bool is_visible(const symbol_t& sym) BOOST_NOEXCEPT { + // `(sym.st_info >> 4) != STB_LOCAL_ && !!sym.st_size` check also workarounds the + // GCC's issue https://sourceware.org/bugzilla/show_bug.cgi?id=13621 + return (sym.st_other & 0x03) == STV_DEFAULT_ && (sym.st_info >> 4) != STB_LOCAL_ && !!sym.st_size; + } + +public: + std::vector symbols() { + std::vector ret; + + std::vector symbols; + std::vector text; + symbols_text(symbols, text); + + ret.reserve(symbols.size()); + for (std::size_t i = 0; i < symbols.size(); ++i) { + if (is_visible(symbols[i])) { + ret.push_back(&text[0] + symbols[i].st_name); + if (ret.back().empty()) { + ret.pop_back(); // Do not show empty names + } + } + } + + return ret; + } + + std::vector symbols(const char* section_name) { + std::vector ret; + + std::size_t index = 0; + std::size_t ptrs_in_section_count = 0; + { + std::vector names; + sections_names_raw(names); + + const header_t elf = header(); + + for (; index < elf.e_shnum; ++index) { + section_t section; + f_.seekg(elf.e_shoff + index * sizeof(section_t)); + read_raw(section); + + if (!std::strcmp(&names[0] + section.sh_name, section_name)) { + if (!section.sh_entsize) { + section.sh_entsize = 1; + } + ptrs_in_section_count = static_cast(section.sh_size / section.sh_entsize); + break; + } + } + } + + std::vector symbols; + std::vector text; + symbols_text(symbols, text); + + if (ptrs_in_section_count < symbols.size()) { + ret.reserve(ptrs_in_section_count); + } else { + ret.reserve(symbols.size()); + } + + for (std::size_t i = 0; i < symbols.size(); ++i) { + if (symbols[i].st_shndx == index && is_visible(symbols[i])) { + ret.push_back(&text[0] + symbols[i].st_name); + if (ret.back().empty()) { + ret.pop_back(); // Do not show empty names + } + } + } + + return ret; + } +}; + +typedef elf_info elf_info32; +typedef elf_info elf_info64; + +}}} // namespace boost::dll::detail + +#endif // BOOST_DLL_DETAIL_POSIX_ELF_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp new file mode 100644 index 00000000000..b9dffdf4fe2 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp @@ -0,0 +1,40 @@ +// Copyright 2016 Klemens Morgenstern +// +// 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) + +// For more information, see http://www.boost.org + +#ifndef BOOST_DLL_DETAIL_GET_MEM_FN_TYPE_HPP_ +#define BOOST_DLL_DETAIL_GET_MEM_FN_TYPE_HPP_ + +namespace boost { namespace dll { namespace detail { + +template +struct get_mem_fn_type; + +template +struct get_mem_fn_type { + typedef Return (Class::*mem_fn)(Args...); +}; + +template +struct get_mem_fn_type { + typedef Return (Class::*mem_fn)(Args...) const ; +}; + +template +struct get_mem_fn_type { + typedef Return (Class::*mem_fn)(Args...) volatile; +}; + +template +struct get_mem_fn_type { + typedef Return (Class::*mem_fn)(Args...) const volatile ; +}; + +}}} // namespace boost::dll::detail + + +#endif /* BOOST_DLL_SMART_LIBRARY_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp new file mode 100644 index 00000000000..b1a22e70122 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp @@ -0,0 +1,290 @@ +// Copyright 2015-2016 Klemens D. Morgenstern +// +// 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) + +#ifndef BOOST_DLL_DETAIL_IMPORT_MANGLED_HELPERS_HPP_ +#define BOOST_DLL_DETAIL_IMPORT_MANGLED_HELPERS_HPP_ + + +#include +#include +#include +#include +#include + + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { namespace experimental { namespace detail { + +//the following could be done by fusion, though it's simple enough to just declare it here. +template +struct sequence {}; + +template struct push_front; +template +struct push_front> +{ + typedef sequence type; +}; + +template +struct unqalified_is_same : + boost::is_same< + typename boost::remove_cv::type, + typename boost::remove_cv::type + > +{ +}; + +/* ********************************** function sequence type traits ******************************/ + +//determine if it's a sequence of functions. +template struct is_function_seq; + +//type-trait for function overloads +template struct is_function_seq> + : boost::conditional< + boost::is_function::value, + is_function_seq>, + boost::false_type>::type +{}; + +template +struct is_function_seq> : boost::is_function +{ +}; + +template<> +struct is_function_seq> : boost::false_type +{ +}; + +/* ********************************* Function Tuple *************************** */ + +//a tuple of plain functions. +template +struct function_tuple; + +template +struct function_tuple + : function_tuple +{ + Return(*f_)(Args...); + + constexpr function_tuple(Return(* t)(Args...), T2* t2, Ts* ... ts) + : function_tuple(t2, ts...) + , f_(t) + {} + + Return operator()(Args...args) const { + return (*f_)(static_cast(args)...); + } + using function_tuple::operator(); +}; + +template +struct function_tuple { + Return(*f_)(Args...); + + constexpr function_tuple(Return(* t)(Args...)) + : f_(t) + {} + + Return operator()(Args...args) const { + return (*f_)(static_cast(args)...); + } +}; + + +/* ********************************** MemFn sequence type traits ******************************/ + +template +struct mem_fn_def +{ + typedef Class class_type; + typedef Func func_type; + typedef typename boost::dll::detail::get_mem_fn_type::mem_fn mem_fn; +}; + +template +struct make_mem_fn_seq; + +// B: is T1 another version of T0? +template +struct make_mem_fn_seq_getter; + +template +struct make_mem_fn_seq_getter +{ + typedef mem_fn_def type; +}; + +template +struct make_mem_fn_seq_getter +{ + typedef mem_fn_def type; +}; + +template +struct make_mem_fn_seq +{ + typedef mem_fn_def mem_fn; + typedef sequence type; +}; + +template +struct make_mem_fn_seq +{ + typedef sequence<> type; +}; + +template +struct make_mem_fn_seq +{ + /* Since we might have ovls, it might be : + * Class, void(int), void(int, int) //--> just us class for both + * Class, const Class, void(int)//--> ovl class. + * + */ + static_assert(boost::is_object::value, ""); + typedef typename make_mem_fn_seq_getter< + unqalified_is_same::value, T0, T1, T2>::type mem_fn_type; + + typedef typename boost::conditional< + unqalified_is_same::value, + make_mem_fn_seq, + make_mem_fn_seq> ::type next; + + typedef typename push_front::type type; +}; + + + + +/* Ok, this needs to be documented, so here's some pseudo-code: + * + * @code + * + * bool unqalified_is_same(lhs, rhs) + * { + * return remove_cv(lhs) == remove_cv(rhs); + * } + * + * mem_fn make_mem_fn_seq_getter(b, cl, T2, T3) + * { + * if (b) //b means, that T2 is another version of cl, i.e. qualified + * return get_mem_fn_type(T2, T3); + * else //means that T2 is a function. + * return get_mem_fn_type(cl, T2); + * } + * + * sequence make_mem_fn_seq(type cl, type T2, type T3, types...) + * { + * mem_fn = make_mem_fn_seq_getter( + * unqalified_is_same(cl, T2), cl, T2, T3); + * + * next = unqalified_is_same(cl, T2) ? + * make_mem_fn_seq(T2, types...) //because: T2 is another version of cl, hence i use this. T3 was already consumed. + * : + * make_mem_fn_seq(Class, T3, types...) //because: T2 was a function, hence it is consumed and class remains unchanged. + * ; + * return push_front(mem_fn, next) ; + * }; + * @endcode + */ + + + +template +struct is_mem_fn_seq_impl +{ + typedef typename boost::conditional< + boost::is_function::value || boost::dll::experimental::detail::unqalified_is_same::value, + typename is_mem_fn_seq_impl::type, + boost::false_type>::type type; +}; + +template +struct is_mem_fn_seq_impl +{ + typedef typename boost::conditional< + boost::is_function::value && boost::is_object::value, + boost::true_type, boost::false_type>::type type; +}; + +template +struct is_mem_fn_seq_impl +{ + typedef typename boost::conditional< + (boost::is_function::value || boost::dll::experimental::detail::unqalified_is_same::value) + && boost::is_function::value, + boost::true_type, boost::false_type>::type type; +}; + +template struct is_mem_fn_seq : boost::false_type {}; + +//If only two arguments are provided at all. +template +struct is_mem_fn_seq> : boost::conditional< + boost::is_object::value && boost::is_function::value, + boost::true_type, boost::false_type>::type +{ +}; + + +template +struct is_mem_fn_seq> : + boost::conditional< + boost::is_class::value && boost::is_function::value, + typename is_mem_fn_seq_impl::type, + boost::false_type>::type {}; + + +/* ********************************** mem fn sequence tuple ******************************/ + +/* A tuple of member functions + * Unlike for plain functions a sequence here might contain classes as well as functions. + */ +template +struct mem_fn_tuple; + +template +struct mem_fn_tuple, T2, Ts...> + : mem_fn_tuple +{ + typedef typename boost::dll::detail::get_mem_fn_type::mem_fn mem_fn; + mem_fn f_; + + constexpr mem_fn_tuple(mem_fn f, typename T2::mem_fn t2, typename Ts::mem_fn ... ts) + : mem_fn_tuple(t2, ts...) + , f_(f) + {} + + Return operator()(Class* const cl, Args...args) const { + return (cl->*f_)(static_cast(args)...); + } + using mem_fn_tuple::operator(); + +}; + +template +struct mem_fn_tuple> { + typedef typename boost::dll::detail::get_mem_fn_type::mem_fn mem_fn; + mem_fn f_; + + constexpr mem_fn_tuple(mem_fn f) + : f_(f) + {} + + Return operator()(Class * const cl, Args...args) const { + return (cl->*f_)(static_cast(args)...); + } +}; + +}}}} +#endif /* BOOST_DLL_DETAIL_IMPORT_MANGLED_HELPERS_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp new file mode 100644 index 00000000000..69119082383 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp @@ -0,0 +1,321 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_MACHO_INFO_HPP +#define BOOST_DLL_DETAIL_MACHO_INFO_HPP + +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#include +#include + +namespace boost { namespace dll { namespace detail { + +typedef int integer_t; +typedef int vm_prot_t; +typedef integer_t cpu_type_t; +typedef integer_t cpu_subtype_t; + +template +struct mach_header_template { + boost::uint32_t magic; + cpu_type_t cputype; + cpu_subtype_t cpusubtype; + boost::uint32_t filetype; + boost::uint32_t ncmds; + boost::uint32_t sizeofcmds; + boost::uint32_t flags[sizeof(AddressOffsetT) / sizeof(uint32_t)]; // Flags and reserved +}; + +typedef mach_header_template mach_header_32_; +typedef mach_header_template mach_header_64_; + +struct load_command_ { + boost::uint32_t cmd; /* type of command */ + boost::uint32_t cmdsize; +}; + +struct load_command_types { + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SEGMENT_ = 0x1); /* segment of this file to be mapped */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SYMTAB_ = 0x2); /* link-edit stab symbol table info */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SYMSEG_ = 0x3); /* link-edit gdb symbol table info (obsolete) */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_THREAD_ = 0x4); /* thread */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_UNIXTHREAD_ = 0x5); /* unix thread (includes a stack) */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOADFVMLIB_ = 0x6); /* load a specified fixed VM shared library */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_IDFVMLIB_ = 0x7); /* fixed VM shared library identification */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_IDENT_ = 0x8); /* object identification info (obsolete) */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_FVMFILE_ = 0x9); /* fixed VM file inclusion (internal use) */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_PREPAGE_ = 0xa); /* prepage command (internal use) */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_DYSYMTAB_ = 0xb); /* dynamic link-edit symbol table info */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOAD_DYLIB_ = 0xc); /* load a dynamically linked shared library */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ID_DYLIB_ = 0xd); /* dynamically linked shared lib ident */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOAD_DYLINKER_ = 0xe); /* load a dynamic linker */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ID_DYLINKER_ = 0xf); /* dynamic linker identification */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_PREBOUND_DYLIB_ = 0x10); /* modules prebound for a dynamically linked shared library */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ROUTINES_ = 0x11); /* image routines */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_FRAMEWORK_ = 0x12); /* sub framework */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_UMBRELLA_ = 0x13); /* sub umbrella */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_CLIENT_ = 0x14); /* sub client */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_LIBRARY_ = 0x15); /* sub library */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_TWOLEVEL_HINTS_ = 0x16); /* two-level namespace lookup hints */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_PREBIND_CKSUM_ = 0x17); /* prebind checksum */ +/* + * After MacOS X 10.1 when a new load command is added that is required to be + * understood by the dynamic linker for the image to execute properly the + * LC_REQ_DYLD bit will be or'ed into the load command constant. If the dynamic + * linker sees such a load command it it does not understand will issue a + * "unknown load command required for execution" error and refuse to use the + * image. Other load commands without this bit that are not understood will + * simply be ignored. + */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_REQ_DYLD_ = 0x80000000); + +/* + * load a dynamically linked shared library that is allowed to be missing + * (all symbols are weak imported). + */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOAD_WEAK_DYLIB_ = (0x18 | LC_REQ_DYLD_)); + + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SEGMENT_64_ = 0x19); /* 64-bit segment of this file to be mapped */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ROUTINES_64_ = 0x1a); /* 64-bit image routines */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_UUID_ = 0x1b); /* the uuid */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_RPATH_ = (0x1c | LC_REQ_DYLD_)); /* runpath additions */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_CODE_SIGNATURE_ = 0x1d); /* local of code signature */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SEGMENT_SPLIT_INFO_= 0x1e); /* local of info to split segments */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_REEXPORT_DYLIB_ = (0x1f | LC_REQ_DYLD_)); /* load and re-export dylib */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LAZY_LOAD_DYLIB_ = 0x20); /* delay load of dylib until first use */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ENCRYPTION_INFO_ = 0x21); /* encrypted segment information */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_DYLD_INFO_ = 0x22); /* compressed dyld information */ + BOOST_STATIC_CONSTANT(boost::uint32_t, LC_DYLD_INFO_ONLY_ = (0x22|LC_REQ_DYLD_)); /* compressed dyld information only */ +}; + +template +struct segment_command_template { + boost::uint32_t cmd; /* LC_SEGMENT_ */ + boost::uint32_t cmdsize; /* includes sizeof section structs */ + char segname[16]; /* segment name */ + AddressOffsetT vmaddr; /* memory address of this segment */ + AddressOffsetT vmsize; /* memory size of this segment */ + AddressOffsetT fileoff; /* file offset of this segment */ + AddressOffsetT filesize; /* amount to map from the file */ + vm_prot_t maxprot; /* maximum VM protection */ + vm_prot_t initprot; /* initial VM protection */ + boost::uint32_t nsects; /* number of sections in segment */ + boost::uint32_t flags; /* flags */ +}; + +typedef segment_command_template segment_command_32_; +typedef segment_command_template segment_command_64_; + +template +struct section_template { + char sectname[16]; /* name of this section */ + char segname[16]; /* segment this section goes in */ + AddressOffsetT addr; /* memory address of this section */ + AddressOffsetT size; /* size in bytes of this section */ + boost::uint32_t offset; /* file offset of this section */ + boost::uint32_t align; /* section alignment (power of 2) */ + boost::uint32_t reloff; /* file offset of relocation entries */ + boost::uint32_t nreloc; /* number of relocation entries */ + boost::uint32_t flags; /* flags (section type and attributes)*/ + boost::uint32_t reserved[1 + sizeof(AddressOffsetT) / sizeof(uint32_t)]; +}; + +typedef section_template section_32_; +typedef section_template section_64_; + +struct symtab_command_ { + boost::uint32_t cmd; /* LC_SYMTAB_ */ + boost::uint32_t cmdsize; /* sizeof(struct symtab_command) */ + boost::uint32_t symoff; /* symbol table offset */ + boost::uint32_t nsyms; /* number of symbol table entries */ + boost::uint32_t stroff; /* string table offset */ + boost::uint32_t strsize; /* string table size in bytes */ +}; + +template +struct nlist_template { + boost::uint32_t n_strx; + boost::uint8_t n_type; + boost::uint8_t n_sect; + boost::uint16_t n_desc; + AddressOffsetT n_value; +}; + +typedef nlist_template nlist_32_; +typedef nlist_template nlist_64_; + +template +class macho_info: public x_info_interface { + boost::filesystem::ifstream& f_; + + typedef boost::dll::detail::mach_header_template header_t; + typedef boost::dll::detail::load_command_ load_command_t; + typedef boost::dll::detail::segment_command_template segment_t; + typedef boost::dll::detail::section_template section_t; + typedef boost::dll::detail::symtab_command_ symbol_header_t; + typedef boost::dll::detail::nlist_template nlist_t; + + BOOST_STATIC_CONSTANT(boost::uint32_t, SEGMENT_CMD_NUMBER = (sizeof(AddressOffsetT) > 4 ? load_command_types::LC_SEGMENT_64_ : load_command_types::LC_SEGMENT_)); + +public: + static bool parsing_supported(boost::filesystem::ifstream& f) { + static const uint32_t magic_bytes = (sizeof(AddressOffsetT) <= sizeof(uint32_t) ? 0xfeedface : 0xfeedfacf); + + uint32_t magic; + f.seekg(0); + f.read(reinterpret_cast(&magic), sizeof(magic)); + return (magic_bytes == magic); + } + + explicit macho_info(boost::filesystem::ifstream& f) BOOST_NOEXCEPT + : f_(f) + {} + +private: + template + inline void read_raw(T& value, std::size_t size = sizeof(T)) const { + f_.read(reinterpret_cast(&value), size); + } + + template + void command_finder(uint32_t cmd_num, F callback_f) { + const header_t h = header(); + load_command_t command; + f_.seekg(sizeof(header_t)); + for (std::size_t i = 0; i < h.ncmds; ++i) { + const boost::filesystem::ifstream::pos_type pos = f_.tellg(); + read_raw(command); + if (command.cmd != cmd_num) { + f_.seekg(pos + static_cast(command.cmdsize)); + continue; + } + + f_.seekg(pos); + callback_f(*this); + f_.seekg(pos + static_cast(command.cmdsize)); + } + } + + struct section_names_gather { + std::vector& ret; + + void operator()(const macho_info& f) const { + segment_t segment; + f.read_raw(segment); + + section_t section; + ret.reserve(ret.size() + segment.nsects); + for (std::size_t j = 0; j < segment.nsects; ++j) { + f.read_raw(section); + // `segname` goes right after the `sectname`. + // Forcing `sectname` to end on '\0' + section.segname[0] = '\0'; + ret.push_back(section.sectname); + if (ret.back().empty()) { + ret.pop_back(); // Do not show empty names + } + } + } + }; + + struct symbol_names_gather { + std::vector& ret; + std::size_t section_index; + + void operator()(const macho_info& f) const { + symbol_header_t symbh; + f.read_raw(symbh); + ret.reserve(ret.size() + symbh.nsyms); + + nlist_t symbol; + std::string symbol_name; + for (std::size_t j = 0; j < symbh.nsyms; ++j) { + f.f_.seekg(symbh.symoff + j * sizeof(nlist_t)); + f.read_raw(symbol); + if (!symbol.n_strx) { + continue; // Symbol has no name + } + + if ((symbol.n_type & 0x0e) != 0xe || !symbol.n_sect) { + continue; // Symbol has no section + } + + if (section_index && section_index != symbol.n_sect) { + continue; // Not in the required section + } + + f.f_.seekg(symbh.stroff + symbol.n_strx); + getline(f.f_, symbol_name, '\0'); + if (symbol_name.empty()) { + continue; + } + + if (symbol_name[0] == '_') { + // Linker adds additional '_' symbol. Could not find official docs for that case. + ret.push_back(symbol_name.c_str() + 1); + } else { + ret.push_back(symbol_name); + } + } + } + }; + +public: + std::vector sections() { + std::vector ret; + section_names_gather f = { ret }; + command_finder(SEGMENT_CMD_NUMBER, f); + return ret; + } + +private: + inline header_t header() { + header_t h; + + f_.seekg(0); + read_raw(h); + + return h; + } + +public: + std::vector symbols() { + std::vector ret; + symbol_names_gather f = { ret, 0 }; + command_finder(load_command_types::LC_SYMTAB_, f); + return ret; + } + + std::vector symbols(const char* section_name) { + // Not very optimal solution + std::vector ret = sections(); + std::vector::iterator it = std::find(ret.begin(), ret.end(), section_name); + if (it == ret.end()) { + // No section with such name + ret.clear(); + return ret; + } + + // section indexes start from 1 + symbol_names_gather f = { ret, static_cast(1 + (it - ret.begin())) }; + ret.clear(); + command_finder(load_command_types::LC_SYMTAB_, f); + return ret; + } +}; + +typedef macho_info macho_info32; +typedef macho_info macho_info64; + +}}} // namespace boost::dll::detail + +#endif // BOOST_DLL_DETAIL_MACHO_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp new file mode 100644 index 00000000000..72668a075f7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp @@ -0,0 +1,430 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_WINDOWS_PE_INFO_HPP +#define BOOST_DLL_DETAIL_WINDOWS_PE_INFO_HPP + +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#include +#include +#include + +namespace boost { namespace dll { namespace detail { + +// reference: +// http://www.joachim-bauch.de/tutorials/loading-a-dll-from-memory/ +// http://msdn.microsoft.com/en-us/magazine/ms809762.aspx +// http://msdn.microsoft.com/en-us/magazine/cc301808.aspx +// + +// Basic Windows typedefs. We can not use header +// because that header must be included only on Windows platform +typedef unsigned char BYTE_; +typedef unsigned short WORD_; +typedef boost::uint32_t DWORD_; +typedef boost::int32_t LONG_; +typedef boost::uint32_t ULONG_; +typedef boost::int64_t LONGLONG_; +typedef boost::uint64_t ULONGLONG_; + +struct IMAGE_DOS_HEADER_ { // 32/64 independent header + boost::dll::detail::WORD_ e_magic; // Magic number + boost::dll::detail::WORD_ e_cblp; // Bytes on last page of file + boost::dll::detail::WORD_ e_cp; // Pages in file + boost::dll::detail::WORD_ e_crlc; // Relocations + boost::dll::detail::WORD_ e_cparhdr; // Size of header in paragraphs + boost::dll::detail::WORD_ e_minalloc; // Minimum extra paragraphs needed + boost::dll::detail::WORD_ e_maxalloc; // Maximum extra paragraphs needed + boost::dll::detail::WORD_ e_ss; // Initial (relative) SS value + boost::dll::detail::WORD_ e_sp; // Initial SP value + boost::dll::detail::WORD_ e_csum; // Checksum + boost::dll::detail::WORD_ e_ip; // Initial IP value + boost::dll::detail::WORD_ e_cs; // Initial (relative) CS value + boost::dll::detail::WORD_ e_lfarlc; // File address of relocation table + boost::dll::detail::WORD_ e_ovno; // Overlay number + boost::dll::detail::WORD_ e_res[4]; // Reserved words + boost::dll::detail::WORD_ e_oemid; // OEM identifier (for e_oeminfo) + boost::dll::detail::WORD_ e_oeminfo; // OEM information; e_oemid specific + boost::dll::detail::WORD_ e_res2[10]; // Reserved words + boost::dll::detail::LONG_ e_lfanew; // File address of new exe header +}; + +struct IMAGE_FILE_HEADER_ { // 32/64 independent header + boost::dll::detail::WORD_ Machine; + boost::dll::detail::WORD_ NumberOfSections; + boost::dll::detail::DWORD_ TimeDateStamp; + boost::dll::detail::DWORD_ PointerToSymbolTable; + boost::dll::detail::DWORD_ NumberOfSymbols; + boost::dll::detail::WORD_ SizeOfOptionalHeader; + boost::dll::detail::WORD_ Characteristics; +}; + +struct IMAGE_DATA_DIRECTORY_ { // 32/64 independent header + boost::dll::detail::DWORD_ VirtualAddress; + boost::dll::detail::DWORD_ Size; +}; + +struct IMAGE_EXPORT_DIRECTORY_ { // 32/64 independent header + boost::dll::detail::DWORD_ Characteristics; + boost::dll::detail::DWORD_ TimeDateStamp; + boost::dll::detail::WORD_ MajorVersion; + boost::dll::detail::WORD_ MinorVersion; + boost::dll::detail::DWORD_ Name; + boost::dll::detail::DWORD_ Base; + boost::dll::detail::DWORD_ NumberOfFunctions; + boost::dll::detail::DWORD_ NumberOfNames; + boost::dll::detail::DWORD_ AddressOfFunctions; + boost::dll::detail::DWORD_ AddressOfNames; + boost::dll::detail::DWORD_ AddressOfNameOrdinals; +}; + +struct IMAGE_SECTION_HEADER_ { // 32/64 independent header + static const std::size_t IMAGE_SIZEOF_SHORT_NAME_ = 8; + + boost::dll::detail::BYTE_ Name[IMAGE_SIZEOF_SHORT_NAME_]; + union { + boost::dll::detail::DWORD_ PhysicalAddress; + boost::dll::detail::DWORD_ VirtualSize; + } Misc; + boost::dll::detail::DWORD_ VirtualAddress; + boost::dll::detail::DWORD_ SizeOfRawData; + boost::dll::detail::DWORD_ PointerToRawData; + boost::dll::detail::DWORD_ PointerToRelocations; + boost::dll::detail::DWORD_ PointerToLinenumbers; + boost::dll::detail::WORD_ NumberOfRelocations; + boost::dll::detail::WORD_ NumberOfLinenumbers; + boost::dll::detail::DWORD_ Characteristics; +}; + + +template +struct IMAGE_OPTIONAL_HEADER_template { + static const std::size_t IMAGE_NUMBEROF_DIRECTORY_ENTRIES_ = 16; + + boost::dll::detail::WORD_ Magic; + boost::dll::detail::BYTE_ MajorLinkerVersion; + boost::dll::detail::BYTE_ MinorLinkerVersion; + boost::dll::detail::DWORD_ SizeOfCode; + boost::dll::detail::DWORD_ SizeOfInitializedData; + boost::dll::detail::DWORD_ SizeOfUninitializedData; + boost::dll::detail::DWORD_ AddressOfEntryPoint; + union { + boost::dll::detail::DWORD_ BaseOfCode; + unsigned char padding_[sizeof(AddressOffsetT) == 8 ? 4 : 8]; // in x64 version BaseOfData does not exist + } BaseOfCode_and_BaseOfData; + + AddressOffsetT ImageBase; + boost::dll::detail::DWORD_ SectionAlignment; + boost::dll::detail::DWORD_ FileAlignment; + boost::dll::detail::WORD_ MajorOperatingSystemVersion; + boost::dll::detail::WORD_ MinorOperatingSystemVersion; + boost::dll::detail::WORD_ MajorImageVersion; + boost::dll::detail::WORD_ MinorImageVersion; + boost::dll::detail::WORD_ MajorSubsystemVersion; + boost::dll::detail::WORD_ MinorSubsystemVersion; + boost::dll::detail::DWORD_ Win32VersionValue; + boost::dll::detail::DWORD_ SizeOfImage; + boost::dll::detail::DWORD_ SizeOfHeaders; + boost::dll::detail::DWORD_ CheckSum; + boost::dll::detail::WORD_ Subsystem; + boost::dll::detail::WORD_ DllCharacteristics; + AddressOffsetT SizeOfStackReserve; + AddressOffsetT SizeOfStackCommit; + AddressOffsetT SizeOfHeapReserve; + AddressOffsetT SizeOfHeapCommit; + boost::dll::detail::DWORD_ LoaderFlags; + boost::dll::detail::DWORD_ NumberOfRvaAndSizes; + IMAGE_DATA_DIRECTORY_ DataDirectory[IMAGE_NUMBEROF_DIRECTORY_ENTRIES_]; +}; + +typedef IMAGE_OPTIONAL_HEADER_template IMAGE_OPTIONAL_HEADER32_; +typedef IMAGE_OPTIONAL_HEADER_template IMAGE_OPTIONAL_HEADER64_; + +template +struct IMAGE_NT_HEADERS_template { + boost::dll::detail::DWORD_ Signature; + IMAGE_FILE_HEADER_ FileHeader; + IMAGE_OPTIONAL_HEADER_template OptionalHeader; +}; + +typedef IMAGE_NT_HEADERS_template IMAGE_NT_HEADERS32_; +typedef IMAGE_NT_HEADERS_template IMAGE_NT_HEADERS64_; + + +template +class pe_info: public x_info_interface { + boost::filesystem::ifstream& f_; + + typedef IMAGE_NT_HEADERS_template header_t; + typedef IMAGE_EXPORT_DIRECTORY_ exports_t; + typedef IMAGE_SECTION_HEADER_ section_t; + typedef IMAGE_DOS_HEADER_ dos_t; + + template + inline void read_raw(T& value, std::size_t size = sizeof(T)) const { + f_.read(reinterpret_cast(&value), size); + } + +public: + static bool parsing_supported(boost::filesystem::ifstream& f) { + dos_t dos; + f.seekg(0); + f.read(reinterpret_cast(&dos), sizeof(dos)); + + // 'MZ' and 'ZM' according to Wikipedia + if (dos.e_magic != 0x4D5A && dos.e_magic != 0x5A4D) { + return false; + } + + header_t h; + f.seekg(dos.e_lfanew); + f.read(reinterpret_cast(&h), sizeof(h)); + + return h.Signature == 0x00004550 // 'PE00' + && h.OptionalHeader.Magic == (sizeof(boost::uint32_t) == sizeof(AddressOffsetT) ? 0x10B : 0x20B); + } + + + explicit pe_info(boost::filesystem::ifstream& f) BOOST_NOEXCEPT + : f_(f) + {} + +private: + inline header_t header() { + header_t h; + + dos_t dos; + f_.seekg(0); + read_raw(dos); + + f_.seekg(dos.e_lfanew); + read_raw(h); + + return h; + } + + inline exports_t exports(const header_t& h) { + exports_t exports; + + static const unsigned int IMAGE_DIRECTORY_ENTRY_EXPORT_ = 0; + const std::size_t exp_virtual_address = h.OptionalHeader.DataDirectory[IMAGE_DIRECTORY_ENTRY_EXPORT_].VirtualAddress; + + const std::size_t real_offset = get_file_offset(exp_virtual_address, h); + BOOST_ASSERT(real_offset); + + f_.seekg(real_offset); + read_raw(exports); + + return exports; + } + + std::size_t get_file_offset(std::size_t virtual_address, const header_t& h) { + section_t image_section_header; + + { // f_.seekg to the beginning on section headers + dos_t dos; + f_.seekg(0); + read_raw(dos); + f_.seekg(dos.e_lfanew + sizeof(header_t)); + } + + for (std::size_t i = 0;i < h.FileHeader.NumberOfSections;++i) { + read_raw(image_section_header); + if (virtual_address >= image_section_header.VirtualAddress + && virtual_address < image_section_header.VirtualAddress + image_section_header.SizeOfRawData) + { + return image_section_header.PointerToRawData + virtual_address - image_section_header.VirtualAddress; + } + } + + return 0; + } + +public: + std::vector sections() { + std::vector ret; + + const header_t h = header(); + ret.reserve(h.FileHeader.NumberOfSections); + + // get names, e.g: .text .rdata .data .rsrc .reloc + section_t image_section_header; + char name_helper[section_t::IMAGE_SIZEOF_SHORT_NAME_ + 1]; + std::memset(name_helper, 0, sizeof(name_helper)); + for (std::size_t i = 0;i < h.FileHeader.NumberOfSections;++i) { + // There is no terminating null character if the string is exactly eight characters long + read_raw(image_section_header); + std::memcpy(name_helper, image_section_header.Name, section_t::IMAGE_SIZEOF_SHORT_NAME_); + + if (name_helper[0] != '/') { + ret.push_back(name_helper); + } else { + // For longer names, image_section_header.Name contains a slash (/) followed by ASCII representation of a decimal number. + // this number is an offset into the string table. + // TODO: fixme + ret.push_back(name_helper); + } + } + + return ret; + } + + std::vector symbols() { + std::vector ret; + + const header_t h = header(); + const exports_t exprt = exports(h); + const std::size_t exported_symbols = exprt.NumberOfNames; + const std::size_t fixed_names_addr = get_file_offset(exprt.AddressOfNames, h); + + ret.reserve(exported_symbols); + boost::dll::detail::DWORD_ name_offset; + std::string symbol_name; + for (std::size_t i = 0;i < exported_symbols;++i) { + f_.seekg(fixed_names_addr + i * sizeof(name_offset)); + read_raw(name_offset); + f_.seekg(get_file_offset(name_offset, h)); + getline(f_, symbol_name, '\0'); + ret.push_back(symbol_name); + } + + return ret; + } + + std::vector symbols(const char* section_name) { + std::vector ret; + + const header_t h = header(); + + std::size_t section_begin_addr = 0; + std::size_t section_end_addr = 0; + + { // getting address range for the section + section_t image_section_header; + char name_helper[section_t::IMAGE_SIZEOF_SHORT_NAME_ + 1]; + std::memset(name_helper, 0, sizeof(name_helper)); + for (std::size_t i = 0;i < h.FileHeader.NumberOfSections;++i) { + // There is no terminating null character if the string is exactly eight characters long + read_raw(image_section_header); + std::memcpy(name_helper, image_section_header.Name, section_t::IMAGE_SIZEOF_SHORT_NAME_); + if (!std::strcmp(section_name, name_helper)) { + section_begin_addr = image_section_header.PointerToRawData; + section_end_addr = section_begin_addr + image_section_header.SizeOfRawData; + } + } + + // returning empty result if section was not found + if(section_begin_addr == 0 || section_end_addr == 0) + return ret; + } + + const exports_t exprt = exports(h); + const std::size_t exported_symbols = exprt.NumberOfFunctions; + const std::size_t fixed_names_addr = get_file_offset(exprt.AddressOfNames, h); + const std::size_t fixed_ordinals_addr = get_file_offset(exprt.AddressOfNameOrdinals, h); + const std::size_t fixed_functions_addr = get_file_offset(exprt.AddressOfFunctions, h); + + ret.reserve(exported_symbols); + boost::dll::detail::DWORD_ ptr; + boost::dll::detail::WORD_ ordinal; + std::string symbol_name; + for (std::size_t i = 0;i < exported_symbols;++i) { + // getting ordinal + f_.seekg(fixed_ordinals_addr + i * sizeof(ordinal)); + read_raw(ordinal); + + // getting function addr + f_.seekg(fixed_functions_addr + ordinal * sizeof(ptr)); + read_raw(ptr); + ptr = static_cast( get_file_offset(ptr, h) ); + + if (ptr >= section_end_addr || ptr < section_begin_addr) { + continue; + } + + f_.seekg(fixed_names_addr + i * sizeof(ptr)); + read_raw(ptr); + f_.seekg(get_file_offset(ptr, h)); + getline(f_, symbol_name, '\0'); + ret.push_back(symbol_name); + } + + return ret; + } + + // a test method to get dependents modules, + // who my plugin imports (1st level only) + /* + e.g. for myself I get: + KERNEL32.dll + MSVCP110D.dll + boost_system-vc-mt-gd-1_56.dll + MSVCR110D.dll + */ + /* + std::vector depend_of(boost::system::error_code &ec) BOOST_NOEXCEPT { + std::vector ret; + + IMAGE_DOS_HEADER* image_dos_header = (IMAGE_DOS_HEADER*)native(); + if(!image_dos_header) { + // ERROR_BAD_EXE_FORMAT + ec = boost::system::error_code( + boost::system::errc::executable_format_error, + boost::system::generic_category() + ); + + return ret; + } + + IMAGE_OPTIONAL_HEADER* image_optional_header = (IMAGE_OPTIONAL_HEADER*)((boost::dll::detail::BYTE_*)native() + image_dos_header->e_lfanew + 24); + if(!image_optional_header) { + // ERROR_BAD_EXE_FORMAT + ec = boost::system::error_code( + boost::system::errc::executable_format_error, + boost::system::generic_category() + ); + + return ret; + } + + IMAGE_IMPORT_DESCRIPTOR* image_import_descriptor = (IMAGE_IMPORT_DESCRIPTOR*)((boost::dll::detail::BYTE_*)native() + image_optional_header->DataDirectory[IMAGE_DIRECTORY_ENTRY_IMPORT].VirtualAddress); + if(!image_import_descriptor) { + // ERROR_BAD_EXE_FORMAT + ec = boost::system::error_code( + boost::system::errc::executable_format_error, + boost::system::generic_category() + ); + + return ret; + } + + while(image_import_descriptor->FirstThunk) { + std::string module_name = reinterpret_cast((boost::dll::detail::BYTE_*)native() + image_import_descriptor->Name); + + if(module_name.size()) { + ret.push_back(module_name); + } + + image_import_descriptor++; + } + + return ret; + } +*/ +}; + +typedef pe_info pe_info32; +typedef pe_info pe_info64; + +}}} // namespace boost::dll::detail + +#endif // BOOST_DLL_DETAIL_WINDOWS_PE_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp new file mode 100644 index 00000000000..65d1a78b96e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp @@ -0,0 +1,169 @@ +// Copyright 2014-2015 Renato Tegon Forti, Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_POSIX_PATH_FROM_HANDLE_HPP +#define BOOST_DLL_DETAIL_POSIX_PATH_FROM_HANDLE_HPP + +#include +#include +#include +#include +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#if BOOST_OS_MACOS || BOOST_OS_IOS + +# include +# include +# include // for std::ptrdiff_t + +namespace boost { namespace dll { namespace detail { + inline void* strip_handle(void* handle) BOOST_NOEXCEPT { + return reinterpret_cast( + (reinterpret_cast(handle) >> 2) << 2 + ); + } + + inline boost::filesystem::path path_from_handle(void* handle, boost::system::error_code &ec) { + handle = strip_handle(handle); + + // Iterate through all images currently in memory + // https://developer.apple.com/library/mac/documentation/Darwin/Reference/ManPages/man3/dyld.3.html + const std::size_t count = _dyld_image_count(); // not thread safe: other thread my [un]load images + for (std::size_t i = 0; i <= count; ++i) { + // on last iteration `i` is equal to `count` which is out of range, so `_dyld_get_image_name` + // will return NULL. `dlopen(NULL, RTLD_LAZY)` call will open the current executable. + const char* image_name = _dyld_get_image_name(i); + + // dlopen/dlclose must not affect `_dyld_image_count()`, because libraries are already loaded and only the internal counter is affected + void* probe_handle = dlopen(image_name, RTLD_LAZY); + dlclose(probe_handle); + + // If the handle is the same as what was passed in (modulo mode bits), return this image name + if (handle == strip_handle(probe_handle)) { + boost::dll::detail::reset_dlerror(); + return image_name; + } + } + + boost::dll::detail::reset_dlerror(); + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + return boost::filesystem::path(); + } + +}}} // namespace boost::dll::detail + +#elif BOOST_OS_ANDROID + +#include + +namespace boost { namespace dll { namespace detail { + + struct soinfo { + // if defined(__work_around_b_24465209__), then an array of char[128] goes here. + // Unfortunately, __work_around_b_24465209__ is visible only during compilation of Android's linker + const void* phdr; + size_t phnum; + void* entry; + void* base; + // ... // Ignoring remaning parts of the structure + }; + + inline boost::filesystem::path path_from_handle(const void* handle, boost::system::error_code &ec) { + static const std::size_t work_around_b_24465209__offset = 128; + const struct soinfo* si = reinterpret_cast( + static_cast(handle) + work_around_b_24465209__offset + ); + boost::filesystem::path ret = boost::dll::symbol_location_ptr(si->base, ec); + + if (ec) { + ec.clear(); + si = static_cast(handle); + return boost::dll::symbol_location_ptr(si->base, ec); + } + + return ret; + } + +}}} // namespace boost::dll::detail + +#else // #if BOOST_OS_MACOS || BOOST_OS_IOS || BOOST_OS_ANDROID + +// for dlinfo +#include + +#if BOOST_OS_QNX +// QNX's copy of and reside in sys folder +# include +#else +# include // struct link_map +#endif + +namespace boost { namespace dll { namespace detail { + +#if BOOST_OS_QNX + // Android and QNX miss struct link_map. QNX misses ElfW macro, so avoiding it. + struct link_map { + void *l_addr; // Base address shared object is loaded at + char *l_name; // Absolute file name object was found in + // ... // Ignoring remaning parts of the structure + }; +#endif // #if BOOST_OS_QNX + + inline boost::filesystem::path path_from_handle(void* handle, boost::system::error_code &ec) { + // RTLD_DI_LINKMAP (RTLD_DI_ORIGIN returns only folder and is not suitable for this case) + // Obtain the Link_map for the handle that is specified. + // The p argument points to a Link_map pointer (Link_map + // **p). The actual storage for the Link_map structure is + // maintained by ld.so.1. + // + // Unfortunately we can not use `dlinfo(handle, RTLD_DI_LINKMAP, &link_map) < 0` + // because it is not supported on MacOS X 10.3, NetBSD 3.0, OpenBSD 3.8, AIX 5.1, + // HP-UX 11, IRIX 6.5, OSF/1 5.1, Cygwin, mingw, Interix 3.5, BeOS. + // Fortunately investigating the sources of open source projects brought the understanding, that + // `handle` is just a `struct link_map*` that contains full library name. + + const struct link_map* link_map = 0; +#if BOOST_OS_BSD_FREE + // FreeBSD has it's own logic http://code.metager.de/source/xref/freebsd/libexec/rtld-elf/rtld.c + // Fortunately it has the dlinfo call. + if (dlinfo(handle, RTLD_DI_LINKMAP, &link_map) < 0) { + link_map = 0; + } +#else + link_map = static_cast(handle); +#endif + if (!link_map) { + boost::dll::detail::reset_dlerror(); + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + return boost::filesystem::path(); + } + + if (!link_map->l_name || *link_map->l_name == '\0') { + return program_location_impl(ec); + } + + return boost::filesystem::path(link_map->l_name); + } + +}}} // namespace boost::dll::detail + +#endif // #if BOOST_OS_MACOS || BOOST_OS_IOS + +#endif // BOOST_DLL_DETAIL_POSIX_PATH_FROM_HANDLE_HPP + + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp new file mode 100644 index 00000000000..bca9b8caef9 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp @@ -0,0 +1,140 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_POSIX_PROGRAM_LOCATION_IMPL_HPP +#define BOOST_DLL_DETAIL_POSIX_PROGRAM_LOCATION_IMPL_HPP + +#include +#include +#include +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#if BOOST_OS_MACOS || BOOST_OS_IOS + +#include + +namespace boost { namespace dll { namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { + ec.clear(); + + char path[1024]; + uint32_t size = sizeof(path); + if (_NSGetExecutablePath(path, &size) == 0) + return boost::filesystem::path(path); + + char *p = new char[size]; + if (_NSGetExecutablePath(p, &size) != 0) { + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + } + + boost::filesystem::path ret(p); + delete[] p; + return ret; + } +}}} // namespace boost::dll::detail + +#elif BOOST_OS_SOLARIS + +#include +namespace boost { namespace dll { namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code& ec) { + ec.clear(); + + return boost::filesystem::path(getexecname()); + } +}}} // namespace boost::dll::detail + +#elif BOOST_OS_BSD_FREE + +#include +#include +#include + +namespace boost { namespace dll { namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code& ec) { + ec.clear(); + + int mib[4]; + mib[0] = CTL_KERN; + mib[1] = KERN_PROC; + mib[2] = KERN_PROC_PATHNAME; + mib[3] = -1; + char buf[10240]; + size_t cb = sizeof(buf); + sysctl(mib, 4, buf, &cb, NULL, 0); + + return boost::filesystem::path(buf); + } +}}} // namespace boost::dll::detail + + + +#elif BOOST_OS_BSD_NET + +#include +namespace boost { namespace dll { namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { + return boost::filesystem::read_symlink("/proc/curproc/exe", ec); + } +}}} // namespace boost::dll::detail + +#elif BOOST_OS_BSD_DRAGONFLY + +#include +namespace boost { namespace dll { namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { + return boost::filesystem::read_symlink("/proc/curproc/file", ec); + } +}}} // namespace boost::dll::detail + +#elif BOOST_OS_QNX + +#include +#include // for std::getline +namespace boost { namespace dll { namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { + ec.clear(); + + std::string s; + std::ifstream ifs("/proc/self/exefile"); + std::getline(ifs, s); + + if (ifs.fail() || s.empty()) { + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + } + + return boost::filesystem::path(s); + } +}}} // namespace boost::dll::detail + +#else // BOOST_OS_LINUX || BOOST_OS_UNIX || BOOST_OS_HPUX || BOOST_OS_ANDROID + +#include +namespace boost { namespace dll { namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { + // We can not use + // boost::dll::detail::path_from_handle(dlopen(NULL, RTLD_LAZY | RTLD_LOCAL), ignore); + // because such code returns empty path. + + return boost::filesystem::read_symlink("/proc/self/exe", ec); // Linux specific + } +}}} // namespace boost::dll::detail + +#endif + +#endif // BOOST_DLL_DETAIL_POSIX_PROGRAM_LOCATION_IMPL_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp new file mode 100644 index 00000000000..509e6c74873 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp @@ -0,0 +1,215 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015-2016 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_SHARED_LIBRARY_IMPL_HPP +#define BOOST_DLL_SHARED_LIBRARY_IMPL_HPP + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include // strncmp +#if !BOOST_OS_MACOS && !BOOST_OS_IOS && !BOOST_OS_QNX +# include +#elif BOOST_OS_QNX +// QNX's copy of and reside in sys folder +# include +#endif + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { namespace detail { + +class shared_library_impl { + + BOOST_MOVABLE_BUT_NOT_COPYABLE(shared_library_impl) + +public: + typedef void* native_handle_t; + + shared_library_impl() BOOST_NOEXCEPT + : handle_(NULL) + {} + + ~shared_library_impl() BOOST_NOEXCEPT { + unload(); + } + + shared_library_impl(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT + : handle_(sl.handle_) + { + sl.handle_ = NULL; + } + + shared_library_impl & operator=(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT { + swap(sl); + return *this; + } + + void load(boost::filesystem::path sl, load_mode::type mode, boost::system::error_code &ec) { + typedef int native_mode_t; + unload(); + + // Do not allow opening NULL paths. User must use program_location() instead + if (sl.empty()) { + boost::dll::detail::reset_dlerror(); + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + return; + } + + // Fixing modes + if (!(mode & load_mode::rtld_now)) { + mode |= load_mode::rtld_lazy; + } + + if (!(mode & load_mode::rtld_global)) { + mode |= load_mode::rtld_local; + } + +#if BOOST_OS_LINUX || BOOST_OS_ANDROID + if (!sl.has_parent_path() && !(mode & load_mode::search_system_folders)) { + sl = "." / sl; + } +#else + if (!sl.is_absolute() && !(mode & load_mode::search_system_folders)) { + boost::system::error_code current_path_ec; + boost::filesystem::path prog_loc = boost::filesystem::current_path(current_path_ec); + if (!current_path_ec) { + prog_loc /= sl; + sl.swap(prog_loc); + } + } +#endif + + mode &= ~load_mode::search_system_folders; + + // Trying to open with appended decorations + if (!!(mode & load_mode::append_decorations)) { + mode &= ~load_mode::append_decorations; + + boost::filesystem::path actual_path = ( + std::strncmp(sl.filename().string().c_str(), "lib", 3) + ? (sl.has_parent_path() ? sl.parent_path() / L"lib" : L"lib").native() + sl.filename().native() + : sl + ); + actual_path += suffix(); + + handle_ = dlopen(actual_path.c_str(), static_cast(mode)); + if (handle_) { + boost::dll::detail::reset_dlerror(); + return; + } + } + + // Opening by exactly specified path + handle_ = dlopen(sl.c_str(), static_cast(mode)); + if (handle_) { + boost::dll::detail::reset_dlerror(); + return; + } + + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + // Maybe user wanted to load the executable itself? Checking... + // We assume that usually user wants to load a dynamic library not the executable itself, that's why + // we try this only after traditional load fails. + boost::system::error_code prog_loc_err; + boost::filesystem::path loc = boost::dll::detail::program_location_impl(prog_loc_err); + if (!prog_loc_err && boost::filesystem::equivalent(sl, loc, prog_loc_err) && !prog_loc_err) { + // As is known the function dlopen() loads the dynamic library file + // named by the null-terminated string filename and returns an opaque + // "handle" for the dynamic library. If filename is NULL, then the + // returned handle is for the main program. + ec.clear(); + boost::dll::detail::reset_dlerror(); + handle_ = dlopen(NULL, static_cast(mode)); + if (!handle_) { + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + } + } + } + + bool is_loaded() const BOOST_NOEXCEPT { + return (handle_ != 0); + } + + void unload() BOOST_NOEXCEPT { + if (!is_loaded()) { + return; + } + + dlclose(handle_); + handle_ = 0; + } + + void swap(shared_library_impl& rhs) BOOST_NOEXCEPT { + boost::swap(handle_, rhs.handle_); + } + + boost::filesystem::path full_module_path(boost::system::error_code &ec) const { + return boost::dll::detail::path_from_handle(handle_, ec); + } + + static boost::filesystem::path suffix() { + // https://sourceforge.net/p/predef/wiki/OperatingSystems/ +#if BOOST_OS_MACOS || BOOST_OS_IOS + return ".dylib"; +#else + return ".so"; +#endif + } + + void* symbol_addr(const char* sb, boost::system::error_code &ec) const BOOST_NOEXCEPT { + // dlsym - obtain the address of a symbol from a dlopen object + void* const symbol = dlsym(handle_, sb); + if (symbol == NULL) { + ec = boost::system::error_code( + boost::system::errc::invalid_seek, + boost::system::generic_category() + ); + } + + // If handle does not refer to a valid object opened by dlopen(), + // or if the named symbol cannot be found within any of the objects + // associated with handle, dlsym() shall return NULL. + // More detailed diagnostic information shall be available through dlerror(). + + return symbol; + } + + native_handle_t native() const BOOST_NOEXCEPT { + return handle_; + } + +private: + native_handle_t handle_; +}; + +}}} // boost::dll::detail + +#endif // BOOST_DLL_SHARED_LIBRARY_IMPL_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp new file mode 100644 index 00000000000..c032633a0b8 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp @@ -0,0 +1,56 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_SYSTEM_ERROR_HPP +#define BOOST_DLL_SYSTEM_ERROR_HPP + +#include +#include +#include +#include +#include + +#if !BOOST_OS_WINDOWS +# include +#endif + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { namespace detail { + + inline void reset_dlerror() BOOST_NOEXCEPT { +#if !BOOST_OS_WINDOWS + const char* const error_txt = dlerror(); + (void)error_txt; +#endif + } + + inline void report_error(const boost::system::error_code& ec, const char* message) { +#if !BOOST_OS_WINDOWS + const char* const error_txt = dlerror(); + if (error_txt) { + boost::throw_exception( + boost::system::system_error( + ec, + message + std::string(" (dlerror system message: ") + error_txt + std::string(")") + ) + ); + } +#endif + + boost::throw_exception( + boost::system::system_error( + ec, message + ) + ); + } + +}}} // boost::dll::detail + +#endif // BOOST_DLL_SYSTEM_ERROR_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp new file mode 100644 index 00000000000..798ae20042e --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp @@ -0,0 +1,83 @@ +// Copyright 2016 Klemens Morgenstern, Antony Polukhin +// +// 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) + +// For more information, see http://www.boost.org + +#ifndef BOOST_DLL_DETAIL_TYPE_INFO_HPP_ +#define BOOST_DLL_DETAIL_TYPE_INFO_HPP_ + +#include +#include + +namespace boost { namespace dll { namespace detail { + +#if defined(BOOST_MSVC) || defined(BOOST_MSVC_VER) + +#if defined ( _WIN64 ) + +template +const std::type_info& load_type_info(Lib & lib, Storage & storage) +{ + struct RTTICompleteObjectLocator + { + boost::detail::winapi::DWORD_ signature; //always zero ? + boost::detail::winapi::DWORD_ offset; //offset of this vtable in the complete class + boost::detail::winapi::DWORD_ cdOffset; //constructor displacement offset + boost::detail::winapi::DWORD_ pTypeDescriptorOffset; //TypeDescriptor of the complete class + boost::detail::winapi::DWORD_ pClassDescriptorOffset; //describes inheritance hierarchy (ignored) + }; + + RTTICompleteObjectLocator** vtable_p = &lib.template get(storage.template get_vtable()); + + vtable_p--; + auto vtable = *vtable_p; + + auto nat = reinterpret_cast(lib.native()); + + nat += vtable->pTypeDescriptorOffset; + + return *reinterpret_cast(nat); + +} + +#else + +template +const std::type_info& load_type_info(Lib & lib, Storage & storage) +{ + struct RTTICompleteObjectLocator + { + boost::detail::winapi::DWORD_ signature; //always zero ? + boost::detail::winapi::DWORD_ offset; //offset of this vtable in the complete class + boost::detail::winapi::DWORD_ cdOffset; //constructor displacement offset + const std::type_info* pTypeDescriptor; //TypeDescriptor of the complete class + void* pClassDescriptor; //describes inheritance hierarchy (ignored) + }; + + RTTICompleteObjectLocator** vtable_p = &lib.template get(storage.template get_vtable()); + + vtable_p--; + auto vtable = *vtable_p; + return *vtable->pTypeDescriptor; + +} + +#endif //_WIN64 + +#else + +template +const std::type_info& load_type_info(Lib & lib, Storage & storage) +{ + return lib.template get(storage.template get_type_info()); + +} + +#endif + + +}}} +#endif /* BOOST_DLL_DETAIL_TYPE_INFO_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp new file mode 100644 index 00000000000..37eee86428c --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp @@ -0,0 +1,62 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_WINDOWS_PATH_FROM_HANDLE_HPP +#define BOOST_DLL_DETAIL_WINDOWS_PATH_FROM_HANDLE_HPP + +#include +#include +#include +#include +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { namespace detail { + + static inline boost::system::error_code last_error_code() BOOST_NOEXCEPT { + boost::detail::winapi::DWORD_ err = boost::detail::winapi::GetLastError(); + return boost::system::error_code( + err, + boost::system::system_category() + ); + } + + inline boost::filesystem::path path_from_handle(boost::detail::winapi::HMODULE_ handle, boost::system::error_code &ec) { + BOOST_STATIC_CONSTANT(boost::detail::winapi::DWORD_, ERROR_INSUFFICIENT_BUFFER_ = 0x7A); + BOOST_STATIC_CONSTANT(boost::detail::winapi::DWORD_, DEFAULT_PATH_SIZE_ = 260); + + // If `handle` parameter is NULL, GetModuleFileName retrieves the path of the + // executable file of the current process. + boost::detail::winapi::WCHAR_ path_hldr[DEFAULT_PATH_SIZE_]; + boost::detail::winapi::GetModuleFileNameW(handle, path_hldr, DEFAULT_PATH_SIZE_); + ec = last_error_code(); + if (!ec) { + return boost::filesystem::path(path_hldr); + } + + for (unsigned i = 2; i < 1025 && static_cast(ec.value()) == ERROR_INSUFFICIENT_BUFFER_; i *= 2) { + std::wstring p(DEFAULT_PATH_SIZE_ * i, L'\0'); + const std::size_t size = boost::detail::winapi::GetModuleFileNameW(handle, &p[0], DEFAULT_PATH_SIZE_ * i); + ec = last_error_code(); + + if (!ec) { + p.resize(size); + return boost::filesystem::path(p); + } + } + + // Error other than ERROR_INSUFFICIENT_BUFFER_ occurred or failed to allocate buffer big enough + return boost::filesystem::path(); + } + +}}} // namespace boost::dll::detail + +#endif // BOOST_DLL_DETAIL_WINDOWS_PATH_FROM_HANDLE_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp new file mode 100644 index 00000000000..48156505f47 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp @@ -0,0 +1,177 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015-2016 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_SHARED_LIBRARY_IMPL_HPP +#define BOOST_DLL_SHARED_LIBRARY_IMPL_HPP + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { namespace detail { + +class shared_library_impl { + BOOST_MOVABLE_BUT_NOT_COPYABLE(shared_library_impl) + +public: + typedef boost::detail::winapi::HMODULE_ native_handle_t; + + shared_library_impl() BOOST_NOEXCEPT + : handle_(NULL) + {} + + ~shared_library_impl() BOOST_NOEXCEPT { + unload(); + } + + shared_library_impl(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT + : handle_(sl.handle_) + { + sl.handle_ = NULL; + } + + shared_library_impl & operator=(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT { + swap(sl); + return *this; + } + + void load(boost::filesystem::path sl, load_mode::type mode, boost::system::error_code &ec) { + typedef boost::detail::winapi::DWORD_ native_mode_t; + unload(); + + if (!sl.is_absolute() && !(mode & load_mode::search_system_folders)) { + + boost::system::error_code current_path_ec; + boost::filesystem::path prog_loc = boost::filesystem::current_path(current_path_ec); + if (!current_path_ec) { + prog_loc /= sl; + sl.swap(prog_loc); + } + } + mode &= ~load_mode::search_system_folders; + + // Trying to open with appended decorations + if (!!(mode & load_mode::append_decorations)) { + mode &= ~load_mode::append_decorations; + + handle_ = boost::detail::winapi::LoadLibraryExW((sl.native() + L".dll").c_str(), 0, static_cast(mode)); + if (!handle_) { + // MinGW loves 'lib' prefix and puts it even on Windows platform + const boost::filesystem::path load_path = (sl.has_parent_path() ? sl.parent_path() / L"lib" : L"lib").native() + sl.filename().native() + L".dll"; + handle_ = boost::detail::winapi::LoadLibraryExW( + load_path.c_str(), + 0, + static_cast(mode) + ); + } + + if (handle_) { + return; + } + } + + // From MSDN: If the string specifies a module name without a path and the + // file name extension is omitted, the function appends the default library + // extension .dll to the module name. + // + // From experiments: Default library extension appended to the module name even if + // we have some path. So we do not check for path, only for extension. We can not be sure that + // such behavior remain across all platforms, so we add L"." by hand. + if (sl.has_extension()) { + handle_ = boost::detail::winapi::LoadLibraryExW(sl.c_str(), 0, static_cast(mode)); + } else { + handle_ = boost::detail::winapi::LoadLibraryExW((sl.native() + L".").c_str(), 0, static_cast(mode)); + } + + // LoadLibraryExW method is capable of self loading from program_location() path. No special actions + // must be taken to allow self loading. + + if (!handle_) { + ec = boost::dll::detail::last_error_code(); + } + } + + bool is_loaded() const BOOST_NOEXCEPT { + return (handle_ != 0); + } + + void unload() BOOST_NOEXCEPT { + if (handle_) { + boost::detail::winapi::FreeLibrary(handle_); + handle_ = 0; + } + } + + void swap(shared_library_impl& rhs) BOOST_NOEXCEPT { + boost::swap(handle_, rhs.handle_); + } + + boost::filesystem::path full_module_path(boost::system::error_code &ec) const { + return boost::dll::detail::path_from_handle(handle_, ec); + } + + static boost::filesystem::path suffix() { + return L".dll"; + } + + void* symbol_addr(const char* sb, boost::system::error_code &ec) const BOOST_NOEXCEPT { + if (is_resource()) { + // `GetProcAddress` could not be called for libraries loaded with + // `LOAD_LIBRARY_AS_DATAFILE`, `LOAD_LIBRARY_AS_DATAFILE_EXCLUSIVE` + // or `LOAD_LIBRARY_AS_IMAGE_RESOURCE`. + ec = boost::system::error_code( + boost::system::errc::operation_not_supported, + boost::system::generic_category() + ); + + return NULL; + } + + // Judging by the documentation of GetProcAddress + // there is no version for UNICODE on desktop/server Windows, because + // names of functions are stored in narrow characters. + void* const symbol = boost::dll::detail::aggressive_ptr_cast( + boost::detail::winapi::get_proc_address(handle_, sb) + ); + if (symbol == NULL) { + ec = boost::dll::detail::last_error_code(); + } + + return symbol; + } + + native_handle_t native() const BOOST_NOEXCEPT { + return handle_; + } + +private: + bool is_resource() const BOOST_NOEXCEPT { + return false; /*!!( + reinterpret_cast(handle_) & static_cast(3) + );*/ + } + + native_handle_t handle_; +}; + +}}} // boost::dll::detail + +#endif // BOOST_DLL_SHARED_LIBRARY_IMPL_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp new file mode 100644 index 00000000000..e53214e1cd6 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp @@ -0,0 +1,32 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_DETAIL_X_INFO_INTERFACE_HPP +#define BOOST_DLL_DETAIL_X_INFO_INTERFACE_HPP + +#include +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +#include +#include + +namespace boost { namespace dll { namespace detail { + +class x_info_interface { +public: + virtual std::vector sections() = 0; + virtual std::vector symbols() = 0; + virtual std::vector symbols(const char* section_name) = 0; + + virtual ~x_info_interface() BOOST_NOEXCEPT {} +}; + +}}} // namespace boost::dll::detail + +#endif // BOOST_DLL_DETAIL_X_INFO_INTERFACE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/import.hpp b/contrib/libboost/boost_1_65_0/boost/dll/import.hpp new file mode 100644 index 00000000000..dcf3086a181 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/import.hpp @@ -0,0 +1,277 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015-2016 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_IMPORT_HPP +#define BOOST_DLL_IMPORT_HPP + +#include +#include +#include +#include +#include +#include +#include + +#if defined(BOOST_NO_CXX11_TRAILING_RESULT_TYPES) || defined(BOOST_NO_CXX11_DECLTYPE) || defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) || defined(BOOST_NO_CXX11_RVALUE_REFERENCES) +# include +#endif + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +/// \file boost/dll/import.hpp +/// \brief Contains all the boost::dll::import* reference counting +/// functions that hold a shared pointer to the instance of +/// boost::dll::shared_library. + +namespace boost { namespace dll { + + +namespace detail { + + template + class library_function { + // Copying of `boost::dll::shared_library` is very expensive, so we use a `shared_ptr` to make it faster. + boost::shared_ptr f_; + + public: + inline library_function(const boost::shared_ptr& lib, T* func_ptr) BOOST_NOEXCEPT + : f_(lib, func_ptr) + {} + +#if defined(BOOST_NO_CXX11_TRAILING_RESULT_TYPES) || defined(BOOST_NO_CXX11_DECLTYPE) || defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) || defined(BOOST_NO_CXX11_RVALUE_REFERENCES) + operator T*() const BOOST_NOEXCEPT { + return f_.get(); + } +#else + + // Compilation error at this point means that imported function + // was called with unmatching parameters. + // + // Example: + // auto f = dll::import("function", "lib.so"); + // f("Hello"); // error: invalid conversion from 'const char*' to 'int' + // f(1, 2); // error: too many arguments to function + // f(); // error: too few arguments to function + template + inline auto operator()(Args&&... args) const + -> decltype( (*f_)(static_cast(args)...) ) + { + return (*f_)(static_cast(args)...); + } +#endif + }; + + template + struct import_type; + + template + struct import_type >::type> { + typedef boost::dll::detail::library_function base_type; + +#if defined(BOOST_NO_CXX11_TRAILING_RESULT_TYPES) || defined(BOOST_NO_CXX11_DECLTYPE) || defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) || defined(BOOST_NO_CXX11_RVALUE_REFERENCES) + typedef boost::function type; +#else + typedef boost::dll::detail::library_function type; +#endif + }; + + template + struct import_type >::type> { + typedef boost::shared_ptr base_type; + typedef boost::shared_ptr type; + }; +} // namespace detail + + +#ifndef BOOST_DLL_DOXYGEN +# define BOOST_DLL_IMPORT_RESULT_TYPE inline typename boost::dll::detail::import_type::type +#endif + + +/*! +* Returns callable object or boost::shared_ptr that holds the symbol imported +* from the loaded library. Returned value refcounts usage +* of the loaded shared library, so that it won't get unload until all copies of return value +* are not destroyed. +* +* This call will succeed if call to \forcedlink{shared_library}`::has(const char* )` +* function with the same symbol name returned `true`. +* +* For importing symbols by \b alias names use \forcedlink{import_alias} method. +* +* \b Examples: +* +* \code +* boost::function f = import("test_lib.so", "integer_func_name"); +* +* auto f_cpp11 = import("test_lib.so", "integer_func_name"); +* \endcode +* +* \code +* boost::shared_ptr i = import("test_lib.so", "integer_name"); +* \endcode +* +* \b Template \b parameter \b T: Type of the symbol that we are going to import. Must be explicitly specified. +* +* \param lib Path to shared library or shared library to load function from. +* \param name Null-terminated C or C++ mangled name of the function to import. Can handle std::string, char*, const char*. +* \param mode An mode that will be used on library load. +* +* \return callable object if T is a function type, or boost::shared_ptr if T is an object type. +* +* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. +* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. +*/ +template +BOOST_DLL_IMPORT_RESULT_TYPE import(const boost::filesystem::path& lib, const char* name, + load_mode::type mode = load_mode::default_mode) +{ + typedef typename boost::dll::detail::import_type::base_type type; + + boost::shared_ptr p = boost::make_shared(lib, mode); + return type(p, boost::addressof(p->get(name))); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import(const boost::filesystem::path& lib, const std::string& name, + load_mode::type mode = load_mode::default_mode) +{ + return import(lib, name.c_str(), mode); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import(const shared_library& lib, const char* name) { + typedef typename boost::dll::detail::import_type::base_type type; + + boost::shared_ptr p = boost::make_shared(lib); + return type(p, boost::addressof(p->get(name))); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import(const shared_library& lib, const std::string& name) { + return import(lib, name.c_str()); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import(BOOST_RV_REF(shared_library) lib, const char* name) { + typedef typename boost::dll::detail::import_type::base_type type; + + boost::shared_ptr p = boost::make_shared( + boost::move(lib) + ); + return type(p, boost::addressof(p->get(name))); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import(BOOST_RV_REF(shared_library) lib, const std::string& name) { + return import(boost::move(lib), name.c_str()); +} + + + + +/*! +* Returns callable object or boost::shared_ptr that holds the symbol imported +* from the loaded library. Returned value refcounts usage +* of the loaded shared library, so that it won't get unload until all copies of return value +* are not destroyed. +* +* This call will succeed if call to \forcedlink{shared_library}`::has(const char* )` +* function with the same symbol name returned `true`. +* +* For importing symbols by \b non \b alias names use \forcedlink{import} method. +* +* \b Examples: +* +* \code +* boost::function f = import_alias("test_lib.so", "integer_func_alias_name"); +* +* auto f_cpp11 = import_alias("test_lib.so", "integer_func_alias_name"); +* \endcode +* +* \code +* boost::shared_ptr i = import_alias("test_lib.so", "integer_alias_name"); +* \endcode +* +* \code +* \endcode +* +* \b Template \b parameter \b T: Type of the symbol alias that we are going to import. Must be explicitly specified. +* +* \param lib Path to shared library or shared library to load function from. +* \param name Null-terminated C or C++ mangled name of the function or variable to import. Can handle std::string, char*, const char*. +* \param mode An mode that will be used on library load. +* +* \return callable object if T is a function type, or boost::shared_ptr if T is an object type. +* +* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. +* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. +*/ +template +BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const boost::filesystem::path& lib, const char* name, + load_mode::type mode = load_mode::default_mode) +{ + typedef typename boost::dll::detail::import_type::base_type type; + + boost::shared_ptr p = boost::make_shared(lib, mode); + return type(p, p->get(name)); +} + +//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const boost::filesystem::path& lib, const std::string& name, + load_mode::type mode = load_mode::default_mode) +{ + return import_alias(lib, name.c_str(), mode); +} + +//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const shared_library& lib, const char* name) { + typedef typename boost::dll::detail::import_type::base_type type; + + boost::shared_ptr p = boost::make_shared(lib); + return type(p, p->get(name)); +} + +//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const shared_library& lib, const std::string& name) { + return import_alias(lib, name.c_str()); +} + +//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import_alias(BOOST_RV_REF(shared_library) lib, const char* name) { + typedef typename boost::dll::detail::import_type::base_type type; + + boost::shared_ptr p = boost::make_shared( + boost::move(lib) + ); + return type(p, p->get(name)); +} + +//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_IMPORT_RESULT_TYPE import_alias(BOOST_RV_REF(shared_library) lib, const std::string& name) { + return import_alias(boost::move(lib), name.c_str()); +} + +#undef BOOST_DLL_IMPORT_RESULT_TYPE + + +}} // boost::dll + +#endif // BOOST_DLL_IMPORT_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp b/contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp new file mode 100644 index 00000000000..866fc9c0459 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp @@ -0,0 +1,558 @@ +// Copyright 2015-2016 Klemens D. Morgenstern +// +// 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) + +#ifndef BOOST_DLL_IMPORT_CLASS_HPP_ +#define BOOST_DLL_IMPORT_CLASS_HPP_ + +#include +#include +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { namespace experimental { + +namespace detail +{ + +template +struct deleter +{ + destructor dtor; + bool use_deleting; + + deleter(const destructor & dtor, bool use_deleting = false) : + dtor(dtor), use_deleting(use_deleting) {} + + void operator()(T*t) + { + if (use_deleting) + dtor.call_deleting(t); + else + { + dtor.call_standard(t); + //the thing is actually an array, so delete[] + auto p = reinterpret_cast(t); + delete [] p; + } + } +}; + +template +struct mem_fn_call_proxy; + +template +struct mem_fn_call_proxy> +{ + typedef boost::dll::experimental::detail::mangled_library_mem_fn mem_fn_t; + Class* t; + mem_fn_t & mem_fn; + + mem_fn_call_proxy(mem_fn_call_proxy&&) = default; + mem_fn_call_proxy(const mem_fn_call_proxy & ) = delete; + mem_fn_call_proxy(Class * t, mem_fn_t & mem_fn) + : t(t), mem_fn(mem_fn) {} + + template + auto operator()(Args&&...args) const + { + return mem_fn(t, std::forward(args)...); + } + +}; + +template +struct mem_fn_call_proxy +{ + T* t; + const std::string &name; + smart_library &_lib; + + mem_fn_call_proxy(mem_fn_call_proxy&&) = default; + mem_fn_call_proxy(const mem_fn_call_proxy&) = delete; + mem_fn_call_proxy(T *t, const std::string &name, smart_library & _lib) + : t(t), name(name), _lib(_lib) {}; + + Return operator()(Args...args) const + { + auto f = _lib.get_mem_fn(name); + return (t->*f)(static_cast(args)...); + } +}; + +} + +template +class imported_class; + +template imported_class +import_class(const smart_library& lib, Args...args); +template imported_class +import_class(const smart_library& lib, const std::string & alias_name, Args...args); +template imported_class +import_class(const smart_library& lib, std::size_t size, Args...args); +template imported_class +import_class(const smart_library& lib, std::size_t size, + const std::string & alias_name, Args...args); + + +/*! This class represents an imported class. + * + * \note It must be constructed via \ref boost::dll::import_class(const smart_library& lib, std::size_t, Args...) + * + * \tparam The type or type-alias of the imported class. + */ +template +class imported_class +{ + smart_library _lib; + std::unique_ptr> _data; + bool _is_allocating; + std::size_t _size; + const std::type_info& _ti; + + template + inline std::unique_ptr> make_data(const smart_library& lib, Args ... args); + template + inline std::unique_ptr> make_data(const smart_library& lib, std::size_t size, Args...args); + + template + imported_class(detail::sequence *, const smart_library& lib, Args...args); + + template + imported_class(detail::sequence *, const smart_library& lib, std::size_t size, Args...args); + + template + imported_class(detail::sequence *, smart_library&& lib, Args...args); + + template + imported_class(detail::sequence *, smart_library&& lib, std::size_t size, Args...args); +public: + //alias to construct with explicit parameter list + template + static imported_class make(smart_library&& lib, Args...args) + { + typedef detail::sequence *seq; + return imported_class(seq(), boost::move(lib), static_cast(args)...); + } + + template + static imported_class make(smart_library&& lib, std::size_t size, Args...args) + { + typedef detail::sequence *seq; + return imported_class(seq(), boost::move(lib), size, static_cast(args)...); + } + template + static imported_class make(const smart_library& lib, Args...args) + { + typedef detail::sequence *seq; + return imported_class(seq(), lib, static_cast(args)...); + } + + template + static imported_class make(const smart_library& lib, std::size_t size, Args...args) + { + typedef detail::sequence *seq; + return imported_class(seq(), lib, size, static_cast(args)...); + } + + typedef imported_class base_t; + ///Returns a pointer to the underlying class + T* get() {return _data.get();} + imported_class() = delete; + + imported_class(imported_class&) = delete; + imported_class(imported_class&&) = default; /// ().empty();} + ///Check if the imported class is move-assignable + bool is_move_assignable() {return !_lib.symbol_storage().template get_mem_fn ("operator=").empty();} + ///Check if the imported class is copy-constructible + bool is_copy_constructible() {return !_lib.symbol_storage().template get_constructor().empty();} + ///Check if the imported class is copy-assignable + bool is_copy_assignable() {return !_lib.symbol_storage().template get_mem_fn("operator=").empty();} + + imported_class copy() const; /// move(); /// & lhs) const; + ///Invoke the move assignment. \attention Undefined behaviour if the imported object is not move assignable. + void move_assign( imported_class & lhs); + + ///Check if the class is loaded. + explicit operator bool() const {return _data;} + + ///Get a const reference to the std::type_info. + const std::type_info& get_type_info() {return _ti;}; + + /*! Call a member function. This returns a proxy to the function. + * The proxy mechanic mechanic is necessary, so the signaute can be passed. + * + * \b Example + * + * \code + * im_class.call("function_name")("MyString"); + * \endcode + */ + template + const detail::mem_fn_call_proxy call(const std::string& name) + { + return detail::mem_fn_call_proxy(_data.get(), name, _lib); + } + /*! Call a qualified member function, i.e. const and or volatile. + * + * \b Example + * + * \code + * im_class.call("function_name")("MyString"); + * \endcode + */ + template>> + const detail::mem_fn_call_proxy call(const std::string& name) + { + return detail::mem_fn_call_proxy(_data.get(), name, _lib); + } + ///Overload of ->* for an imported method. + template + const detail::mem_fn_call_proxy> + operator->*(detail::mangled_library_mem_fn& mn) + { + return detail::mem_fn_call_proxy>(_data.get(), mn); + } + + ///Import a method of the class. + template + typename boost::dll::experimental::detail::mangled_import_type>::type + import(const std::string & name) + { + return boost::dll::experimental::import_mangled(_lib, name); + } +}; + + + +//helper function, uses the allocating +template +template +inline std::unique_ptr> imported_class::make_data(const smart_library& lib, Args ... args) +{ + constructor ctor = lib.get_constructor(); + destructor dtor = lib.get_destructor (); + + if (!ctor.has_allocating() || !dtor.has_deleting()) + { + boost::system::error_code ec; + + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + // report_error() calls dlsym, do not use it here! + boost::throw_exception( + boost::system::system_error( + ec, "boost::dll::detail::make_data() failed: no allocating ctor or dtor was found" + ) + ); + } + + return std::unique_ptr> ( + ctor.call_allocating(static_cast(args)...), + detail::deleter(dtor, false /* not deleting dtor*/)); +} + +//helper function, using the standard +template +template +inline std::unique_ptr> imported_class::make_data(const smart_library& lib, std::size_t size, Args...args) +{ + constructor ctor = lib.get_constructor(); + destructor dtor = lib.get_destructor (); + + if (!ctor.has_standard() || !dtor.has_standard()) + { + boost::system::error_code ec; + + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + // report_error() calls dlsym, do not use it here! + boost::throw_exception( + boost::system::system_error( + ec, "boost::dll::detail::make_data() failed: no regular ctor or dtor was found" + ) + ); + } + + T *data = reinterpret_cast(new char[size]); + + ctor.call_standard(data, static_cast(args)...); + + return std::unique_ptr> ( + reinterpret_cast(data), + detail::deleter(dtor, false /* not deleting dtor*/)); + +} + + +template +template +imported_class::imported_class(detail::sequence *, const smart_library & lib, Args...args) + : _lib(lib), + _data(make_data(lib, static_cast(args)...)), + _is_allocating(false), + _size(0), + _ti(lib.get_type_info()) +{ + +} + +template +template +imported_class::imported_class(detail::sequence *, const smart_library & lib, std::size_t size, Args...args) + : _lib(lib), + _data(make_data(lib, size, static_cast(args)...)), + _is_allocating(true), + _size(size), + _ti(lib.get_type_info()) +{ + +} + +template +template +imported_class::imported_class(detail::sequence *, smart_library && lib, Args...args) + : _lib(boost::move(lib)), + _data(make_data(lib, static_cast(args)...)), + _is_allocating(false), + _size(0), + _ti(lib.get_type_info()) +{ + +} + +template +template +imported_class::imported_class(detail::sequence *, smart_library && lib, std::size_t size, Args...args) + : _lib(boost::move(lib)), + _data(make_data(lib, size, static_cast(args)...)), + _is_allocating(true), + _size(size), + _ti(lib.get_type_info()) +{ + +} + +template +inline imported_class boost::dll::experimental::imported_class::copy() const +{ + if (this->_is_allocating) + return imported_class::template make(_lib, *_data); + else + return imported_class::template make(_lib, _size, *_data); +} + +template +inline imported_class boost::dll::experimental::imported_class::move() +{ + if (this->_is_allocating) + return imported_class::template make(_lib, *_data); + else + return imported_class::template make(_lib, _size, *_data); +} + +template +inline void boost::dll::experimental::imported_class::copy_assign(const imported_class& lhs) const +{ + this->call("operator=")(*lhs._data); +} + +template +inline void boost::dll::experimental::imported_class::move_assign(imported_class& lhs) +{ + this->call("operator=")(static_cast(*lhs._data)); +} + + + +/*! +* Returns an instance of \ref imported_class which allows to call or import more functions. +* It takes a copy of the smart_libray, so no added type_aliases will be visible, +* for the object. +* +* Few compilers do implement an allocating constructor, which allows the construction +* of the class without knowing the size. That is not portable, so the actual size of the class +* shall always be provided. +* +* \b Example: +* +* \code +* auto import_class(lib, "class_name", 20, "param1", 42); +* \endcode +* +* In this example we construct an instance of the class "class_name" with the size 20, which has "type_alias" as an alias, +* through a constructor which takes a const-ref of std::string and an std::size_t parameter. +* +* \tparam T Class type or alias +* \tparam Args Constructor argument list. +* \param lib Path to shared library or shared library to load function from. +* \param name Null-terminated C or C++ mangled name of the function to import. Can handle std::string, char*, const char*. +* \param mode An mode that will be used on library load. +* +* \return class object. +* +* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. +* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. +*/ +template imported_class +import_class(const smart_library& lib_, std::size_t size, Args...args) +{ + smart_library lib(lib_); + + return imported_class::template make(boost::move(lib), size, static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(const smart_library& lib_, Args...args) +{ + smart_library lib(lib_); + return imported_class::template make(boost::move(lib), static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(const smart_library& lib_, const std::string & alias_name, Args...args) +{ + smart_library lib(lib_); + lib.add_type_alias(alias_name); + return imported_class::template make(boost::move(lib), static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(const smart_library& lib_, std::size_t size, const std::string & alias_name, Args...args) +{ + smart_library lib(lib_); + + lib.add_type_alias(alias_name); + return imported_class::template make(boost::move(lib), size, static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(const smart_library& lib_, const std::string & alias_name, std::size_t size, Args...args) +{ + smart_library lib(lib_); + + lib.add_type_alias(alias_name); + return imported_class::template make(boost::move(lib), size, static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(smart_library && lib, Args...args) +{ + return imported_class::template make(boost::move(lib), static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(smart_library && lib, const std::string & alias_name, Args...args) +{ + lib.add_type_alias(alias_name); + return imported_class::template make(boost::move(lib), static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(smart_library && lib, std::size_t size, Args...args) +{ + return imported_class::template make(boost::move(lib), size, static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(smart_library && lib, std::size_t size, const std::string & alias_name, Args...args) +{ + lib.add_type_alias(alias_name); + return imported_class::template make(boost::move(lib), size, static_cast(args)...); +} + +//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) +template imported_class +import_class(smart_library && lib, const std::string & alias_name, std::size_t size, Args...args) +{ + lib.add_type_alias(alias_name); + return imported_class::template make(boost::move(lib), size, static_cast(args)...); +} + + + +/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) + * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. + */ + +template imported_class +import_class(smart_library & lib, Args...args) +{ + return imported_class::template make(lib, static_cast(args)...); +} + +/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) + * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. + */ +template imported_class +import_class(smart_library & lib, const std::string & alias_name, Args...args) +{ + lib.add_type_alias(alias_name); + return imported_class::template make(lib, static_cast(args)...); +} + +/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) + * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. + */ +template imported_class +import_class(smart_library & lib, std::size_t size, Args...args) +{ + return imported_class::template make(lib, size, static_cast(args)...); +} + +/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) + * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. + */ +template imported_class +import_class(smart_library & lib, std::size_t size, const std::string & alias_name, Args...args) +{ + lib.add_type_alias(alias_name); + return imported_class::template make(lib, size, static_cast(args)...); +} + +/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) + * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. + */ +template imported_class +import_class(smart_library & lib, const std::string & alias_name, std::size_t size, Args...args) +{ + lib.add_type_alias(alias_name); + return imported_class::template make(lib, size, static_cast(args)...); +} + +} +} +} + + + +#endif /* BOOST_DLL_IMPORT_CLASS_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp b/contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp new file mode 100644 index 00000000000..69490ed19d7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp @@ -0,0 +1,309 @@ +// Copyright 2015-2016 Klemens D. Morgenstern +// +// 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) + + +#ifndef BOOST_DLL_IMPORT_MANGLED_HPP_ +#define BOOST_DLL_IMPORT_MANGLED_HPP_ + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { namespace experimental { + +namespace detail +{ + + + +template +class mangled_library_function { + // Copying of `boost::dll::shared_library` is very expensive, so we use a `shared_ptr` to make it faster. + boost::shared_ptr lib_; + function_tuple f_; +public: + constexpr mangled_library_function(const boost::shared_ptr& lib, Ts*... func_ptr) BOOST_NOEXCEPT + : lib_(lib) + , f_(func_ptr...) + {} + + + // Compilation error at this point means that imported function + // was called with unmatching parameters. + // + // Example: + // auto f = dll::import_mangled("function", "lib.so"); + // f("Hello"); // error: invalid conversion from 'const char*' to 'int' + // f(1, 2); // error: too many arguments to function + // f(); // error: too few arguments to function + template + auto operator()(Args&&... args) const + -> decltype( f_(static_cast(args)...) ) + { + return f_(static_cast(args)...); + } +}; + + +template +class mangled_library_mem_fn; + +template +class mangled_library_mem_fn> { + // Copying of `boost::dll::shared_library` is very expensive, so we use a `shared_ptr` to make it faster. + typedef mem_fn_tuple call_tuple_t; + boost::shared_ptr lib_; + call_tuple_t f_; + +public: + constexpr mangled_library_mem_fn(const boost::shared_ptr& lib, typename Ts::mem_fn... func_ptr) BOOST_NOEXCEPT + : lib_(lib) + , f_(func_ptr...) + {} + + template + auto operator()(ClassIn *cl, Args&&... args) const + -> decltype( f_(cl, static_cast(args)...) ) + { + return f_(cl, static_cast(args)...); + } +}; + + + + +// simple enough to be here +template struct is_variable : boost::false_type {}; +template struct is_variable> : boost::is_object {}; + +template ::value, + bool isMemFn = is_mem_fn_seq ::value, + bool isVariable = is_variable ::value> +struct mangled_import_type; + +template +struct mangled_import_type, true,false,false> //is function +{ + typedef boost::dll::experimental::detail::mangled_library_function type; + static type make( + const boost::dll::experimental::smart_library& p, + const std::string& name) + { + return type( + boost::make_shared(p.shared_lib()), + boost::addressof(p.get_function(name))...); + } +}; + +template +struct mangled_import_type, false, true, false> //is member-function +{ + typedef typename boost::dll::experimental::detail::make_mem_fn_seq::type actual_sequence; + typedef typename boost::dll::experimental::detail::mangled_library_mem_fn type; + + + template + static type make_impl( + const boost::dll::experimental::smart_library& p, + const std::string & name, + sequence * ) + { + return type(boost::make_shared(p.shared_lib()), + p.get_mem_fn(name)...); + } + + static type make( + const boost::dll::experimental::smart_library& p, + const std::string& name) + { + return make_impl(p, name, static_cast(nullptr)); + } + +}; + +template +struct mangled_import_type, false, false, true> //is variable +{ + typedef boost::shared_ptr type; + + static type make( + const boost::dll::experimental::smart_library& p, + const std::string& name) + { + return type( + boost::make_shared(p.shared_lib()), + boost::addressof(p.get_variable(name))); + } + +}; + + +} // namespace detail + + +#ifndef BOOST_DLL_DOXYGEN +# define BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE inline typename \ + boost::dll::experimental::detail::mangled_import_type>::type +#endif + +/* + * Variants: + * import_mangled("Stuff"); + * import_mangled("Function"); + * import mangled("Function"); + */ + +/*! +* Returns callable object or boost::shared_ptr that holds the symbol imported +* from the loaded library. Returned value refcounts usage +* of the loaded shared library, so that it won't get unload until all copies of return value +* are not destroyed. +* +* For importing symbols by \b alias names use \forcedlink{import_alias} method. +* +* \b Examples: +* +* \code +* boost::function f = import_mangled("test_lib.so", "integer_func_name"); +* +* auto f_cpp11 = import_mangled("test_lib.so", "integer_func_name"); +* \endcode +* +* \code +* boost::shared_ptr i = import_mangled("test_lib.so", "integer_name"); +* \endcode +* +* Additionally you can also import overloaded symbols, including member-functions. +* +* \code +* auto fp = import_mangled("test_lib.so", "func"); +* \endcode +* +* \code +* auto fp = import_mangled("test_lib.so", "func"); +* \endcode +* +* If qualified member-functions are needed, this can be set by repeating the class name with const or volatile. +* All following signatures after the redifintion will use this, i.e. the latest. +* +* * * \code +* auto fp = import_mangled("test_lib.so", "func"); +* \endcode +* +* \b Template \b parameter \b T: Type of the symbol that we are going to import. Must be explicitly specified. +* +* \param lib Path to shared library or shared library to load function from. +* \param name Null-terminated C or C++ mangled name of the function to import. Can handle std::string, char*, const char*. +* \param mode An mode that will be used on library load. +* +* \return callable object if T is a function type, or boost::shared_ptr if T is an object type. +* +* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. +* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. +*/ + + +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const boost::filesystem::path& lib, const char* name, + load_mode::type mode = load_mode::default_mode) +{ + typedef typename boost::dll::experimental::detail::mangled_import_type< + boost::dll::experimental::detail::sequence> type; + + boost::dll::experimental::smart_library p(lib, mode); + //the load + return type::make(p, name); +} + + + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const boost::filesystem::path& lib, const std::string& name, + load_mode::type mode = load_mode::default_mode) +{ + return import_mangled(lib, name.c_str(), mode); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const smart_library& lib, const char* name) { + typedef typename boost::dll::experimental::detail::mangled_import_type> type; + + return type::make(lib, name); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const smart_library& lib, const std::string& name) { + return import_mangled(lib, name.c_str()); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(smart_library) lib, const char* name) { + typedef typename boost::dll::experimental::detail::mangled_import_type> type; + + return type::make(lib, name); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(smart_library) lib, const std::string& name) { + return import_mangled(boost::move(lib), name.c_str()); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const shared_library& lib, const char* name) { + typedef typename boost::dll::experimental::detail::mangled_import_type> type; + + boost::shared_ptr p = boost::make_shared(lib); + return type::make(p, name); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const shared_library& lib, const std::string& name) { + return import_mangled(lib, name.c_str()); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(shared_library) lib, const char* name) { + typedef typename boost::dll::experimental::detail::mangled_import_type> type; + + boost::dll::experimental::smart_library p(boost::move(lib)); + + return type::make(p, name); +} + +//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) +template +BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(shared_library) lib, const std::string& name) { + return import_mangled(boost::move(lib), name.c_str()); +} + +#undef BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE + +}}} + + +#endif /* BOOST_DLL_IMPORT_MANGLED_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp new file mode 100644 index 00000000000..4666a16353a --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp @@ -0,0 +1,181 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_LIBRARY_INFO_HPP +#define BOOST_DLL_LIBRARY_INFO_HPP + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +/// \file boost/dll/library_info.hpp +/// \brief Contains only the boost::dll::library_info class that is capable of +/// extracting different information from binaries. + +namespace boost { namespace dll { + +/*! +* \brief Class that is capable of extracting different information from a library or binary file. +* Currently understands ELF, MACH-O and PE formats on all the platforms. +*/ +class library_info: private boost::noncopyable { +private: + boost::filesystem::ifstream f_; + + boost::aligned_storage< // making my own std::aligned_union from scratch. TODO: move to TypeTraits + boost::mpl::deref< + boost::mpl::max_element< + boost::mpl::vector_c + >::type + >::type::value + >::type impl_; + + /// @cond + boost::dll::detail::x_info_interface& impl() BOOST_NOEXCEPT { + return *reinterpret_cast(impl_.address()); + } + + inline static void throw_if_in_32bit_impl(boost::true_type /* is_32bit_platform */) { + boost::throw_exception(std::runtime_error("Not native format: 64bit binary")); + } + + inline static void throw_if_in_32bit_impl(boost::false_type /* is_32bit_platform */) BOOST_NOEXCEPT {} + + + inline static void throw_if_in_32bit() { + throw_if_in_32bit_impl( boost::integral_constant() ); + } + + static void throw_if_in_windows() { +#if BOOST_OS_WINDOWS + boost::throw_exception(std::runtime_error("Not native format: not a PE binary")); +#endif + } + + static void throw_if_in_linux() { +#if !BOOST_OS_WINDOWS && !BOOST_OS_MACOS && !BOOST_OS_IOS + boost::throw_exception(std::runtime_error("Not native format: not an ELF binary")); +#endif + } + + static void throw_if_in_macos() { +#if BOOST_OS_MACOS || BOOST_OS_IOS + boost::throw_exception(std::runtime_error("Not native format: not an Mach-O binary")); +#endif + } + + void init(bool throw_if_not_native) { + + if (boost::dll::detail::elf_info32::parsing_supported(f_)) { + if (throw_if_not_native) { throw_if_in_windows(); throw_if_in_macos(); } + + new (impl_.address()) boost::dll::detail::elf_info32(f_); + } else if (boost::dll::detail::elf_info64::parsing_supported(f_)) { + if (throw_if_not_native) { throw_if_in_windows(); throw_if_in_macos(); throw_if_in_32bit(); } + + new (impl_.address()) boost::dll::detail::elf_info64(f_); + } else if (boost::dll::detail::pe_info32::parsing_supported(f_)) { + if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_macos(); } + + new (impl_.address()) boost::dll::detail::pe_info32(f_); + } else if (boost::dll::detail::pe_info64::parsing_supported(f_)) { + if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_macos(); throw_if_in_32bit(); } + + new (impl_.address()) boost::dll::detail::pe_info64(f_); + } else if (boost::dll::detail::macho_info32::parsing_supported(f_)) { + if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_windows(); } + + new (impl_.address()) boost::dll::detail::macho_info32(f_); + } else if (boost::dll::detail::macho_info64::parsing_supported(f_)) { + if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_windows(); throw_if_in_32bit(); } + + new (impl_.address()) boost::dll::detail::macho_info64(f_); + } else { + boost::throw_exception(std::runtime_error("Unsupported binary format")); + } + } + /// @endcond + +public: + /*! + * Opens file with specified path and prepares for information extraction. + * \param library_path Path to the binary file from which the info must be extracted. + * \param throw_if_not_native_format Throw an exception if this file format is not + * supported by OS. + */ + explicit library_info(const boost::filesystem::path& library_path, bool throw_if_not_native_format = true) + : f_(library_path, std::ios_base::in | std::ios_base::binary) + , impl_() + { + f_.exceptions( + boost::filesystem::ifstream::failbit + | boost::filesystem::ifstream::badbit + | boost::filesystem::ifstream::eofbit + ); + + init(throw_if_not_native_format); + } + + /*! + * \return List of sections that exist in binary file. + */ + std::vector sections() { + return impl().sections(); + } + + /*! + * \return List of all the exportable symbols from all the sections that exist in binary file. + */ + std::vector symbols() { + return impl().symbols(); + } + + /*! + * \param section_name Name of the section from which symbol names must be returned. + * \return List of symbols from the specified section. + */ + std::vector symbols(const char* section_name) { + return impl().symbols(section_name); + } + + + //! \overload std::vector symbols(const char* section_name) + std::vector symbols(const std::string& section_name) { + return impl().symbols(section_name.c_str()); + } + + /*! + * \throw Nothing. + */ + ~library_info() BOOST_NOEXCEPT { + typedef boost::dll::detail::x_info_interface T; + impl().~T(); + } +}; + +}} // namespace boost::dll +#endif // BOOST_DLL_LIBRARY_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp new file mode 100644 index 00000000000..3f94e2ee1bf --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp @@ -0,0 +1,237 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015-2017 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_RUNTIME_SYMBOL_INFO_HPP +#define BOOST_DLL_RUNTIME_SYMBOL_INFO_HPP + +#include +#include +#include +#include +#if BOOST_OS_WINDOWS +# include +# include +#else +# include +# include +#endif + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +/// \file boost/dll/runtime_symbol_info.hpp +/// \brief Provides methods for getting acceptable by boost::dll::shared_library location of symbol, source line or program. +namespace boost { namespace dll { + +#if BOOST_OS_WINDOWS +namespace detail { + inline boost::filesystem::path program_location_impl(boost::system::error_code& ec) { + return boost::dll::detail::path_from_handle(NULL, ec); + } +} // namespace detail +#endif + + /*! + * On success returns full path and name to the binary object that holds symbol pointed by ptr_to_symbol. + * + * \param ptr_to_symbol Pointer to symbol which location is to be determined. + * \param ec Variable that will be set to the result of the operation. + * \return Path to the binary object that holds symbol or empty path in case error. + * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. + * + * \b Examples: + * \code + * int main() { + * dll::symbol_location_ptr(std::set_terminate(0)); // returns "/some/path/libmy_terminate_handler.so" + * dll::symbol_location_ptr(::signal(SIGSEGV, SIG_DFL)); // returns "/some/path/libmy_symbol_handler.so" + * } + * \endcode + */ + template + inline boost::filesystem::path symbol_location_ptr(T ptr_to_symbol, boost::system::error_code& ec) { + BOOST_STATIC_ASSERT_MSG(boost::is_pointer::value, "boost::dll::symbol_location_ptr works only with pointers! `ptr_to_symbol` must be a pointer"); + boost::filesystem::path ret; + if (!ptr_to_symbol) { + ec = boost::system::error_code( + boost::system::errc::bad_address, + boost::system::generic_category() + ); + + return ret; + } + ec.clear(); + + const void* ptr = boost::dll::detail::aggressive_ptr_cast(ptr_to_symbol); + +#if BOOST_OS_WINDOWS + boost::detail::winapi::MEMORY_BASIC_INFORMATION_ mbi; + if (!boost::detail::winapi::VirtualQuery(ptr, &mbi, sizeof(mbi))) { + ec = boost::dll::detail::last_error_code(); + return ret; + } + + return boost::dll::detail::path_from_handle(reinterpret_cast(mbi.AllocationBase), ec); +#else + Dl_info info; + + // Some of the libc headers miss `const` in `dladdr(const void*, Dl_info*)` + const int res = dladdr(const_cast(ptr), &info); + + if (res) { + ret = info.dli_fname; + } else { + boost::dll::detail::reset_dlerror(); + ec = boost::system::error_code( + boost::system::errc::bad_address, + boost::system::generic_category() + ); + } + + return ret; +#endif + } + + //! \overload symbol_location_ptr(const void* ptr_to_symbol, boost::system::error_code& ec) + template + inline boost::filesystem::path symbol_location_ptr(T ptr_to_symbol) { + boost::filesystem::path ret; + boost::system::error_code ec; + ret = boost::dll::symbol_location_ptr(ptr_to_symbol, ec); + + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::symbol_location_ptr(T ptr_to_symbol) failed"); + } + + return ret; + } + + /*! + * On success returns full path and name of the binary object that holds symbol. + * + * \tparam T Type of the symbol, must not be explicitly specified. + * \param symbol Symbol which location is to be determined. + * \param ec Variable that will be set to the result of the operation. + * \return Path to the binary object that holds symbol or empty path in case error. + * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. + * + * \b Examples: + * \code + * int var; + * void foo() {} + * + * int main() { + * dll::symbol_location(var); // returns program location + * dll::symbol_location(foo); // returns program location + * dll::symbol_location(std::cerr); // returns location of libstdc++: "/usr/lib/x86_64-linux-gnu/libstdc++.so.6" + * dll::symbol_location(std::placeholders::_1); // returns location of libstdc++: "/usr/lib/x86_64-linux-gnu/libstdc++.so.6" + * dll::symbol_location(std::puts); // returns location of libc: "/lib/x86_64-linux-gnu/libc.so.6" + * } + * \endcode + */ + template + inline boost::filesystem::path symbol_location(const T& symbol, boost::system::error_code& ec) { + ec.clear(); + return boost::dll::symbol_location_ptr( + boost::dll::detail::aggressive_ptr_cast(boost::addressof(symbol)), + ec + ); + } + +#if BOOST_COMP_MSVC < BOOST_VERSION_NUMBER(14,0,0) + // Without this MSVC 7.1 fails with: + // ..\boost\dll\runtime_symbol_info.hpp(133) : error C2780: 'filesystem::path dll::symbol_location(const T &)' : expects 1 arguments - 2 provided + template + inline boost::filesystem::path symbol_location(const T& symbol, const char* /*workaround*/ = 0) +#else + //! \overload symbol_location(const T& symbol, boost::system::error_code& ec) + template + inline boost::filesystem::path symbol_location(const T& symbol) +#endif + { + boost::filesystem::path ret; + boost::system::error_code ec; + ret = boost::dll::symbol_location_ptr( + boost::dll::detail::aggressive_ptr_cast(boost::addressof(symbol)), + ec + ); + + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::symbol_location(const T& symbol) failed"); + } + + return ret; + } + + /// @cond + // We have anonymous namespace here to make sure that `this_line_location()` method is instantiated in + // current translation unit and is not shadowed by instantiations from other units. + namespace { + /// @endcond + + /*! + * On success returns full path and name of the binary object that holds the current line of code + * (the line in which the `this_line_location()` method was called). + * + * \param ec Variable that will be set to the result of the operation. + * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. + */ + static inline boost::filesystem::path this_line_location(boost::system::error_code& ec) { + typedef boost::filesystem::path(func_t)(boost::system::error_code& ); + func_t& f = this_line_location; + return boost::dll::symbol_location(f, ec); + } + + //! \overload this_line_location(boost::system::error_code& ec) + static inline boost::filesystem::path this_line_location() { + boost::filesystem::path ret; + boost::system::error_code ec; + ret = this_line_location(ec); + + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::this_line_location() failed"); + } + + return ret; + } + + /// @cond + } // anonymous namespace + /// @endcond + + /*! + * On success returns full path and name of the currently running program (the one which contains the `main()` function). + * + * Return value can be used as a parameter for shared_library. See Tutorial "Linking plugin into the executable" + * for usage example. Flag '-rdynamic' must be used when linking the plugin into the executable + * on Linux OS. + * + * \param ec Variable that will be set to the result of the operation. + * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. + */ + inline boost::filesystem::path program_location(boost::system::error_code& ec) { + ec.clear(); + return boost::dll::detail::program_location_impl(ec); + } + + //! \overload program_location(boost::system::error_code& ec) { + inline boost::filesystem::path program_location() { + boost::filesystem::path ret; + boost::system::error_code ec; + ret = boost::dll::detail::program_location_impl(ec); + + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::program_location() failed"); + } + + return ret; + } + +}} // namespace boost::dll + +#endif // BOOST_DLL_RUNTIME_SYMBOL_INFO_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp b/contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp new file mode 100644 index 00000000000..86aed9433ed --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp @@ -0,0 +1,550 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015-2016 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_SHARED_LIBRARY_HPP +#define BOOST_DLL_SHARED_LIBRARY_HPP + +/// \file boost/dll/shared_library.hpp +/// \brief Contains the boost::dll::shared_library class, core class for all the +/// DLL/DSO operations. + +#include +#include +#include +#include +#include +#include +#include + +#if BOOST_OS_WINDOWS +# include +#else +# include +#endif + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +namespace boost { namespace dll { + +/*! +* \brief This class can be used to load a +* Dynamic link libraries (DLL's) or Shared Libraries, also know +* as dynamic shared objects (DSO's) and get their exported +* symbols (functions and variables). +* +* shared_library instances share reference count to an actual loaded DLL/DSO, so it +* is safe and memory efficient to have multiple instances of shared_library referencing the same DLL/DSO +* even if those instances were loaded using different paths (relative + absolute) referencing the same object. +* +* On Linux/POSIX link with library "dl". "-fvisibility=hidden" flag is also recommended for use on Linux/POSIX. +*/ +class shared_library +/// @cond + : private boost::dll::detail::shared_library_impl +/// @endcond +{ + typedef boost::dll::detail::shared_library_impl base_t; + BOOST_COPYABLE_AND_MOVABLE(shared_library) + +public: +#ifdef BOOST_DLL_DOXYGEN + typedef platform_specific native_handle_t; +#else + typedef shared_library_impl::native_handle_t native_handle_t; +#endif + + /*! + * Creates in anstance that does not reference any DLL/DSO. + * + * \post this->is_loaded() returns false. + * \throw Nothing. + */ + shared_library() BOOST_NOEXCEPT {} + + /*! + * Copy constructor that increments the reference count of an underlying shared library. + * Same as calling constructor with `lib.location()` parameter. + * + * \param lib A library to copy. + * \post lib == *this + * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. + */ + shared_library(const shared_library& lib) + : base_t() + { + assign(lib); + } + + /*! + * Copy constructor that increments the reference count of an underlying shared library. + * Same as calling constructor with `lib.location(), ec` parameters. + * + * \param lib A shared library to copy. + * \param ec Variable that will be set to the result of the operation. + * \post lib == *this + * \throw std::bad_alloc in case of insufficient memory. + */ + shared_library(const shared_library& lib, boost::system::error_code& ec) + : base_t() + { + assign(lib, ec); + } + + /*! + * Move constructor. Does not invalidate existing symbols and functions loaded from lib. + * + * \param lib A shared library to move from. + * \post lib.is_loaded() returns false, this->is_loaded() return true. + * \throw Nothing. + */ + shared_library(BOOST_RV_REF(shared_library) lib) BOOST_NOEXCEPT + : base_t(boost::move(static_cast(lib))) + {} + + /*! + * Loads a library by specified path with a specified mode. + * + * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, + * const wchar_t* or boost::filesystem::path. + * \param mode A mode that will be used on library load. + * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. + */ + explicit shared_library(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { + shared_library::load(lib_path, mode); + } + + /*! + * Loads a library by specified path with a specified mode. + * + * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, + * const wchar_t* or boost::filesystem::path. + * \param mode A mode that will be used on library load. + * \param ec Variable that will be set to the result of the operation. + * \throw std::bad_alloc in case of insufficient memory. + */ + shared_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { + shared_library::load(lib_path, mode, ec); + } + + //! \overload shared_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) + shared_library(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { + shared_library::load(lib_path, mode, ec); + } + + /*! + * Assignment operator. If this->is_loaded() then calls this->unload(). Does not invalidate existing symbols and functions loaded from lib. + * + * \param lib A shared library to assign from. + * \post lib == *this + * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. + */ + shared_library& operator=(BOOST_COPY_ASSIGN_REF(shared_library) lib) { + boost::system::error_code ec; + assign(lib, ec); + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::shared_library::operator= failed"); + } + + return *this; + } + + /*! + * Move assignment operator. If this->is_loaded() then calls this->unload(). Does not invalidate existing symbols and functions loaded from lib. + * + * \param lib A library to move from. + * \post lib.is_loaded() returns false. + * \throw Nothing. + */ + shared_library& operator=(BOOST_RV_REF(shared_library) lib) BOOST_NOEXCEPT { + if (lib.native() != native()) { + swap(lib); + } + + return *this; + } + + /*! + * Destroys the object by calling `unload()`. If library was loaded multiple times + * by different instances, the actual DLL/DSO won't be unloaded until + * there is at least one instance that references the DLL/DSO. + * + * \throw Nothing. + */ + ~shared_library() BOOST_NOEXCEPT {} + + /*! + * Makes *this share the same shared object as lib. If *this is loaded, then unloads it. + * + * \post lib.location() == this->location(), lib == *this + * \param lib A library to copy. + * \param ec Variable that will be set to the result of the operation. + * \throw std::bad_alloc in case of insufficient memory. + */ + shared_library& assign(const shared_library& lib, boost::system::error_code& ec) { + ec.clear(); + + if (native() == lib.native()) { + return *this; + } + + if (!lib) { + unload(); + return *this; + } + + boost::filesystem::path loc = lib.location(ec); + if (ec) { + return *this; + } + + shared_library copy(loc, ec); + if (ec) { + return *this; + } + + swap(copy); + return *this; + } + + /*! + * Makes *this share the same shared object as lib. If *this is loaded, then unloads it. + * + * \param lib A library instance to assign from. + * \post lib.location() == this->location() + * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. + */ + shared_library& assign(const shared_library& lib) { + boost::system::error_code ec; + assign(lib, ec); + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::shared_library::assign() failed"); + } + + return *this; + } + + /*! + * Loads a library by specified path with a specified mode. + * + * Note that if some library is already loaded in this instance, load will + * call unload() and then load the new provided library. + * + * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, + * const wchar_t* or boost::filesystem::path. + * \param mode A mode that will be used on library load. + * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. + * + */ + void load(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { + boost::system::error_code ec; + + base_t::load(lib_path, mode, ec); + + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::shared_library::load() failed"); + } + } + + /*! + * Loads a library by specified path with a specified mode. + * + * Note that if some library is already loaded in this instance, load will + * call unload() and then load the new provided library. + * + * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, + * const wchar_t* or boost::filesystem::path. + * \param ec Variable that will be set to the result of the operation. + * \param mode A mode that will be used on library load. + * \throw std::bad_alloc in case of insufficient memory. + */ + void load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { + ec.clear(); + base_t::load(lib_path, mode, ec); + } + + //! \overload void load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) + void load(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { + ec.clear(); + base_t::load(lib_path, mode, ec); + } + + /*! + * Unloads a shared library. If library was loaded multiple times + * by different instances, the actual DLL/DSO won't be unloaded until + * there is at least one instance that references the DLL/DSO. + * + * \post this->is_loaded() returns false. + * \throw Nothing. + */ + void unload() BOOST_NOEXCEPT { + base_t::unload(); + } + + /*! + * Check if an library is loaded. + * + * \return true if a library has been loaded. + * \throw Nothing. + */ + bool is_loaded() const BOOST_NOEXCEPT { + return base_t::is_loaded(); + } + + /*! + * Check if an library is not loaded. + * + * \return true if a library has not been loaded. + * \throw Nothing. + */ + bool operator!() const BOOST_NOEXCEPT { + return !is_loaded(); + } + + /*! + * Check if an library is loaded. + * + * \return true if a library has been loaded. + * \throw Nothing. + */ + BOOST_EXPLICIT_OPERATOR_BOOL() + + /*! + * Search for a given symbol on loaded library. Works for all symbols, including alias names. + * + * \param symbol_name Null-terminated symbol name. Can handle std::string, char*, const char*. + * \return `true` if the loaded library contains a symbol with a given name. + * \throw Nothing. + */ + bool has(const char* symbol_name) const BOOST_NOEXCEPT { + boost::system::error_code ec; + return is_loaded() && !!base_t::symbol_addr(symbol_name, ec) && !ec; + } + + //! \overload bool has(const char* symbol_name) const + bool has(const std::string& symbol_name) const BOOST_NOEXCEPT { + return has(symbol_name.c_str()); + } + + /*! + * Returns reference to the symbol (function or variable) with the given name from the loaded library. + * This call will always succeed and throw nothing if call to `has(const char* )` + * member function with the same symbol name returned `true`. + * + * \b Example: + * \code + * int& i0 = lib.get("integer_name"); + * int& i1 = *lib.get("integer_alias_name"); + * \endcode + * + * \tparam T Type of the symbol that we are going to import. Must be explicitly specified. + * \param symbol_name Null-terminated symbol name. Can handle std::string, char*, const char*. + * \return Reference to the symbol. + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + */ + template + inline typename boost::enable_if_c::value || boost::is_reference::value, T>::type get(const std::string& symbol_name) const { + return get(symbol_name.c_str()); + } + + //! \overload T& get(const std::string& symbol_name) const + template + inline typename boost::disable_if_c::value || boost::is_reference::value, T&>::type get(const std::string& symbol_name) const { + return get(symbol_name.c_str()); + } + + //! \overload T& get(const std::string& symbol_name) const + template + inline typename boost::enable_if_c::value || boost::is_reference::value, T>::type get(const char* symbol_name) const { + return boost::dll::detail::aggressive_ptr_cast( + get_void(symbol_name) + ); + } + + //! \overload T& get(const std::string& symbol_name) const + template + inline typename boost::disable_if_c::value || boost::is_reference::value, T&>::type get(const char* symbol_name) const { + return *boost::dll::detail::aggressive_ptr_cast( + get_void(symbol_name) + ); + } + + /*! + * Returns a symbol (function or variable) from a shared library by alias name of the symbol. + * + * \b Example: + * \code + * int& i = lib.get_alias("integer_alias_name"); + * \endcode + * + * \tparam T Type of the symbol that we are going to import. Must be explicitly specified.. + * \param alias_name Null-terminated alias symbol name. Can handle std::string, char*, const char*. + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + */ + template + inline T& get_alias(const char* alias_name) const { + return *get(alias_name); + } + + //! \overload T& get_alias(const char* alias_name) const + template + inline T& get_alias(const std::string& alias_name) const { + return *get(alias_name.c_str()); + } + +private: + /// @cond + // get_void is required to reduce binary size: it does not depend on a template + // parameter and will be instantiated only once. + void* get_void(const char* sb) const { + boost::system::error_code ec; + + if (!is_loaded()) { + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + // report_error() calls dlsym, do not use it here! + boost::throw_exception( + boost::system::system_error( + ec, "boost::dll::shared_library::get() failed: no library was loaded" + ) + ); + } + + void* const ret = base_t::symbol_addr(sb, ec); + if (ec || !ret) { + boost::dll::detail::report_error(ec, "boost::dll::shared_library::get() failed"); + } + + return ret; + } + /// @endcond + +public: + + /*! + * Returns the native handler of the loaded library. + * + * \return Platform-specific handle. + */ + native_handle_t native() const BOOST_NOEXCEPT { + return base_t::native(); + } + + /*! + * Returns full path and name of this shared object. + * + * \b Example: + * \code + * shared_library lib("test_lib.dll"); + * filesystem::path full_path = lib.location(); // C:\Windows\System32\test_lib.dll + * \endcode + * + * \return Full path to the shared library. + * \throw boost::system::system_error, std::bad_alloc. + */ + boost::filesystem::path location() const { + boost::system::error_code ec; + if (!is_loaded()) { + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + boost::throw_exception( + boost::system::system_error( + ec, "boost::dll::shared_library::location() failed (no library was loaded)" + ) + ); + } + + boost::filesystem::path full_path = base_t::full_module_path(ec); + + if (ec) { + boost::dll::detail::report_error(ec, "boost::dll::shared_library::location() failed"); + } + + return full_path; + } + + /*! + * Returns full path and name of shared module. + * + * \b Example: + * \code + * shared_library lib("test_lib.dll"); + * filesystem::path full_path = lib.location(); // C:\Windows\System32\test_lib.dll + * \endcode + * + * \param ec Variable that will be set to the result of the operation. + * \return Full path to the shared library. + * \throw std::bad_alloc. + */ + boost::filesystem::path location(boost::system::error_code& ec) const { + if (!is_loaded()) { + ec = boost::system::error_code( + boost::system::errc::bad_file_descriptor, + boost::system::generic_category() + ); + + return boost::filesystem::path(); + } + + ec.clear(); + return base_t::full_module_path(ec); + } + + /*! + * Returns suffix of shared module: + * in a call to load() or the constructor/load. + * + * \return The suffix od shared module: ".dll" (Windows), ".so" (Unix/Linux/BSD), ".dylib" (MacOS/IOS) + */ + static boost::filesystem::path suffix() { + return base_t::suffix(); + } + + /*! + * Swaps two libraries. Does not invalidate existing symbols and functions loaded from libraries. + * + * \param rhs Library to swap with. + * \throw Nothing. + */ + void swap(shared_library& rhs) BOOST_NOEXCEPT { + base_t::swap(rhs); + } +}; + + + +/// Very fast equality check that compares the actual DLL/DSO objects. Throws nothing. +inline bool operator==(const shared_library& lhs, const shared_library& rhs) BOOST_NOEXCEPT { + return lhs.native() == rhs.native(); +} + +/// Very fast inequality check that compares the actual DLL/DSO objects. Throws nothing. +inline bool operator!=(const shared_library& lhs, const shared_library& rhs) BOOST_NOEXCEPT { + return lhs.native() != rhs.native(); +} + +/// Compare the actual DLL/DSO objects without any guarantee to be stable between runs. Throws nothing. +inline bool operator<(const shared_library& lhs, const shared_library& rhs) BOOST_NOEXCEPT { + return lhs.native() < rhs.native(); +} + +/// Swaps two shared libraries. Does not invalidate symbols and functions loaded from libraries. Throws nothing. +inline void swap(shared_library& lhs, shared_library& rhs) BOOST_NOEXCEPT { + lhs.swap(rhs); +} + +}} // boost::dll + +#endif // BOOST_DLL_SHARED_LIBRARY_HPP + diff --git a/contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp b/contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp new file mode 100644 index 00000000000..5627e42dfd1 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp @@ -0,0 +1,249 @@ +// Copyright 2014 Renato Tegon Forti, Antony Polukhin. +// Copyright 2015-2016 Antony Polukhin. +// +// 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) + +#ifndef BOOST_DLL_SHARED_LIBRARY_MODE_HPP +#define BOOST_DLL_SHARED_LIBRARY_MODE_HPP + +#include +#include +#include + +#if BOOST_OS_WINDOWS +//#include +#include +#else +# include +#endif + +#ifdef BOOST_HAS_PRAGMA_ONCE +# pragma once +#endif + +/// \file boost/dll/shared_library_load_mode.hpp +/// \brief Contains only the boost::dll::load_mode::type enum and operators related to it. + +namespace boost { namespace dll { namespace load_mode { + +/*! Library load modes. +* +* Each of system family provides own modes. Flags not supported by a particular platform will be silently ignored. +* +* For a detailed description of platform specific options see: +* Windows specific options, +* POSIX specific options. +* +*/ + +enum type { +#ifdef BOOST_DLL_DOXYGEN + /*! + * Default open mode. See the \b Default: comments below to find out the flags that are enabled by default. + */ + default_mode, + + /*! + * \b Platforms: Windows + * + * \b Default: disabled + * + * If this value is used, and the executable module is a DLL, the system does + * not call DllMain for process and thread initialization and termination. + * Also, the system does not load additional executable modules that are + * referenced by the specified module. + * + * Note Do not use this value; it is provided only for backward compatibility. + * If you are planning to access only data or resources in the DLL, use + * LOAD_LIBRARY_AS_DATAFILE_EXCLUSIVE or LOAD_LIBRARY_AS_IMAGE_RESOURCE + * or both. + */ + dont_resolve_dll_references, + + /*! + * \b Platforms: Windows + * + * \b Default: disabled + * + * If this value is used, the system does not check AppLocker rules or + * apply Software Restriction Policies for the DLL. + */ + load_ignore_code_authz_level, + + /*! + * \b Platforms: Windows + * + * \b Default: disabled + * + * If this value is used and lpFileName specifies an absolute path, + * the system uses the alternate file search strategy. + * + * This value cannot be combined with any LOAD_LIBRARY_SEARCH flag. + */ + load_with_altered_search_path, + + /*! + * \b Platforms: POSIX + * + * \b Default: enabled + * + * Relocations shall be performed at an implementation-defined time, ranging + * from the time of the dlopen() call until the first reference to a given + * symbol occurs. + * + * Specifying RTLD_LAZY should improve performance on implementations + * supporting dynamic symbol binding as a process may not reference all of + * the functions in any given object. And, for systems supporting dynamic + * symbol resolution for normal process execution, this behavior mimics + * the normal handling of process execution. + */ + rtld_lazy, + + /*! + * \b Platforms: POSIX + * + * \b Default: disabled + * + * All necessary relocations shall be performed when the object is first + * loaded. This may waste some processing if relocations are performed for + * functions that are never referenced. This behavior may be useful for + * plugins that need to know as soon as an object is loaded that all + * symbols referenced during execution are available. + */ + rtld_now, + + /*! + * \b Platforms: POSIX + * + * \b Default: disabled + * + * The object's symbols shall be made available for the relocation + * processing of any other object. In addition, symbol lookup using + * dlopen(0, mode) and an associated dlsym() allows objects loaded + * with this mode to be searched. + */ + rtld_global, + + /*! + * \b Platforms: POSIX + * + * \b Default: enabled + * + * The object's symbols shall not be made available for the relocation + * processing of any other object. + * + * This is a default Windows behavior that can not be changed. + */ + rtld_local, + + /*! + * \b Platforms: POSIX (requires glibc >= 2.3.4) + * + * \b Default: disabled + * + * The object will use its own symbols in preference to global symbols + * with the same name contained in libraries that have already been loaded. + * This flag is not specified in POSIX.1-2001. + */ + rtld_deepbind, + + /*! + * \b Platforms: Windows, POSIX + * + * \b Default: disabled + * + * Append a platform specific extension and prefix to shared library filename before trying to load it. + * If load attempt fails, try to load with exactly specified name. + * + * \b Example: + * \code + * // Opens `./my_plugins/plugin1.dll` on Windows, `./my_plugins/libplugin1.so` on Linux, `./my_plugins/libplugin1.dylib` on MacOS. + * // If that fails, loads `./my_plugins/plugin1` + * boost::dll::shared_library lib("./my_plugins/plugin1", load_mode::append_decorations); + * \endcode + */ + append_decorations, + /*! + * \b Platforms: Windows, POSIX + * + * \b Default: disabled + * + * Allow loading from system folders if path to library contains no parent path. + */ + search_system_folders +#elif BOOST_OS_WINDOWS + default_mode = 0, + dont_resolve_dll_references = boost::detail::winapi::DONT_RESOLVE_DLL_REFERENCES_, + load_ignore_code_authz_level = boost::detail::winapi::LOAD_IGNORE_CODE_AUTHZ_LEVEL_, + load_with_altered_search_path = boost::detail::winapi::LOAD_WITH_ALTERED_SEARCH_PATH_, + rtld_lazy = 0, + rtld_now = 0, + rtld_global = 0, + rtld_local = 0, + rtld_deepbind = 0, + append_decorations = 0x00800000, + search_system_folders = (append_decorations << 1) +#else + default_mode = 0, + dont_resolve_dll_references = 0, + load_ignore_code_authz_level = 0, + load_with_altered_search_path = 0, + rtld_lazy = RTLD_LAZY, + rtld_now = RTLD_NOW, + rtld_global = RTLD_GLOBAL, + rtld_local = RTLD_LOCAL, + +#if BOOST_LIB_C_GNU < BOOST_VERSION_NUMBER(2,3,4) + rtld_deepbind = 0, +#else + rtld_deepbind = RTLD_DEEPBIND, +#endif + + append_decorations = 0x00800000, + search_system_folders = (append_decorations << 1) +#endif +}; + + +/// Free operators for load_mode::type flag manipulation. +BOOST_CONSTEXPR inline type operator|(type left, type right) BOOST_NOEXCEPT { + return static_cast( + static_cast(left) | static_cast(right) + ); +} +BOOST_CXX14_CONSTEXPR inline type& operator|=(type& left, type right) BOOST_NOEXCEPT { + left = left | right; + return left; +} + +BOOST_CONSTEXPR inline type operator&(type left, type right) BOOST_NOEXCEPT { + return static_cast( + static_cast(left) & static_cast(right) + ); +} +BOOST_CXX14_CONSTEXPR inline type& operator&=(type& left, type right) BOOST_NOEXCEPT { + left = left & right; + return left; +} + +BOOST_CONSTEXPR inline type operator^(type left, type right) BOOST_NOEXCEPT { + return static_cast( + static_cast(left) ^ static_cast(right) + ); +} +BOOST_CXX14_CONSTEXPR inline type& operator^=(type& left, type right) BOOST_NOEXCEPT { + left = left ^ right; + return left; +} + +BOOST_CONSTEXPR inline type operator~(type left) BOOST_NOEXCEPT { + return static_cast( + ~static_cast(left) + ); +} + +}}} // boost::dll::load_mode + +#endif // BOOST_DLL_SHARED_LIBRARY_MODE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp b/contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp new file mode 100644 index 00000000000..c5859251de7 --- /dev/null +++ b/contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp @@ -0,0 +1,462 @@ +// Copyright 2016 Klemens Morgenstern +// +// 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) + +#ifndef BOOST_DLL_SMART_LIBRARY_HPP_ +#define BOOST_DLL_SMART_LIBRARY_HPP_ + +/// \file boost/dll/smart_library.hpp +/// \warning Extremely experimental! Requires C++14! Will change in next version of Boost! boost/dll/smart_library.hpp is not included in boost/dll.hpp +/// \brief Contains the boost::dll::experimental::smart_library class for loading mangled symbols. + +#if BOOST_COMP_GNUC || BOOST_COMP_CLANG || BOOST_COMP_HPACC || BOOST_COMP_IBM +#include +#elif BOOST_COMP_MSVC +#include +#else +#error "Compiler not supported" +#endif + +#include +#include +#include +#include +#include +#include +#include +#include + + + +namespace boost { +namespace dll { +namespace experimental { + +using boost::dll::detail::constructor; +using boost::dll::detail::destructor; + +/*! +* \brief This class is an extension of \ref shared_library, which allows to load C++ symbols. +* +* This class allows type safe loading of overloaded functions, member-functions, constructors and variables. +* It also allows to overwrite classes so they can be loaded, while being declared with different names. +* +* \warning Is still very experimental. +* +* Currently known limitations: +* +* Member functions must be defined outside of the class to be exported. That is: +* \code +* //not exported: +* struct BOOST_SYMBOL_EXPORT my_class { void func() {}}; +* //exported +* struct BOOST_SYMBOL_EXPORT my_class { void func();}; +* void my_class::func() {}; +* \endcode +* +* With the current analysis, the first version does get exported in MSVC. +* MinGW also does export it, BOOST_SYMBOL_EXPORT is written before it. To allow this on windows one can use +* BOOST_DLL_MEMBER_EXPORT for this, so that MinGW and MSVC can provide those functions. This does however not work with gcc on linux. +* +* Direct initialization of members. +* On linux the following member variable i will not be initialized when using the allocating contructor: +* \code +* struct BOOST_SYMBOL_EXPORT my_class { int i; my_class() : i(42) {} }; +* \endcode +* +* This does however not happen when the value is set inside the constructor function. +*/ +class smart_library { + shared_library _lib; + detail::mangled_storage_impl _storage; + +public: + /*! + * Get the underlying shared_library + */ + const shared_library &shared_lib() const {return _lib;} + + using mangled_storage = detail::mangled_storage_impl; + /*! + * Acces to the mangled storage, which is created on construction. + * + * \throw Nothing. + */ + const mangled_storage &symbol_storage() const {return _storage;} + + ///Overload, for current development. + mangled_storage &symbol_storage() {return _storage;} + + //! \copydoc shared_library::shared_library() + smart_library() BOOST_NOEXCEPT {}; + + //! \copydoc shared_library::shared_library(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) + smart_library(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { + _lib.load(lib_path, mode); + _storage.load(lib_path); + } + + //! \copydoc shared_library::shared_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) + smart_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { + load(lib_path, mode, ec); + } + + //! \copydoc shared_library::shared_library(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) + smart_library(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { + load(lib_path, mode, ec); + } + /*! + * copy a smart_library object. + * + * \param lib A smart_library to move from. + * + * \throw Nothing. + */ + smart_library(const smart_library & lib) BOOST_NOEXCEPT + : _lib(lib._lib), _storage(lib._storage) + {} + /*! + * Move a smart_library object. + * + * \param lib A smart_library to move from. + * + * \throw Nothing. + */ + smart_library(BOOST_RV_REF(smart_library) lib) BOOST_NOEXCEPT + : _lib(boost::move(lib._lib)), _storage(boost::move(lib._storage)) + {} + + /*! + * Construct from a shared_library object. + * + * \param lib A shared_library to move from. + * + * \throw Nothing. + */ + explicit smart_library(const shared_library & lib) BOOST_NOEXCEPT + : _lib(lib) + { + _storage.load(lib.location()); + } + /*! + * Construct from a shared_library object. + * + * \param lib A shared_library to move from. + * + * \throw Nothing. + */ + explicit smart_library(BOOST_RV_REF(shared_library) lib) BOOST_NOEXCEPT + : _lib(boost::move(static_cast(lib))) + { + _storage.load(lib.location()); + } + + /*! + * Destroys the smart_library. + * `unload()` is called if the DLL/DSO was loaded. If library was loaded multiple times + * by different instances of shared_library, the actual DLL/DSO won't be unloaded until + * there is at least one instance of shared_library. + * + * \throw Nothing. + */ + ~smart_library() BOOST_NOEXCEPT {}; + + //! \copydoc shared_library::load(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) + void load(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { + boost::system::error_code ec; + _storage.load(lib_path); + _lib.load(lib_path, mode, ec); + + if (ec) { + boost::dll::detail::report_error(ec, "load() failed"); + } + } + + //! \copydoc shared_library::load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) + void load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { + ec.clear(); + _storage.load(lib_path); + _lib.load(lib_path, mode, ec); + } + + //! \copydoc shared_library::load(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) + void load(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { + ec.clear(); + _storage.load(lib_path); + _lib.load(lib_path, mode, ec); + } + + /*! + * Load a variable from the referenced library. + * + * Unlinke shared_library::get this function will also load scoped variables, which also includes static class members. + * + * \note When mangled, MSVC will also check the type. + * + * \param name Name of the variable + * \tparam T Type of the variable + * \return A reference to the variable of type T. + * + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + */ + template + T& get_variable(const std::string &name) const { + return _lib.get(_storage.get_variable(name)); + } + + /*! + * Load a function from the referenced library. + * + * \b Example: + * + * \code + * smart_library lib("test_lib.so"); + * typedef int (&add_ints)(int, int); + * typedef double (&add_doubles)(double, double); + * add_ints f1 = lib.get_function ("func_name"); + * add_doubles f2 = lib.get_function("func_name"); + * \endcode + * + * \note When mangled, MSVC will also check the return type. + * + * \param name Name of the function. + * \tparam Func Type of the function, required for determining the overload + * \return A reference to the function of type F. + * + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + */ + template + Func& get_function(const std::string &name) const { + return _lib.get(_storage.get_function(name)); + } + + /*! + * Load a member-function from the referenced library. + * + * \b Example (import class is MyClass, which is available inside the library and the host): + * + * \code + * smart_library lib("test_lib.so"); + * + * typedef int MyClass(*func)(int); + * typedef int MyClass(*func_const)(int) const; + * + * add_ints f1 = lib.get_mem_fn ("MyClass::function"); + * add_doubles f2 = lib.get_mem_fn("MyClass::function"); + * \endcode + * + * \note When mangled, MSVC will also check the return type. + * + * \param name Name of the function. + * \tparam Class The class the function is a member of. If Class is const, the function will be assumed as taking a const this-pointer. The same applies for volatile. + * \tparam Func Signature of the function, required for determining the overload + * \return A pointer to the member-function with the signature provided + * + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + */ + template + typename boost::dll::detail::get_mem_fn_type::mem_fn get_mem_fn(const std::string& name) const { + return _lib.get::mem_fn>( + _storage.get_mem_fn(name) + ); + } + + /*! + * Load a constructor from the referenced library. + * + * \b Example (import class is MyClass, which is available inside the library and the host): + * + * \code + * smart_library lib("test_lib.so"); + * + * constructor(); + * \endcode + * + * \tparam Signature Signature of the function, required for determining the overload. The return type is the class which this is the constructor of. + * \return A constructor object. + * + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + */ + template + constructor get_constructor() const { + return boost::dll::detail::load_ctor(_lib, _storage.get_constructor()); + } + + /*! + * Load a destructor from the referenced library. + * + * \b Example (import class is MyClass, which is available inside the library and the host): + * + * \code + * smart_library lib("test_lib.so"); + * + * destructor f1 = lib.get_mem_fn(); + * \endcode + * + * \tparam Class The class whichs destructor shall be loaded + * \return A destructor object. + * + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + * + */ + template + destructor get_destructor() const { + return boost::dll::detail::load_dtor(_lib, _storage.get_destructor()); + } + /*! + * Load the typeinfo of the given type. + * + * \b Example (import class is MyClass, which is available inside the library and the host): + * + * \code + * smart_library lib("test_lib.so"); + * + * std::type_info &ti = lib.get_Type_info(); + * \endcode + * + * \tparam Class The class whichs typeinfo shall be loaded + * \return A reference to a type_info object. + * + * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. + * + */ + template + const std::type_info& get_type_info() const + { + return boost::dll::detail::load_type_info(_lib, _storage); + } + /** + * This function can be used to add a type alias. + * + * This is to be used, when a class shall be imported, which is not declared on the host side. + * + * Example: + * \code + * smart_library lib("test_lib.so"); + * + * lib.add_type_alias("MyClass"); //when using MyAlias, the library will look for MyClass + * + * //get the destructor of MyClass + * destructor dtor = lib.get_destructor(); + * \endcode + * + * + * \param name Name of the class the alias is for. + * + * \attention If the alias-type is not large enough for the imported class, it will result in undefined behaviour. + * \warning The alias will only be applied for the type signature, it will not replace the token in the scoped name. + */ + template void add_type_alias(const std::string& name) { + this->_storage.add_alias(name); + } + + //! \copydoc shared_library::unload() + void unload() BOOST_NOEXCEPT { + _storage.clear(); + _lib.unload(); + } + + //! \copydoc shared_library::is_loaded() const + bool is_loaded() const BOOST_NOEXCEPT { + return _lib.is_loaded(); + } + + //! \copydoc shared_library::operator!() const + bool operator!() const BOOST_NOEXCEPT { + return !is_loaded(); + } + + //! \copydoc shared_library::operator bool() const + BOOST_EXPLICIT_OPERATOR_BOOL() + + //! \copydoc shared_library::has(const char* symbol_name) const + bool has(const char* symbol_name) const BOOST_NOEXCEPT { + return _lib.has(symbol_name); + } + + //! \copydoc shared_library::has(const std::string& symbol_name) const + bool has(const std::string& symbol_name) const BOOST_NOEXCEPT { + return _lib.has(symbol_name); + } + + //! \copydoc shared_library::assign(const shared_library& lib) + smart_library& assign(const smart_library& lib) { + _lib.assign(lib._lib); + _storage.assign(lib._storage); + return *this; + } + + //! \copydoc shared_library::swap(shared_library& rhs) + void swap(smart_library& rhs) BOOST_NOEXCEPT { + _lib.swap(rhs._lib); + _storage.swap(rhs._storage); + } +}; + +/// Very fast equality check that compares the actual DLL/DSO objects. Throws nothing. +inline bool operator==(const smart_library& lhs, const smart_library& rhs) BOOST_NOEXCEPT { + return lhs.shared_lib().native() == rhs.shared_lib().native(); +} + +/// Very fast inequality check that compares the actual DLL/DSO objects. Throws nothing. +inline bool operator!=(const smart_library& lhs, const smart_library& rhs) BOOST_NOEXCEPT { + return lhs.shared_lib().native() != rhs.shared_lib().native(); +} + +/// Compare the actual DLL/DSO objects without any guarantee to be stable between runs. Throws nothing. +inline bool operator<(const smart_library& lhs, const smart_library& rhs) BOOST_NOEXCEPT { + return lhs.shared_lib().native() < rhs.shared_lib().native(); +} + +/// Swaps two shared libraries. Does not invalidate symbols and functions loaded from libraries. Throws nothing. +inline void swap(smart_library& lhs, smart_library& rhs) BOOST_NOEXCEPT { + lhs.swap(rhs); +} + + +#ifdef BOOST_DLL_DOXYGEN +/** Helper functions for overloads. + * + * Gets either a variable, function or member-function, depending on the signature. + * + * @code + * smart_library sm("lib.so"); + * get(sm, "space::value"); //import a variable + * get(sm, "space::func"); //import a function + * get(sm, "space::class_::mem_fn"); //import a member function + * @endcode + * + * @param sm A reference to the @ref smart_library + * @param name The name of the entity to import + */ +template +void get(const smart_library& sm, const std::string &name); +#endif + +template +T& get(const smart_library& sm, const std::string &name, typename boost::enable_if,T>::type* = nullptr) + +{ + return sm.get_variable(name); +} + +template +auto get(const smart_library& sm, const std::string &name, typename boost::enable_if>::type* = nullptr) +{ + return sm.get_function(name); +} + +template +auto get(const smart_library& sm, const std::string &name) -> typename detail::get_mem_fn_type::mem_fn +{ + return sm.get_mem_fn(name); +} + + +} /* namespace experimental */ +} /* namespace dll */ +} /* namespace boost */ + +#endif /* BOOST_DLL_SMART_LIBRARY_HPP_ */ diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp new file mode 100644 index 00000000000..4f75fc792a2 --- /dev/null +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -0,0 +1,42 @@ +#include + +namespace DB +{ + +CatBoostModel::CatBoostModel(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) + : lifetime(config, config_prefix) +{ + +} + +const ExternalLoadableLifetime & CatBoostModel::getLifetime() const +{ + return lifetime; +} + +bool CatBoostModel::isModified() const +{ + return true; +} + +std::unique_ptr CatBoostModel::cloneObject() const +{ + return nullptr; +} + +size_t CatBoostModel::getFloatFeaturesCount() const +{ + return 0; +} + +size_t CatBoostModel::getCatFeaturesCount() const +{ + return 0; +} + +void CatBoostModel::apply(const Columns & floatColumns, const Columns & catColumns, ColumnFloat64 & result) +{ + +} + +} diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h new file mode 100644 index 00000000000..3163532b009 --- /dev/null +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -0,0 +1,39 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ + +class CatBoostModel : public IExternalLoadable +{ +public: + CatBoostModel(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); + + const ExternalLoadableLifetime & getLifetime() const override; + + std::string getName() const override { return name; } + + bool supportUpdates() const override { return true; } + + bool isModified() const override; + + std::unique_ptr cloneObject() const override; + + std::exception_ptr getCreationException() const override { return creation_exception; } + + size_t getFloatFeaturesCount() const; + size_t getCatFeaturesCount() const; + + void apply(const Columns & floatColumns, const Columns & catColumns, ColumnFloat64 & result); + +private: + ExternalLoadableLifetime lifetime; + std::string name; + std::exception_ptr creation_exception; + +}; + +} From cbd03af500c618660090250b68313840b12dfb3b Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 4 Oct 2017 18:15:40 +0300 Subject: [PATCH 04/32] Fixed integration tests after introduction of 'clickhouse server' commands. [#CLICKHOUSE-2] --- dbms/tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 57f165d9099..b20749a08d3 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -176,6 +176,7 @@ services: - {logs_dir}:/var/log/clickhouse-server/ entrypoint: - /usr/bin/clickhouse + - server - --config-file=/etc/clickhouse-server/config.xml - --log-file=/var/log/clickhouse-server/clickhouse-server.log depends_on: {depends_on} From b60c5fb097da6bb9f72241864dfcaa8256f4917f Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 4 Oct 2017 18:18:05 +0300 Subject: [PATCH 05/32] Try fix work with not connected sockets (#1320) * Lazy fd_to_replica_state_idx init; Fix work with not connected sockets * Style * const --- dbms/src/Client/Connection.cpp | 4 ++-- dbms/src/Client/MultiplexedConnections.cpp | 13 ++++++++++--- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 8d3ab2a4f13..fc9315210b2 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -95,11 +95,11 @@ void Connection::disconnect() { //LOG_TRACE(log_wrapper.get(), "Disconnecting"); + in = nullptr; + out = nullptr; // can write to socket if (socket) socket->close(); socket = nullptr; - in = nullptr; - out = nullptr; connected = false; } diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index f13587d79c9..fb2baa105a6 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -20,7 +20,6 @@ MultiplexedConnections::MultiplexedConnections(Connection & connection, const Se ReplicaState replica_state; replica_state.connection = &connection; replica_states.push_back(replica_state); - fd_to_replica_state_idx.emplace(connection.socket->impl()->sockfd(), 0); active_connection_count = 1; } @@ -36,7 +35,6 @@ MultiplexedConnections::MultiplexedConnections( return; replica_states.reserve(connections.size()); - fd_to_replica_state_idx.reserve(connections.size()); for (size_t i = 0; i < connections.size(); ++i) { Connection * connection = &(*connections[i]); @@ -47,7 +45,6 @@ MultiplexedConnections::MultiplexedConnections( replica_state.connection = connection; replica_states.push_back(std::move(replica_state)); - fd_to_replica_state_idx.emplace(connection->socket->impl()->sockfd(), i); } active_connection_count = connections.size(); @@ -304,6 +301,16 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead } auto & socket = read_list[rand() % read_list.size()]; + if (fd_to_replica_state_idx.empty()) + { + fd_to_replica_state_idx.reserve(replica_states.size()); + size_t replica_state_number = 0; + for (const auto & replica_state : replica_states) + { + fd_to_replica_state_idx.emplace(replica_state.connection->socket->impl()->sockfd(), replica_state_number); + ++replica_state_number; + } + } return replica_states[fd_to_replica_state_idx.at(socket.impl()->sockfd())]; } From 93e1401b3575f7a86089e1524114f015b6f48fd9 Mon Sep 17 00:00:00 2001 From: robot-metrika-test Date: Thu, 5 Oct 2017 12:22:56 +0300 Subject: [PATCH 06/32] Auto version update to [54297] --- 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 f8aceaeac86..cbaed040a12 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.54294-testing) -set(VERSION_REVISION 54294) +set(VERSION_DESCRIBE v1.1.54297-testing) +set(VERSION_REVISION 54297) # end of autochange set (VERSION_MAJOR 1) diff --git a/debian/changelog b/debian/changelog index 1872c8a5e9b..403d0960054 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (1.1.54294) unstable; urgency=low +clickhouse (1.1.54297) unstable; urgency=low * Modified source code - -- Fri, 29 Sep 2017 01:08:46 +0300 + -- Thu, 05 Oct 2017 12:22:56 +0300 From e817de7e21c6806564ee36bb0c227bacfeb304b0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Oct 2017 21:05:30 +0300 Subject: [PATCH 07/32] added CatBoostModel [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 110 +++++++++++++++++++++++- dbms/src/Dictionaries/CatBoostModel.h | 19 +++- 2 files changed, 126 insertions(+), 3 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 4f75fc792a2..deb2a0e7af8 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -1,14 +1,122 @@ #include +#include +#include namespace DB { -CatBoostModel::CatBoostModel(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +namespace +{ + +struct CatBoostWrapperApi +{ + typedef void ModelCalcerHandle; + + ModelCalcerHandle * (* ModelCalcerCreate)(); + + void (* ModelCalcerDelete)(ModelCalcerHandle * calcer); + + const char * (* GetErrorString)(); + + bool (* LoadFullModelFromFile)(ModelCalcerHandle * calcer, const char * filename); + + bool (* CalcModelPredictionFlat)(ModelCalcerHandle * calcer, size_t docCount, + const float ** floatFeatures, size_t floatFeaturesSize, + double * result, size_t resultSize); + + bool (* CalcModelPrediction)(ModelCalcerHandle * calcer, size_t docCount, + const float ** floatFeatures, size_t floatFeaturesSize, + const char *** catFeatures, size_t catFeaturesSize, + double * result, size_t resultSize); + + bool (* CalcModelPredictionWithHashedCatFeatures)(ModelCalcerHandle * calcer, size_t docCount, + const float ** floatFeatures, size_t floatFeaturesSize, + const int ** catFeatures, size_t catFeaturesSize, + double * result, size_t resultSize); + + int (* GetStringCatFeatureHash)(const char * data, size_t size); + + int (* GetIntegerCatFeatureHash)(long long val); +}; + +class CatBoostWrapperHolder : public CatBoostWrapperApiProvider +{ +public: + CatBoostWrapperHolder(const std::string & lib_path) : lib(lib_path), lib_path(lib_path) { initApi(); } + + const CatBoostWrapperApi & getApi() const override { return api; } + const std::string & getCurrentPath() const { return lib_path; } + +private: + CatBoostWrapperApi api; + std::string lib_path; + boost::dll::shared_library lib; + + void initApi(); + + template + void load(T& func, const std::string & name) + { + using Type = std::remove_pointer::type; + func = lib.get(name); + } +}; + +void CatBoostWrapperHolder::initApi() +{ + load(api.ModelCalcerCreate, "ModelCalcerCreate"); + load(api.ModelCalcerDelete, "ModelCalcerDelete"); + load(api.GetErrorString, "GetErrorString"); + load(api.LoadFullModelFromFile, "LoadFullModelFromFile"); + load(api.CalcModelPredictionFlat, "CalcModelPredictionFlat"); + load(api.CalcModelPrediction, "CalcModelPrediction"); + load(api.CalcModelPredictionWithHashedCatFeatures, "CalcModelPredictionWithHashedCatFeatures"); + load(api.GetStringCatFeatureHash, "GetStringCatFeatureHash"); + load(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash"); +} + +std::shared_ptr getCatBoostWrapperHolder(const std::string & lib_path) +{ + static std::weak_ptr ptr; + static std::mutex mutex; + + std::lock_guard lock(mutex); + auto result = ptr.lock(); + + if (!result || result->getCurrentPath() != lib_path) + { + result = std::make_shared(lib_path); + /// This assignment is not atomic, which prevents from creating lock only inside 'if'. + ptr = result; + } + + return result; +} + +} + +CatBoostModel::CatBoostModel(const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, const std::string & lib_path) : lifetime(config, config_prefix) { } +CatBoostModel::CatBoostModel(const std::string & name, const std::string & model_path, const std::string & lib_path, + const ExternalLoadableLifetime & lifetime) + : name(name), model_path(model_path), lifetime(lifetime) +{ + try + { + api_provider = getCatBoostWrapperHolder(lib_path); + api = &api_provider->getApi(); + } + catch (...) + { + creation_exception = std::current_exception(); + } +} + const ExternalLoadableLifetime & CatBoostModel::getLifetime() const { return lifetime; diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index 3163532b009..adacffab05b 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -7,10 +7,20 @@ namespace DB { +struct CatBoostWrapperApi; +class CatBoostWrapperApiProvider +{ +public: + virtual ~CatBoostWrapperApiProvider() = default; + virtual const CatBoostWrapperApi & getApi() const = 0; +}; + + class CatBoostModel : public IExternalLoadable { public: - CatBoostModel(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); + CatBoostModel(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + const std::string & lib_path); const ExternalLoadableLifetime & getLifetime() const override; @@ -30,10 +40,15 @@ public: void apply(const Columns & floatColumns, const Columns & catColumns, ColumnFloat64 & result); private: - ExternalLoadableLifetime lifetime; std::string name; + std::string model_path; + ExternalLoadableLifetime lifetime; std::exception_ptr creation_exception; + std::shared_ptr api_provider; + const CatBoostWrapperApi * api; + CatBoostModel(const std::string & name, const std::string & model_path, + const std::string & lib_path, const ExternalLoadableLifetime & lifetime); }; } From ff088b4a1da1745ec3e9b63a316f1a964f5059c8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Oct 2017 23:13:44 +0300 Subject: [PATCH 08/32] modified CatBoostModel [#CLICKHOUSE-3305] --- dbms/src/Core/ErrorCodes.cpp | 2 + dbms/src/Dictionaries/CatBoostModel.cpp | 368 +++++++++++++++++++++++- dbms/src/Dictionaries/CatBoostModel.h | 19 +- 3 files changed, 372 insertions(+), 17 deletions(-) diff --git a/dbms/src/Core/ErrorCodes.cpp b/dbms/src/Core/ErrorCodes.cpp index 40e76c82ca1..ffeda42047b 100644 --- a/dbms/src/Core/ErrorCodes.cpp +++ b/dbms/src/Core/ErrorCodes.cpp @@ -384,6 +384,8 @@ namespace ErrorCodes extern const int UNKNOWN_STATUS_OF_DISTRIBUTED_DDL_TASK = 379; extern const int CANNOT_KILL = 380; extern const int HTTP_LENGTH_REQUIRED = 381; + extern const int CANNOT_LOAD_CATBOOST_MODEL = 382; + extern const int CANNOT_APPLY_CATBOOST_MODEL = 383; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index deb2a0e7af8..b79cdd95661 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -1,10 +1,26 @@ #include +#include #include #include +#include +#include +#include +#include +#include +#include +#include namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +extern const int CANNOT_LOAD_CATBOOST_MODEL; +extern const int CANNOT_APPLY_CATBOOST_MODEL; +} + namespace { @@ -39,10 +55,322 @@ struct CatBoostWrapperApi int (* GetIntegerCatFeatureHash)(long long val); }; -class CatBoostWrapperHolder : public CatBoostWrapperApiProvider + +class CatBoostModelHolder +{ +private: + CatBoostWrapperApi::ModelCalcerHandle * handle; + CatBoostWrapperApi * api; +public: + explicit CatBoostModelHolder(CatBoostWrapperApi * api) : api(api) { handle = api->ModelCalcerCreate(); } + ~CatBoostModelHolder() { api->ModelCalcerDelete(handle); } + + CatBoostWrapperApi::ModelCalcerHandle * get() { return handle; } + explicit operator CatBoostWrapperApi::ModelCalcerHandle * () { return handle; } +}; + + +class CatBoostModelImpl : public ICatBoostModel { public: - CatBoostWrapperHolder(const std::string & lib_path) : lib(lib_path), lib_path(lib_path) { initApi(); } + CatBoostModelImpl(CatBoostWrapperApi * api, const std::string & model_path) : api(api) + { + auto handle_ = std::make_unique(api); + if (!handle_) + { + std::string msg = "Cannot create CatBoost model: "; + throw Exception(msg + api->GetErrorString(), ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL); + } + if (!api->LoadFullModelFromFile(handle_.get(), model_path.c_str())) + { + std::string msg = "Cannot load CatBoost model: "; + throw Exception(msg + api->GetErrorString(), ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL); + } + handle = std::move(handle_); + } + + ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) + { + if (columns.empty()) + throw Exception("Got empty columns list for CatBoost model.", ErrorCodes::BAD_ARGUMENTS); + + if (columns.size() != float_features_count + cat_features_count) + { + std::string msg; + { + WriteBufferFromString buffer(msg); + buffer << "Number of columns is different with number of features: "; + buffer << columns.size() << " vs " << float_features_count << " + " << cat_features_count; + } + throw Exception(msg, ErrorCodes::BAD_ARGUMENTS); + } + + for (size_t i = 0; i < float_features_count; ++i) + { + if (!columns[i]->isNumeric()) + { + std::string msg; + { + WriteBufferFromString buffer(msg); + buffer << "Column " << i << "should be numeric to make float feature."; + } + throw Exception(msg, ErrorCodes::BAD_ARGUMENTS); + } + } + + bool cat_features_are_strings = true; + for (size_t i = float_features_count; i < float_features_count + cat_features_count; ++i) + { + const auto & column = columns[i]; + if (column->isNumeric()) + cat_features_are_strings = false; + else if (!(typeid_cast(column.get()) + || typeid_cast(column.get()))) + { + std::string msg; + { + WriteBufferFromString buffer(msg); + buffer << "Column " << i << "should be numeric or string."; + } + throw Exception(msg, ErrorCodes::BAD_ARGUMENTS); + } + } + + return calcImpl(columns, float_features_count, cat_features_count, cat_features_are_strings); + } + +private: + std::unique_ptr handle; + CatBoostWrapperApi * api; + + /// Buffer should be allocated with features_count * column->size() elements. + /// Place column elements in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] + template + void placeColumnAsNumber(const ColumnPtr & column, T * buffer, size_t features_count) + { + size_t size = column->size(); + FieldVisitorConvertToNumber visitor; + for (size_t i = 0; i < size; ++i) + { + /// TODO: Replace with column visitor. + Field field; + column->get(i, field); + *buffer = applyVisitor(visitor, field); + buffer += features_count; + } + } + + /// Buffer should be allocated with features_count * column->size() elements. + /// Place string pointers in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] + void placeStringColumn(const ColumnString & column, const char ** buffer, size_t features_count) + { + size_t size = column.size(); + for (size_t i = 0; i < size; ++i) + { + *buffer = const_cast(column.getDataAtWithTerminatingZero(i).data); + buffer += features_count; + } + } + + /// Buffer should be allocated with features_count * column->size() elements. + /// Place string pointers in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] + /// Returns PODArray which holds data (because ColumnFixedString doesn't store terminating zero). + PODArray placeFixedStringColumn(const ColumnFixedString & column, const char ** buffer, size_t features_count) + { + size_t size = column.size(); + size_t str_size = column.getN(); + PODArray data(size * (str_size + 1)); + char * data_ptr = data.data(); + + for (size_t i = 0; i < size; ++i) + { + auto ref = column.getDataAt(i); + memcpy(data_ptr, ref.data, ref.size); + data_ptr[ref.size] = 0; + *buffer = data_ptr; + data_ptr += ref.size + 1; + buffer += features_count; + } + + return data; + } + + /// Place columns into buffer, returns column which holds placed data. Buffer should contains column->size() values. + template + ColumnPtr placeNumericColumns(const Columns & columns, size_t offset, size_t size, const T** buffer) + { + if (size == 0) + return nullptr; + size_t column_size = columns[offset]->size(); + auto data_column = std::make_shared>(size * column_size); + T* data = data_column->getData().data(); + for (size_t i = offset; i < offset + size; ++i) + { + const auto & column = columns[i]; + if (column->isNumeric()) + placeColumnAsNumber(column, data + i, size); + } + + for (size_t i = 0; i < column_size; ++i) + { + *buffer = data; + ++buffer; + data += size; + } + + return data_column; + } + + /// Place columns into buffer, returns data which was used for fixed string columns. + /// Buffer should contains column->size() values, each value contains size strings. + std::vector> placeStringColumns( + const Columns & columns, size_t offset, size_t size, const char *** buffer) + { + if (size == 0) + return {}; + size_t column_size = columns[offset]->size(); + + std::vector> data; + for (size_t i = offset; i < offset + size; ++i) + { + const auto & column = columns[i]; + if (auto column_string = typeid_cast(column.get())) + placeStringColumn(*column_string, buffer[i], size); + else if (auto column_fixed_string = typeid_cast(column.get())) + data.push_back(placeFixedStringColumn(*column_fixed_string, buffer[i], size)); + else + throw Exception("Cannot place string column.", ErrorCodes::LOGICAL_ERROR); + } + + return data; + } + + /// Calc hash for string cat feature at ps positions. + template + void calcStringHashes(const Column * column, size_t features_count, size_t ps, const int ** buffer) + { + size_t column_size = column->size(); + for (size_t j = 0; j < column_size; ++j) + { + auto ref = column->getDataAt(j); + const_cast(*buffer)[ps] = api->GetStringCatFeatureHash(ref.data, ref.size); + buffer += features_count; + } + } + + /// Calc hash for int cat feature at ps position. Buffer at positions ps should contains unhashed values. + void calcIntHashes(size_t column_size, size_t features_count, size_t ps, const int ** buffer) + { + for (size_t j = 0; j < column_size; ++j) + { + const_cast(*buffer)[ps] = api->GetIntegerCatFeatureHash((*buffer)[ps]); + buffer += features_count; + } + } + + void calcHashes(const Columns & columns, size_t offset, size_t size, const int ** buffer) + { + if (size == 0) + return; + size_t column_size = columns[offset]->size(); + + std::vector> data; + for (size_t i = offset; i < offset + size; ++i) + { + const auto & column = columns[i]; + auto buffer_ptr = buffer; + if (auto column_string = typeid_cast(column.get())) + calcStringHashes(column_string, size, column_size, buffer); + else if (auto column_fixed_string = typeid_cast(column.get())) + calcStringHashes(column_fixed_string, size, column_size, buffer); + else + calcIntHashes(column_size, size, column_size, buffer); + } + } + + void fillCatFeaturesBuffer(const char *** cat_features, const char ** buffer, + size_t column_size, size_t cat_features_count) + { + for (size_t i = 0; i < column_size; ++i) + { + *cat_features = buffer; + ++cat_features; + buffer += cat_features_count; + } + } + + ColumnPtr calcImpl(const Columns & columns, size_t float_features_count, size_t cat_features_count, + bool cat_features_are_strings) + { + // size_t size = columns.size(); + size_t column_size = columns.front()->size(); + + PODArray float_features(column_size); + auto float_features_buf = float_features.data(); + auto float_features_col = placeNumericColumns(columns, 0, float_features_count, float_features_buf); + + auto result= std::make_shared(column_size); + auto result_buf = result->getData().data(); + + std::string error_msg = "Error occurred while applying CatBoost model: "; + + if (cat_features_count == 0) + { + if (!api->CalcModelPredictionFlat(handle.get(), column_size, + float_features_buf, float_features_count, + result_buf, column_size)) + { + + throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); + } + return result; + } + + + if (cat_features_are_strings) + { + PODArray cat_features_holder(cat_features_count * column_size); + PODArray cat_features(column_size); + auto cat_features_buf = cat_features.data(); + + fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count); + auto fixed_strings_data = placeStringColumns(columns, float_features_count, + cat_features_count, cat_features_buf); + + if (!api->CalcModelPrediction(handle.get(), column_size, + float_features_buf, float_features_count, + cat_features_buf, cat_features_count, + result_buf, column_size)) + { + throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); + } + } + else + { + PODArray cat_features(column_size); + auto cat_features_buf = cat_features.data(); + auto cat_features_col = placeNumericColumns(columns, float_features_count, + cat_features_count, cat_features_buf); + calcHashes(columns, float_features_count, cat_features_count, cat_features_buf); + if (!api->CalcModelPredictionWithHashedCatFeatures( + handle.get(), column_size, + float_features_buf, float_features_count, + cat_features_buf, cat_features_count, + result_buf, column_size)) + { + throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); + } + } + + return result; + } +}; + + +class CatBoostLibHolder: public CatBoostWrapperApiProvider +{ +public: + explicit CatBoostLibHolder(const std::string & lib_path) : lib(lib_path), lib_path(lib_path) { initApi(); } const CatBoostWrapperApi & getApi() const override { return api; } const std::string & getCurrentPath() const { return lib_path; } @@ -62,7 +390,7 @@ private: } }; -void CatBoostWrapperHolder::initApi() +void CatBoostLibHolder::initApi() { load(api.ModelCalcerCreate, "ModelCalcerCreate"); load(api.ModelCalcerDelete, "ModelCalcerDelete"); @@ -75,9 +403,9 @@ void CatBoostWrapperHolder::initApi() load(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash"); } -std::shared_ptr getCatBoostWrapperHolder(const std::string & lib_path) +std::shared_ptr getCatBoostWrapperHolder(const std::string & lib_path) { - static std::weak_ptr ptr; + static std::weak_ptr ptr; static std::mutex mutex; std::lock_guard lock(mutex); @@ -85,7 +413,7 @@ std::shared_ptr getCatBoostWrapperHolder(const std::strin if (!result || result->getCurrentPath() != lib_path) { - result = std::make_shared(lib_path); + result = std::make_shared(lib_path); /// This assignment is not atomic, which prevents from creating lock only inside 'if'. ptr = result; } @@ -103,13 +431,14 @@ CatBoostModel::CatBoostModel(const Poco::Util::AbstractConfiguration & config, } CatBoostModel::CatBoostModel(const std::string & name, const std::string & model_path, const std::string & lib_path, - const ExternalLoadableLifetime & lifetime) - : name(name), model_path(model_path), lifetime(lifetime) + const ExternalLoadableLifetime & lifetime, + size_t float_features_count, size_t cat_features_count) + : name(name), model_path(model_path), lib_path(lib_path), lifetime(lifetime), + float_features_count(float_features_count), cat_features_count(cat_features_count) { try { - api_provider = getCatBoostWrapperHolder(lib_path); - api = &api_provider->getApi(); + init(lib_path); } catch (...) { @@ -117,6 +446,13 @@ CatBoostModel::CatBoostModel(const std::string & name, const std::string & model } } +void CatBoostModel::init(const std::string & lib_path) +{ + api_provider = getCatBoostWrapperHolder(lib_path); + api = &api_provider->getApi(); + model = std::make_unique(api, model_path); +} + const ExternalLoadableLifetime & CatBoostModel::getLifetime() const { return lifetime; @@ -129,22 +465,24 @@ bool CatBoostModel::isModified() const std::unique_ptr CatBoostModel::cloneObject() const { - return nullptr; + return std::make_unique(name, model_path, lib_path, lifetime, float_features_count, cat_features_count); } size_t CatBoostModel::getFloatFeaturesCount() const { - return 0; + return float_features_count; } size_t CatBoostModel::getCatFeaturesCount() const { - return 0; + return cat_features_count; } -void CatBoostModel::apply(const Columns & floatColumns, const Columns & catColumns, ColumnFloat64 & result) +ColumnPtr CatBoostModel::apply(const Columns & columns) { - + if (!model) + throw Exception("CatBoost model was not loaded.", ErrorCodes::LOGICAL_ERROR); + return model->calc(columns, float_features_count, cat_features_count); } } diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index adacffab05b..11e7c8c7131 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -15,6 +15,12 @@ public: virtual const CatBoostWrapperApi & getApi() const = 0; }; +class ICatBoostModel +{ +public: + virtual ~ICatBoostModel() = default; + virtual ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) = 0; +}; class CatBoostModel : public IExternalLoadable { @@ -37,18 +43,27 @@ public: size_t getFloatFeaturesCount() const; size_t getCatFeaturesCount() const; - void apply(const Columns & floatColumns, const Columns & catColumns, ColumnFloat64 & result); + ColumnPtr apply(const Columns & columns); private: std::string name; std::string model_path; + std::string lib_path; ExternalLoadableLifetime lifetime; std::exception_ptr creation_exception; std::shared_ptr api_provider; const CatBoostWrapperApi * api; + std::unique_ptr model; + + size_t float_features_count; + size_t cat_features_count; + CatBoostModel(const std::string & name, const std::string & model_path, - const std::string & lib_path, const ExternalLoadableLifetime & lifetime); + const std::string & lib_path, const ExternalLoadableLifetime & lifetime, + size_t float_features_count, size_t cat_features_count); + + void init(const std::string & lib_path); }; } From bb4547140bb6910d452ed20d86979bf7f2e71806 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Oct 2017 13:43:42 +0300 Subject: [PATCH 09/32] added ExternalModels, applyModel [#CLICKHOUSE-3305] --- .../src/Functions/FunctionsExternalModels.cpp | 4 +++ dbms/src/Functions/FunctionsExternalModels.h | 26 +++++++++++++++++++ dbms/src/Interpreters/ExternalModels.cpp | 4 +++ dbms/src/Interpreters/ExternalModels.h | 8 ++++++ 4 files changed, 42 insertions(+) create mode 100644 dbms/src/Functions/FunctionsExternalModels.cpp create mode 100644 dbms/src/Functions/FunctionsExternalModels.h create mode 100644 dbms/src/Interpreters/ExternalModels.cpp create mode 100644 dbms/src/Interpreters/ExternalModels.h diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp new file mode 100644 index 00000000000..24549703943 --- /dev/null +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -0,0 +1,4 @@ +// +// Created by nik-kochetov on 17.10.17. +// + diff --git a/dbms/src/Functions/FunctionsExternalModels.h b/dbms/src/Functions/FunctionsExternalModels.h new file mode 100644 index 00000000000..3388ff3ce81 --- /dev/null +++ b/dbms/src/Functions/FunctionsExternalModels.h @@ -0,0 +1,26 @@ + + +class FunctionModelEvaluate final : public IFunction +{ +public: + static constexpr auto name = "modelEvaluate"; + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getExternalDictionaries()); + } + + FunctionModelEvaluate(const ExternalModles & models) : models(models) {} + + String getName() const override { return name; } + +private: + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override; + + + const ExternalModles & models; +}; diff --git a/dbms/src/Interpreters/ExternalModels.cpp b/dbms/src/Interpreters/ExternalModels.cpp new file mode 100644 index 00000000000..645377d3030 --- /dev/null +++ b/dbms/src/Interpreters/ExternalModels.cpp @@ -0,0 +1,4 @@ +// +// Created by nik-kochetov on 16.10.17. +// + diff --git a/dbms/src/Interpreters/ExternalModels.h b/dbms/src/Interpreters/ExternalModels.h new file mode 100644 index 00000000000..cf17084b524 --- /dev/null +++ b/dbms/src/Interpreters/ExternalModels.h @@ -0,0 +1,8 @@ +// +// Created by nik-kochetov on 16.10.17. +// + +#ifndef CLICKHOUSE_EXTERNALMODELS_H +#define CLICKHOUSE_EXTERNALMODELS_H + +#endif //CLICKHOUSE_EXTERNALMODELS_H From fdf258f8ed8fdebf5d45426063afca87e088556e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Oct 2017 13:44:46 +0300 Subject: [PATCH 10/32] added ExternalModels to Context [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] fix build [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 52 ++++++------ dbms/src/Dictionaries/CatBoostModel.h | 22 ++--- .../src/Functions/FunctionsExternalModels.cpp | 57 ++++++++++++- dbms/src/Functions/FunctionsExternalModels.h | 26 +++--- dbms/src/Interpreters/Context.cpp | 33 ++++++++ dbms/src/Interpreters/Context.h | 5 ++ dbms/src/Interpreters/ExternalModels.cpp | 81 ++++++++++++++++++- dbms/src/Interpreters/ExternalModels.h | 45 +++++++++-- 8 files changed, 261 insertions(+), 60 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index b79cdd95661..83ae29bd046 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -21,8 +21,6 @@ extern const int CANNOT_LOAD_CATBOOST_MODEL; extern const int CANNOT_APPLY_CATBOOST_MODEL; } -namespace -{ struct CatBoostWrapperApi { @@ -56,13 +54,16 @@ struct CatBoostWrapperApi }; +namespace +{ + class CatBoostModelHolder { private: CatBoostWrapperApi::ModelCalcerHandle * handle; - CatBoostWrapperApi * api; + const CatBoostWrapperApi * api; public: - explicit CatBoostModelHolder(CatBoostWrapperApi * api) : api(api) { handle = api->ModelCalcerCreate(); } + explicit CatBoostModelHolder(const CatBoostWrapperApi * api) : api(api) { handle = api->ModelCalcerCreate(); } ~CatBoostModelHolder() { api->ModelCalcerDelete(handle); } CatBoostWrapperApi::ModelCalcerHandle * get() { return handle; } @@ -73,7 +74,7 @@ public: class CatBoostModelImpl : public ICatBoostModel { public: - CatBoostModelImpl(CatBoostWrapperApi * api, const std::string & model_path) : api(api) + CatBoostModelImpl(const CatBoostWrapperApi * api, const std::string & model_path) : api(api) { auto handle_ = std::make_unique(api); if (!handle_) @@ -89,7 +90,7 @@ public: handle = std::move(handle_); } - ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) + ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) const override { if (columns.empty()) throw Exception("Got empty columns list for CatBoost model.", ErrorCodes::BAD_ARGUMENTS); @@ -141,12 +142,12 @@ public: private: std::unique_ptr handle; - CatBoostWrapperApi * api; + const CatBoostWrapperApi * api; /// Buffer should be allocated with features_count * column->size() elements. /// Place column elements in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] template - void placeColumnAsNumber(const ColumnPtr & column, T * buffer, size_t features_count) + void placeColumnAsNumber(const ColumnPtr & column, T * buffer, size_t features_count) const { size_t size = column->size(); FieldVisitorConvertToNumber visitor; @@ -162,7 +163,7 @@ private: /// Buffer should be allocated with features_count * column->size() elements. /// Place string pointers in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] - void placeStringColumn(const ColumnString & column, const char ** buffer, size_t features_count) + void placeStringColumn(const ColumnString & column, const char ** buffer, size_t features_count) const { size_t size = column.size(); for (size_t i = 0; i < size; ++i) @@ -175,7 +176,8 @@ private: /// Buffer should be allocated with features_count * column->size() elements. /// Place string pointers in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] /// Returns PODArray which holds data (because ColumnFixedString doesn't store terminating zero). - PODArray placeFixedStringColumn(const ColumnFixedString & column, const char ** buffer, size_t features_count) + PODArray placeFixedStringColumn( + const ColumnFixedString & column, const char ** buffer, size_t features_count) const { size_t size = column.size(); size_t str_size = column.getN(); @@ -197,12 +199,12 @@ private: /// Place columns into buffer, returns column which holds placed data. Buffer should contains column->size() values. template - ColumnPtr placeNumericColumns(const Columns & columns, size_t offset, size_t size, const T** buffer) + ColumnPtr placeNumericColumns(const Columns & columns, size_t offset, size_t size, const T** buffer) const { if (size == 0) return nullptr; size_t column_size = columns[offset]->size(); - auto data_column = std::make_shared>(size * column_size); + auto data_column = std::make_shared>(size * column_size); T* data = data_column->getData().data(); for (size_t i = offset; i < offset + size; ++i) { @@ -224,11 +226,10 @@ private: /// Place columns into buffer, returns data which was used for fixed string columns. /// Buffer should contains column->size() values, each value contains size strings. std::vector> placeStringColumns( - const Columns & columns, size_t offset, size_t size, const char *** buffer) + const Columns & columns, size_t offset, size_t size, const char *** buffer) const { if (size == 0) return {}; - size_t column_size = columns[offset]->size(); std::vector> data; for (size_t i = offset; i < offset + size; ++i) @@ -247,7 +248,7 @@ private: /// Calc hash for string cat feature at ps positions. template - void calcStringHashes(const Column * column, size_t features_count, size_t ps, const int ** buffer) + void calcStringHashes(const Column * column, size_t features_count, size_t ps, const int ** buffer) const { size_t column_size = column->size(); for (size_t j = 0; j < column_size; ++j) @@ -259,7 +260,7 @@ private: } /// Calc hash for int cat feature at ps position. Buffer at positions ps should contains unhashed values. - void calcIntHashes(size_t column_size, size_t features_count, size_t ps, const int ** buffer) + void calcIntHashes(size_t column_size, size_t features_count, size_t ps, const int ** buffer) const { for (size_t j = 0; j < column_size; ++j) { @@ -268,7 +269,7 @@ private: } } - void calcHashes(const Columns & columns, size_t offset, size_t size, const int ** buffer) + void calcHashes(const Columns & columns, size_t offset, size_t size, const int ** buffer) const { if (size == 0) return; @@ -278,7 +279,6 @@ private: for (size_t i = offset; i < offset + size; ++i) { const auto & column = columns[i]; - auto buffer_ptr = buffer; if (auto column_string = typeid_cast(column.get())) calcStringHashes(column_string, size, column_size, buffer); else if (auto column_fixed_string = typeid_cast(column.get())) @@ -289,7 +289,7 @@ private: } void fillCatFeaturesBuffer(const char *** cat_features, const char ** buffer, - size_t column_size, size_t cat_features_count) + size_t column_size, size_t cat_features_count) const { for (size_t i = 0; i < column_size; ++i) { @@ -300,7 +300,7 @@ private: } ColumnPtr calcImpl(const Columns & columns, size_t float_features_count, size_t cat_features_count, - bool cat_features_are_strings) + bool cat_features_are_strings) const { // size_t size = columns.size(); size_t column_size = columns.front()->size(); @@ -370,7 +370,7 @@ private: class CatBoostLibHolder: public CatBoostWrapperApiProvider { public: - explicit CatBoostLibHolder(const std::string & lib_path) : lib(lib_path), lib_path(lib_path) { initApi(); } + explicit CatBoostLibHolder(const std::string & lib_path) : lib_path(lib_path), lib(lib_path) { initApi(); } const CatBoostWrapperApi & getApi() const override { return api; } const std::string & getCurrentPath() const { return lib_path; } @@ -385,7 +385,7 @@ private: template void load(T& func, const std::string & name) { - using Type = std::remove_pointer::type; + using Type = typename std::remove_pointer::type; func = lib.get(name); } }; @@ -423,12 +423,6 @@ std::shared_ptr getCatBoostWrapperHolder(const std::string & } -CatBoostModel::CatBoostModel(const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, const std::string & lib_path) - : lifetime(config, config_prefix) -{ - -} CatBoostModel::CatBoostModel(const std::string & name, const std::string & model_path, const std::string & lib_path, const ExternalLoadableLifetime & lifetime, @@ -478,7 +472,7 @@ size_t CatBoostModel::getCatFeaturesCount() const return cat_features_count; } -ColumnPtr CatBoostModel::apply(const Columns & columns) +ColumnPtr CatBoostModel::evaluate(const Columns & columns) const { if (!model) throw Exception("CatBoost model was not loaded.", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index 11e7c8c7131..82473662e57 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -19,14 +19,22 @@ class ICatBoostModel { public: virtual ~ICatBoostModel() = default; - virtual ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) = 0; + virtual ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) const = 0; }; -class CatBoostModel : public IExternalLoadable +class IModel : public IExternalLoadable { public: - CatBoostModel(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const std::string & lib_path); + virtual ColumnPtr evaluate(const Columns & columns) const = 0; +}; + +class CatBoostModel : public IModel +{ +public: + CatBoostModel(const std::string & name, const std::string & model_path, + const std::string & lib_path, const ExternalLoadableLifetime & lifetime, + size_t float_features_count, size_t cat_features_count); + const ExternalLoadableLifetime & getLifetime() const override; @@ -43,7 +51,7 @@ public: size_t getFloatFeaturesCount() const; size_t getCatFeaturesCount() const; - ColumnPtr apply(const Columns & columns); + ColumnPtr evaluate(const Columns & columns) const override; private: std::string name; @@ -59,10 +67,6 @@ private: size_t float_features_count; size_t cat_features_count; - CatBoostModel(const std::string & name, const std::string & model_path, - const std::string & lib_path, const ExternalLoadableLifetime & lifetime, - size_t float_features_count, size_t cat_features_count); - void init(const std::string & lib_path); }; diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index 24549703943..aebe6fd7751 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -1,4 +1,55 @@ -// -// Created by nik-kochetov on 17.10.17. -// +#include +#include +#include +#include +#include +#include +#include +#include +namespace DB +{ + +FunctionPtr FunctionModelEvaluate::create(const Context & context) +{ + return std::make_shared(context.getExternalModels()); +} + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION; + extern const int ILLEGAL_COLUMN; +} + +DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const DataTypes & arguments) const +{ + if (arguments.size() < 2) + throw Exception("Function " + getName() + " expects at least 2 arguments", + ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION); + + if (!checkDataType(arguments[0].get())) + throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + + return std::make_shared(); +} + +void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) +{ + const auto name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); + if (!name_col) + throw Exception("First argument of function " + getName() + " must be a constant string", + ErrorCodes::ILLEGAL_COLUMN); + + auto model = models.getModel(name_col->getValue()); + + Columns columns(arguments.size() - 1); + for (auto i : ext::range(0, columns.size())) + columns[i] = block.getByPosition(arguments[i + 1]).column; + + block.getByPosition(result).column = model->evaluate(columns); +} + +} diff --git a/dbms/src/Functions/FunctionsExternalModels.h b/dbms/src/Functions/FunctionsExternalModels.h index 3388ff3ce81..79b403a779e 100644 --- a/dbms/src/Functions/FunctionsExternalModels.h +++ b/dbms/src/Functions/FunctionsExternalModels.h @@ -1,26 +1,32 @@ +#pragma once +#include +namespace DB +{ + +class ExternalModels; class FunctionModelEvaluate final : public IFunction { public: static constexpr auto name = "modelEvaluate"; - static FunctionPtr create(const Context & context) - { - return std::make_shared(context.getExternalDictionaries()); - } + static FunctionPtr create(const Context & context); - FunctionModelEvaluate(const ExternalModles & models) : models(models) {} + explicit FunctionModelEvaluate(const ExternalModels & models) : models(models) {} String getName() const override { return name; } -private: - size_t getNumberOfArguments() const override { return 2; } + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override; + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; - - const ExternalModles & models; +private: + const ExternalModels & models; }; + +} diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 1e0023a1b3e..af01fe02508 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -94,6 +95,7 @@ struct ContextShared /// Separate mutex for access of dictionaries. Separate mutex to avoid locks when server doing request to itself. mutable std::mutex embedded_dictionaries_mutex; mutable std::mutex external_dictionaries_mutex; + mutable std::mutex external_models_mutex; /// Separate mutex for re-initialization of zookeer session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; @@ -111,6 +113,7 @@ struct ContextShared FormatFactory format_factory; /// Formats. mutable std::shared_ptr embedded_dictionaries; /// Metrica's dictionaeis. Have lazy initialization. mutable std::shared_ptr external_dictionaries; + mutable std::shared_ptr external_models; String default_profile_name; /// Default profile name used for default values. Users users; /// Known users. Quotas quotas; /// Known quotas for resource use. @@ -1044,6 +1047,17 @@ ExternalDictionaries & Context::getExternalDictionaries() } +const ExternalModels & Context::getExternalModels() const +{ + return getExternalModelsImpl(false); +} + +ExternalModels & Context::getExternalModels() +{ + return getExternalModelsImpl(false); +} + + EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const { std::lock_guard lock(shared->embedded_dictionaries_mutex); @@ -1069,6 +1083,19 @@ ExternalDictionaries & Context::getExternalDictionariesImpl(const bool throw_on_ return *shared->external_dictionaries; } +ExternalModels & Context::getExternalModelsImpl(bool throw_on_error) const +{ + std::lock_guard lock(shared->external_models_mutex); + + if (!shared->external_models) + { + if (!this->global_context) + throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); + shared->external_models = std::make_shared(*this->global_context, throw_on_error); + } + + return *shared->external_models; +} void Context::tryCreateEmbeddedDictionaries() const { @@ -1082,6 +1109,12 @@ void Context::tryCreateExternalDictionaries() const } +void Context::tryCreateExternalModels() const +{ + static_cast(getExternalModelsImpl(true)); +} + + void Context::setProgressCallback(ProgressCallback callback) { /// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed. diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 8795c5a9799..aee320f7766 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -35,6 +35,7 @@ struct ContextShared; class QuotaForIntervals; class EmbeddedDictionaries; class ExternalDictionaries; +class ExternalModels; class InterserverIOHandler; class BackgroundProcessingPool; class ReshardingWorker; @@ -209,10 +210,13 @@ public: const EmbeddedDictionaries & getEmbeddedDictionaries() const; const ExternalDictionaries & getExternalDictionaries() const; + const ExternalModels & getExternalModels() const; EmbeddedDictionaries & getEmbeddedDictionaries(); ExternalDictionaries & getExternalDictionaries(); + ExternalModels & getExternalModels(); void tryCreateEmbeddedDictionaries() const; void tryCreateExternalDictionaries() const; + void tryCreateExternalModels() const; /// I/O formats. BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, size_t max_block_size) const; @@ -362,6 +366,7 @@ private: EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const; ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const; + ExternalModels & getExternalModelsImpl(bool throw_on_error) const; StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const; diff --git a/dbms/src/Interpreters/ExternalModels.cpp b/dbms/src/Interpreters/ExternalModels.cpp index 645377d3030..8f3d6cc98d0 100644 --- a/dbms/src/Interpreters/ExternalModels.cpp +++ b/dbms/src/Interpreters/ExternalModels.cpp @@ -1,4 +1,79 @@ -// -// Created by nik-kochetov on 16.10.17. -// +#include +#include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_CONFIG_PARAMETER; +} + +namespace +{ +const ExternalLoaderUpdateSettings & getExternalModelsUpdateSettings() +{ + static ExternalLoaderUpdateSettings settings; + static std::once_flag flag; + + std::call_once(flag, [] { + settings.check_period_sec = 5; + settings.backoff_initial_sec = 5; + /// 10 minutes + settings.backoff_max_sec = 10 * 60; + }); + + return settings; +} + +const ExternalLoaderConfigSettings & getExternalModelsConfigSettings() +{ + static ExternalLoaderConfigSettings settings; + static std::once_flag flag; + + std::call_once(flag, [] { + settings.external_config = "model"; + settings.external_name = "name"; + + settings.path_setting_name = "models_config"; + }); + + return settings; +} +} + + +ExternalModels::ExternalModels(Context & context, bool throw_on_error) + : ExternalLoader(context.getConfigRef(), + getExternalModelsUpdateSettings(), + getExternalModelsConfigSettings(), + &Logger::get("ExternalModels"), + "external model"), + context(context) +{ + init(throw_on_error); +} + +std::unique_ptr ExternalModels::create( + const std::string & name, const Configuration & config, const std::string & config_prefix) +{ + String type = config.getString(config_prefix + ".type"); + ExternalLoadableLifetime lifetime(config, config_prefix); + + /// TODO: add models factory. + if (type == "catboost") + { + return std::make_unique( + name, config.getString(config_prefix + ".path"), + context.getConfigRef().getString("catboost_dynamic_library_path"), + lifetime, config.getUInt(config_prefix + ".float_features_count"), + config.getUInt(config_prefix + ".cat_features_count") + ); + } + else + { + throw Exception("Unknown model type: " + type, ErrorCodes::INVALID_CONFIG_PARAMETER); + } +} + +} diff --git a/dbms/src/Interpreters/ExternalModels.h b/dbms/src/Interpreters/ExternalModels.h index cf17084b524..889cd80e8e1 100644 --- a/dbms/src/Interpreters/ExternalModels.h +++ b/dbms/src/Interpreters/ExternalModels.h @@ -1,8 +1,41 @@ -// -// Created by nik-kochetov on 16.10.17. -// +#pragma once -#ifndef CLICKHOUSE_EXTERNALMODELS_H -#define CLICKHOUSE_EXTERNALMODELS_H +#include +#include +#include +#include -#endif //CLICKHOUSE_EXTERNALMODELS_H + +namespace DB +{ + +class Context; + +/// Manages user-defined dictionaries. +class ExternalModels : public ExternalLoader +{ +public: + using ModelPtr = std::shared_ptr; + + /// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. + ExternalModels(Context & context, bool throw_on_error); + + /// Forcibly reloads specified model. + void reloadModel(const std::string & name) { reload(name); } + + ModelPtr getModel(const std::string & name) const + { + return std::static_pointer_cast(getLoadable(name)); + } + +protected: + + std::unique_ptr create(const std::string & name, const Configuration & config, + const std::string & config_prefix) override; + +private: + + Context & context; +}; + +} From 48de8de6e0e6d9e25620676271c345dcaa25bf76 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Oct 2017 15:02:30 +0300 Subject: [PATCH 11/32] added registerFunctionsExternalModels [#CLICKHOUSE-3305] --- dbms/src/Functions/FunctionsExternalModels.cpp | 13 ++++++++++--- dbms/src/Functions/FunctionsExternalModels.h | 4 ++++ dbms/src/Functions/registerFunctions.cpp | 2 ++ 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index aebe6fd7751..9617bd07c78 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -1,11 +1,13 @@ +#include +#include +#include + #include +#include #include #include #include #include -#include -#include -#include namespace DB { @@ -52,4 +54,9 @@ void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arg block.getByPosition(result).column = model->evaluate(columns); } +void registerFunctionsExternalModels(FunctionFactory & factory) +{ + factory.registerFunction(); +} + } diff --git a/dbms/src/Functions/FunctionsExternalModels.h b/dbms/src/Functions/FunctionsExternalModels.h index 79b403a779e..74149920e6f 100644 --- a/dbms/src/Functions/FunctionsExternalModels.h +++ b/dbms/src/Functions/FunctionsExternalModels.h @@ -6,6 +6,10 @@ namespace DB class ExternalModels; +/// Evaluate external model. +/// First argument - model name, the others - model arguments. +/// * for CatBoost model - float features first, then categorical +/// Result - Float64. class FunctionModelEvaluate final : public IFunction { public: diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index eb40e486646..b05a473879c 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -20,6 +20,7 @@ void registerFunctionsConversion(FunctionFactory &); void registerFunctionsDateTime(FunctionFactory &); void registerFunctionsEmbeddedDictionaries(FunctionFactory &); void registerFunctionsExternalDictionaries(FunctionFactory &); +void registerFunctionsExternalModels(FunctionFactory &); void registerFunctionsFormatting(FunctionFactory &); void registerFunctionsHashing(FunctionFactory &); void registerFunctionsHigherOrder(FunctionFactory &); @@ -54,6 +55,7 @@ void registerFunctions() registerFunctionsDateTime(factory); registerFunctionsEmbeddedDictionaries(factory); registerFunctionsExternalDictionaries(factory); + registerFunctionsExternalModels(factory); registerFunctionsFormatting(factory); registerFunctionsHashing(factory); registerFunctionsHigherOrder(factory); From ce602387d433136b6967566c7bb1fca13ccedb55 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Oct 2017 18:38:36 +0300 Subject: [PATCH 12/32] fixed extermal models lifetime in config [#CLICKHOUSE-3305] --- dbms/src/Interpreters/ExternalModels.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ExternalModels.cpp b/dbms/src/Interpreters/ExternalModels.cpp index 8f3d6cc98d0..1a848fa4353 100644 --- a/dbms/src/Interpreters/ExternalModels.cpp +++ b/dbms/src/Interpreters/ExternalModels.cpp @@ -58,7 +58,7 @@ std::unique_ptr ExternalModels::create( const std::string & name, const Configuration & config, const std::string & config_prefix) { String type = config.getString(config_prefix + ".type"); - ExternalLoadableLifetime lifetime(config, config_prefix); + ExternalLoadableLifetime lifetime(config, config_prefix + ".lifetime"); /// TODO: add models factory. if (type == "catboost") From 9384bfdc2888867b4d9d2166f60a2d919acd54c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Oct 2017 12:29:00 +0300 Subject: [PATCH 13/32] fix invalid catboost model handle [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 83ae29bd046..412cdc1a923 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -82,7 +82,7 @@ public: std::string msg = "Cannot create CatBoost model: "; throw Exception(msg + api->GetErrorString(), ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL); } - if (!api->LoadFullModelFromFile(handle_.get(), model_path.c_str())) + if (!api->LoadFullModelFromFile(handle_->get(), model_path.c_str())) { std::string msg = "Cannot load CatBoost model: "; throw Exception(msg + api->GetErrorString(), ErrorCodes::CANNOT_LOAD_CATBOOST_MODEL); @@ -316,7 +316,7 @@ private: if (cat_features_count == 0) { - if (!api->CalcModelPredictionFlat(handle.get(), column_size, + if (!api->CalcModelPredictionFlat(handle->get(), column_size, float_features_buf, float_features_count, result_buf, column_size)) { @@ -337,7 +337,7 @@ private: auto fixed_strings_data = placeStringColumns(columns, float_features_count, cat_features_count, cat_features_buf); - if (!api->CalcModelPrediction(handle.get(), column_size, + if (!api->CalcModelPrediction(handle->get(), column_size, float_features_buf, float_features_count, cat_features_buf, cat_features_count, result_buf, column_size)) @@ -353,7 +353,7 @@ private: cat_features_count, cat_features_buf); calcHashes(columns, float_features_count, cat_features_count, cat_features_buf); if (!api->CalcModelPredictionWithHashedCatFeatures( - handle.get(), column_size, + handle->get(), column_size, float_features_buf, float_features_count, cat_features_buf, cat_features_count, result_buf, column_size)) From e23e15dba8a4c52b5d9ffba7cafddd25e277db9a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Oct 2017 12:52:36 +0300 Subject: [PATCH 14/32] fix arguments count in FunctionModelEvaluate [#CLICKHOUSE-3305] --- dbms/src/Functions/FunctionsExternalModels.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index 9617bd07c78..f6843a178a8 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -47,9 +47,10 @@ void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arg auto model = models.getModel(name_col->getValue()); - Columns columns(arguments.size() - 1); - for (auto i : ext::range(0, columns.size())) - columns[i] = block.getByPosition(arguments[i + 1]).column; + Columns columns; + columns.reserve(arguments.size()); + for (auto i : ext::range(1, arguments.size())) + columns.push_back(block.getByPosition(arguments[i]).column); block.getByPosition(result).column = model->evaluate(columns); } From 4efee711149f863f01d66fbfc85a4aaf13552608 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Oct 2017 13:05:58 +0300 Subject: [PATCH 15/32] fix hash index in CatBoostModel [#CLICKHOUSE-3305] fix hash index in CatBoostModel [#CLICKHOUSE-3305] fix hash index in CatBoostModel [#CLICKHOUSE-3305] fix hash index in CatBoostModel [#CLICKHOUSE-3305] fix string features in CatBoostModel [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 38 ++++++++++++++----------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 412cdc1a923..888bb710673 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -206,9 +206,9 @@ private: size_t column_size = columns[offset]->size(); auto data_column = std::make_shared>(size * column_size); T* data = data_column->getData().data(); - for (size_t i = offset; i < offset + size; ++i) + for (size_t i = 0; i < size; ++i) { - const auto & column = columns[i]; + const auto & column = columns[offset + i]; if (column->isNumeric()) placeColumnAsNumber(column, data + i, size); } @@ -226,19 +226,19 @@ private: /// Place columns into buffer, returns data which was used for fixed string columns. /// Buffer should contains column->size() values, each value contains size strings. std::vector> placeStringColumns( - const Columns & columns, size_t offset, size_t size, const char *** buffer) const + const Columns & columns, size_t offset, size_t size, const char ** buffer) const { if (size == 0) return {}; std::vector> data; - for (size_t i = offset; i < offset + size; ++i) + for (size_t i = 0; i < size; ++i) { - const auto & column = columns[i]; + const auto & column = columns[offset + i]; if (auto column_string = typeid_cast(column.get())) - placeStringColumn(*column_string, buffer[i], size); + placeStringColumn(*column_string, buffer + i, size); else if (auto column_fixed_string = typeid_cast(column.get())) - data.push_back(placeFixedStringColumn(*column_fixed_string, buffer[i], size)); + data.push_back(placeFixedStringColumn(*column_fixed_string, buffer + i, size)); else throw Exception("Cannot place string column.", ErrorCodes::LOGICAL_ERROR); } @@ -248,27 +248,30 @@ private: /// Calc hash for string cat feature at ps positions. template - void calcStringHashes(const Column * column, size_t features_count, size_t ps, const int ** buffer) const + void calcStringHashes(const Column * column, size_t ps, const int ** buffer) const { size_t column_size = column->size(); for (size_t j = 0; j < column_size; ++j) { auto ref = column->getDataAt(j); const_cast(*buffer)[ps] = api->GetStringCatFeatureHash(ref.data, ref.size); - buffer += features_count; + ++buffer; } } /// Calc hash for int cat feature at ps position. Buffer at positions ps should contains unhashed values. - void calcIntHashes(size_t column_size, size_t features_count, size_t ps, const int ** buffer) const + void calcIntHashes(size_t column_size, size_t ps, const int ** buffer) const { for (size_t j = 0; j < column_size; ++j) { const_cast(*buffer)[ps] = api->GetIntegerCatFeatureHash((*buffer)[ps]); - buffer += features_count; + ++buffer; } } + /// buffer contains column->size() rows and size columns. + /// For int cat features calc hash inplace. + /// For string cat features calc hash from column rows. void calcHashes(const Columns & columns, size_t offset, size_t size, const int ** buffer) const { if (size == 0) @@ -276,18 +279,19 @@ private: size_t column_size = columns[offset]->size(); std::vector> data; - for (size_t i = offset; i < offset + size; ++i) + for (size_t i = 0; i < size; ++i) { - const auto & column = columns[i]; + const auto & column = columns[offset + i]; if (auto column_string = typeid_cast(column.get())) - calcStringHashes(column_string, size, column_size, buffer); + calcStringHashes(column_string, i, buffer); else if (auto column_fixed_string = typeid_cast(column.get())) - calcStringHashes(column_fixed_string, size, column_size, buffer); + calcStringHashes(column_fixed_string, i, buffer); else - calcIntHashes(column_size, size, column_size, buffer); + calcIntHashes(column_size, i, buffer); } } + /// buffer[column_size * cat_features_count] -> char * => cat_features[column_size][cat_features_count] -> char * void fillCatFeaturesBuffer(const char *** cat_features, const char ** buffer, size_t column_size, size_t cat_features_count) const { @@ -335,7 +339,7 @@ private: fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count); auto fixed_strings_data = placeStringColumns(columns, float_features_count, - cat_features_count, cat_features_buf); + cat_features_count, cat_features_holder.data()); if (!api->CalcModelPrediction(handle->get(), column_size, float_features_buf, float_features_count, From d5d53da01bc843e2e660631051b57649fbef5aaf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Oct 2017 18:12:34 +0300 Subject: [PATCH 16/32] added comments [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 30 ++++++++++++++++--------- dbms/src/Dictionaries/CatBoostModel.h | 18 ++++++++++----- 2 files changed, 31 insertions(+), 17 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 888bb710673..cf0e4a14609 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -22,6 +22,7 @@ extern const int CANNOT_APPLY_CATBOOST_MODEL; } +/// CatBoost wrapper interface functions. struct CatBoostWrapperApi { typedef void ModelCalcerHandle; @@ -90,7 +91,7 @@ public: handle = std::move(handle_); } - ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) const override + ColumnPtr eval(const Columns & columns, size_t float_features_count, size_t cat_features_count) const override { if (columns.empty()) throw Exception("Got empty columns list for CatBoost model.", ErrorCodes::BAD_ARGUMENTS); @@ -137,7 +138,7 @@ public: } } - return calcImpl(columns, float_features_count, cat_features_count, cat_features_are_strings); + return evalImpl(columns, float_features_count, cat_features_count, cat_features_are_strings); } private: @@ -303,20 +304,24 @@ private: } } - ColumnPtr calcImpl(const Columns & columns, size_t float_features_count, size_t cat_features_count, + /// Convert values to row-oriented format and call evaluation function from CatBoost wrapper api. + /// * CalcModelPredictionFlat if no cat features + /// * CalcModelPrediction if all cat features are strings + /// * CalcModelPredictionWithHashedCatFeatures if has int cat features. + ColumnPtr evalImpl(const Columns & columns, size_t float_features_count, size_t cat_features_count, bool cat_features_are_strings) const { - // size_t size = columns.size(); + std::string error_msg = "Error occurred while applying CatBoost model: "; size_t column_size = columns.front()->size(); - PODArray float_features(column_size); - auto float_features_buf = float_features.data(); - auto float_features_col = placeNumericColumns(columns, 0, float_features_count, float_features_buf); - auto result= std::make_shared(column_size); auto result_buf = result->getData().data(); - std::string error_msg = "Error occurred while applying CatBoost model: "; + /// Prepare float features. + PODArray float_features(column_size); + auto float_features_buf = float_features.data(); + /// Store all float data into single column. float_features is a list of pointers to it. + auto float_features_col = placeNumericColumns(columns, 0, float_features_count, float_features_buf); if (cat_features_count == 0) { @@ -330,14 +335,16 @@ private: return result; } - + /// Prepare cat features. if (cat_features_are_strings) { + /// cat_features_holder stores pointers to ColumnString data or fixed_strings_data. PODArray cat_features_holder(cat_features_count * column_size); PODArray cat_features(column_size); auto cat_features_buf = cat_features.data(); fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count); + /// Fixed strings are stored without termination zero, so have to copy data into fixed_strings_data auto fixed_strings_data = placeStringColumns(columns, float_features_count, cat_features_count, cat_features_holder.data()); @@ -371,6 +378,7 @@ private: }; +/// Holds CatBoost wrapper library and provides wrapper interface. class CatBoostLibHolder: public CatBoostWrapperApiProvider { public: @@ -480,7 +488,7 @@ ColumnPtr CatBoostModel::evaluate(const Columns & columns) const { if (!model) throw Exception("CatBoost model was not loaded.", ErrorCodes::LOGICAL_ERROR); - return model->calc(columns, float_features_count, cat_features_count); + return model->eval(columns, float_features_count, cat_features_count); } } diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index 82473662e57..884f367f25b 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -7,6 +7,7 @@ namespace DB { +/// CatBoost wrapper interface functions. struct CatBoostWrapperApi; class CatBoostWrapperApiProvider { @@ -15,13 +16,17 @@ public: virtual const CatBoostWrapperApi & getApi() const = 0; }; +/// CatBoost model interface. class ICatBoostModel { public: virtual ~ICatBoostModel() = default; - virtual ColumnPtr calc(const Columns & columns, size_t float_features_count, size_t cat_features_count) const = 0; + /// Evaluate model. Use first `float_features_count` columns as float features, + /// the others `cat_features_count` as categorical features. + virtual ColumnPtr eval(const Columns & columns, size_t float_features_count, size_t cat_features_count) const = 0; }; +/// General ML model evaluator interface. class IModel : public IExternalLoadable { public: @@ -35,6 +40,12 @@ public: const std::string & lib_path, const ExternalLoadableLifetime & lifetime, size_t float_features_count, size_t cat_features_count); + ColumnPtr evaluate(const Columns & columns) const override; + + size_t getFloatFeaturesCount() const; + size_t getCatFeaturesCount() const; + + /// IExternalLoadable interface. const ExternalLoadableLifetime & getLifetime() const override; @@ -48,11 +59,6 @@ public: std::exception_ptr getCreationException() const override { return creation_exception; } - size_t getFloatFeaturesCount() const; - size_t getCatFeaturesCount() const; - - ColumnPtr evaluate(const Columns & columns) const override; - private: std::string name; std::string model_path; From 54786902c3bba98a6b3b47faabbdb21da9f9b910 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 25 Oct 2017 15:09:25 +0300 Subject: [PATCH 17/32] added test [#CLICKHOUSE-3305] added test [#CLICKHOUSE-3305] added test [#CLICKHOUSE-3305] added test [#CLICKHOUSE-3305] added test [#CLICKHOUSE-3305] added test [#CLICKHOUSE-3305] added test [#CLICKHOUSE-3305] added test [#CLICKHOUSE-3305] --- .../catboost/data/build_catboost.sh | 17 ++ .../catboost/helpers/__init__.py | 0 .../catboost/helpers/client.py | 42 ++++ .../catboost/helpers/generate.py | 15 ++ .../catboost/helpers/server.py | 67 +++++ .../catboost/helpers/server_with_models.py | 166 ++++++++++++ .../external_models/catboost/helpers/table.py | 69 +++++ .../external_models/catboost/helpers/train.py | 28 +++ .../tests/external_models/catboost/pytest.ini | 3 + .../test_apply_catboost_model/test.py | 236 ++++++++++++++++++ 10 files changed, 643 insertions(+) create mode 100755 dbms/tests/external_models/catboost/data/build_catboost.sh create mode 100644 dbms/tests/external_models/catboost/helpers/__init__.py create mode 100644 dbms/tests/external_models/catboost/helpers/client.py create mode 100644 dbms/tests/external_models/catboost/helpers/generate.py create mode 100644 dbms/tests/external_models/catboost/helpers/server.py create mode 100644 dbms/tests/external_models/catboost/helpers/server_with_models.py create mode 100644 dbms/tests/external_models/catboost/helpers/table.py create mode 100644 dbms/tests/external_models/catboost/helpers/train.py create mode 100644 dbms/tests/external_models/catboost/pytest.ini create mode 100644 dbms/tests/external_models/catboost/test_apply_catboost_model/test.py diff --git a/dbms/tests/external_models/catboost/data/build_catboost.sh b/dbms/tests/external_models/catboost/data/build_catboost.sh new file mode 100755 index 00000000000..080c5bcb245 --- /dev/null +++ b/dbms/tests/external_models/catboost/data/build_catboost.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" + +cd $DIR +git clone https://github.com/catboost/catboost.git + + +cd "${DIR}/catboost/catboost/libs/model_interface" +../../../ya make -r -o "${DIR}/build/lib" -j4 +cd $DIR +ln -sf "${DIR}/build/lib/catboost/libs/model_interface/libcatboostmodel.so" libcatboostmodel.so + +cd "${DIR}/catboost/catboost/python-package/catboost" +../../../ya make -r -DUSE_ARCADIA_PYTHON=no -DPYTHON_CONFIG=python2-config -j4 +cd $DIR +ln -sf "${DIR}/catboost/catboost/python-package" python-package diff --git a/dbms/tests/external_models/catboost/helpers/__init__.py b/dbms/tests/external_models/catboost/helpers/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/external_models/catboost/helpers/client.py b/dbms/tests/external_models/catboost/helpers/client.py new file mode 100644 index 00000000000..ecf44a0b65c --- /dev/null +++ b/dbms/tests/external_models/catboost/helpers/client.py @@ -0,0 +1,42 @@ +import subprocess +import threading +import os + + +class ClickHouseClient: + def __init__(self, binary_path, port): + self.binary_path = binary_path + self.port = port + + def query(self, query, timeout=10, pipe=None): + + result = [] + process = [] + + def run(path, port, text, result, in_pipe, process): + + if in_pipe is None: + in_pipe = subprocess.PIPE + + pipe = subprocess.Popen([path, 'client', '--port', str(port), '-q', text], + stdin=in_pipe, stdout=subprocess.PIPE, stderr=subprocess.PIPE, close_fds=True) + process.append(pipe) + stdout_data, stderr_data = pipe.communicate() + + if stderr_data: + raise Exception('Error while executing query: {}\nstdout:\n{}\nstderr:\n{}' + .format(text, stdout_data, stderr_data)) + + result.append(stdout_data) + + thread = threading.Thread(target=run, args=(self.binary_path, self.port, query, result, pipe, process)) + thread.start() + thread.join(timeout) + if thread.isAlive(): + if len(process): + process[0].kill() + thread.join() + raise Exception('timeout exceed for query: ' + query) + + if len(result): + return result[0] diff --git a/dbms/tests/external_models/catboost/helpers/generate.py b/dbms/tests/external_models/catboost/helpers/generate.py new file mode 100644 index 00000000000..a7c1f3d9e98 --- /dev/null +++ b/dbms/tests/external_models/catboost/helpers/generate.py @@ -0,0 +1,15 @@ +import numpy as np + + +def generate_uniform_int_column(size, low, high, seed=0): + np.random.seed(seed) + return np.random.randint(low, high, size) + + +def generate_uniform_float_column(size, low, high, seed=0): + np.random.seed(seed) + return np.random.random(size) * (high - low) + low + + +def generate_uniform_string_column(size, samples, seed): + return np.array(samples)[generate_uniform_int_column(size, 0, len(samples), seed)] diff --git a/dbms/tests/external_models/catboost/helpers/server.py b/dbms/tests/external_models/catboost/helpers/server.py new file mode 100644 index 00000000000..fb7f033a4b3 --- /dev/null +++ b/dbms/tests/external_models/catboost/helpers/server.py @@ -0,0 +1,67 @@ +import subprocess +import threading +import socket +import time + + +class ClickHouseServer: + def __init__(self, binary_path, config_path, stdout_file=None, stderr_file=None, shutdown_timeout=10): + self.binary_path = binary_path + self.config_path = config_path + self.pipe = None + self.stdout_file = stdout_file + self.stderr_file = stderr_file + self.shutdown_timeout = shutdown_timeout + + def start(self): + cmd = [self.binary_path, 'server', '--config', self.config_path] + out_pipe = None + err_pipe = None + if self.stdout_file is not None: + out_pipe = open(self.stdout_file, 'w') + if self.stderr_file is not None: + err_pipe = open(self.stderr_file, 'w') + self.pipe = subprocess.Popen(cmd, stdout=out_pipe, stderr=err_pipe) + + def wait_for_request(self, port, timeout=1): + try: + s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + # is not working + # s.settimeout(timeout) + + step = 0.01 + for iter in range(int(timeout / step)): + if s.connect_ex(('127.0.0.1', port)) == 0: + return + time.sleep(step) + + s.connect(('127.0.0.1', port)) + except socket.error as socketerror: + print "Error: ", socketerror + raise + + def shutdown(self, timeout=10): + + def wait(pipe): + pipe.wait() + + if self.pipe is not None: + self.pipe.terminate() + thread = threading.Thread(target=wait, args=(self.pipe,)) + thread.start() + thread.join(timeout) + if thread.isAlive(): + self.pipe.kill() + thread.join() + + if self.pipe.stdout is not None: + self.pipe.stdout.close() + if self.pipe.stderr is not None: + self.pipe.stderr.close() + + def __enter__(self): + self.start() + return self + + def __exit__(self, type, value, traceback): + self.shutdown(self.shutdown_timeout) diff --git a/dbms/tests/external_models/catboost/helpers/server_with_models.py b/dbms/tests/external_models/catboost/helpers/server_with_models.py new file mode 100644 index 00000000000..7f64fdb4cdb --- /dev/null +++ b/dbms/tests/external_models/catboost/helpers/server_with_models.py @@ -0,0 +1,166 @@ +from server import ClickHouseServer +from client import ClickHouseClient +from table import ClickHouseTable +import os +import errno +from shutil import rmtree + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +CATBOOST_ROOT = os.path.dirname(SCRIPT_DIR) + +CLICKHOUSE_CONFIG = \ +''' + + Europe/Moscow + :: + {path} + {tmp_path} + {models_config} + 5368709120 + users.xml + {tcp_port} + {catboost_dynamic_library_path} + +''' + +CLICKHOUSE_USERS = \ +''' + + + + + + 1 + + + + + + + readonly + default + + + + + default + default + + ::1 + 127.0.0.1 + + + + + + + + + + +''' + +CATBOOST_MODEL_CONFIG = \ +''' + + + catboost + {name} + {path} + {float_features_count} + {cat_features_count} + 0 + + +''' + + +class ClickHouseServerWithCatboostModels: + def __init__(self, name, binary_path, port, shutdown_timeout=10, clean_folder=False): + self.models = {} + self.name = name + self.binary_path = binary_path + self.port = port + self.shutdown_timeout = shutdown_timeout + self.clean_folder = clean_folder + self.root = os.path.join(CATBOOST_ROOT, 'data', 'servers') + self.config_path = os.path.join(self.root, 'config.xml') + self.users_path = os.path.join(self.root, 'users.xml') + self.models_dir = os.path.join(self.root, 'models') + self.server = None + + def _get_server(self): + stdout_file = os.path.join(self.root, 'server_stdout.txt') + stderr_file = os.path.join(self.root, 'server_stderr.txt') + return ClickHouseServer(self.binary_path, self.config_path, stdout_file, stderr_file, self.shutdown_timeout) + + def add_model(self, model_name, model, float_features_count, cat_features_count): + self.models[model_name] = (float_features_count, cat_features_count, model) + + def apply_model(self, name, df, cat_feature_names): + names = list(df) + float_feature_names = tuple(name for name in names if name not in cat_feature_names) + with ClickHouseTable(self.server, self.port, name, df) as table: + return table.apply_model(name, cat_feature_names, float_feature_names) + + def _create_root(self): + try: + os.makedirs(self.root) + except OSError as exc: # Python >2.5 + if exc.errno == errno.EEXIST and os.path.isdir(self.root): + pass + else: + raise + + def _clean_root(self): + rmtree(self.root) + + def _save_config(self): + params = { + 'tcp_port': self.port, + 'path': os.path.join(self.root, 'clickhouse'), + 'tmp_path': os.path.join(self.root, 'clickhouse', 'tmp'), + 'models_config': os.path.join(self.models_dir, '*_model.xml'), + 'catboost_dynamic_library_path': os.path.join(CATBOOST_ROOT, 'data', 'libcatboostmodel.so') + } + config = CLICKHOUSE_CONFIG.format(**params) + + with open(self.config_path, 'w') as f: + f.write(config) + + with open(self.users_path, 'w') as f: + f.write(CLICKHOUSE_USERS) + + def _save_models(self): + if not os.path.exists(self.models_dir): + os.makedirs(self.models_dir) + + for name, params in self.models.items(): + float_features_count, cat_features_count, model = params + model_path = os.path.join(self.models_dir, name + '.cbm') + config_path = os.path.join(self.models_dir, name + '_model.xml') + params = { + 'name': name, + 'path': model_path, + 'float_features_count': float_features_count, + 'cat_features_count': cat_features_count + } + config = CATBOOST_MODEL_CONFIG.format(**params) + with open(config_path, 'w') as f: + f.write(config) + + model.save_model(model_path) + + def __enter__(self): + self._create_root() + self._save_config() + self._save_models() + self.server = self._get_server().__enter__() + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + res = self.server.__exit__(exc_type, exc_val, exc_tb) + if self.clean_folder: + self._clean_root() + return res + diff --git a/dbms/tests/external_models/catboost/helpers/table.py b/dbms/tests/external_models/catboost/helpers/table.py new file mode 100644 index 00000000000..2e9c454ab10 --- /dev/null +++ b/dbms/tests/external_models/catboost/helpers/table.py @@ -0,0 +1,69 @@ +from server import ClickHouseServer +from client import ClickHouseClient +from pandas import DataFrame +import os +import threading +import tempfile + + +class ClickHouseTable: + def __init__(self, server, port, table_name, df): + self.server = server + self.port = port + self.table_name = table_name + self.df = df + + if not isinstance(self.server, ClickHouseServer): + raise Exception('Expected ClickHouseServer, got ' + repr(self.server)) + if not isinstance(self.df, DataFrame): + raise Exception('Expected DataFrame, got ' + repr(self.df)) + + self.server.wait_for_request(port) + self.client = ClickHouseClient(server.binary_path, port) + + def _convert(self, name): + types_map = { + 'float64': 'Float64', + 'int64': 'Int64', + 'float32': 'Float32', + 'int32': 'Int32' + } + + if name in types_map: + return types_map[name] + return 'String' + + def _create_table_from_df(self): + self.client.query('create database if not exists test') + self.client.query('drop table if exists test.{}'.format(self.table_name)) + + column_types = list(self.df.dtypes) + column_names = list(self.df) + schema = ', '.join((name + ' ' + self._convert(str(t)) for name, t in zip(column_names, column_types))) + print 'schema:', schema + + create_query = 'create table test.{} (date Date DEFAULT today(), {}) engine = MergeTree(date, (date), 8192)' + self.client.query(create_query.format(self.table_name, schema)) + + insert_query = 'insert into test.{} ({}) format CSV' + + with tempfile.TemporaryFile() as tmp_file: + self.df.to_csv(tmp_file, header=False, index=False) + tmp_file.seek(0) + self.client.query(insert_query.format(self.table_name, ', '.join(column_names)), pipe=tmp_file) + + def apply_model(self, model_name, float_columns, cat_columns): + columns = ', '.join(list(float_columns) + list(cat_columns)) + query = "select modelEvaluate('{}', {}) from test.{} format TSV" + result = self.client.query(query.format(model_name, columns, self.table_name)) + return tuple(map(float, filter(len, map(str.strip, result.split())))) + + def _drop_table(self): + self.client.query('drop table test.{}'.format(self.table_name)) + + def __enter__(self): + self._create_table_from_df() + return self + + def __exit__(self, type, value, traceback): + self._drop_table() diff --git a/dbms/tests/external_models/catboost/helpers/train.py b/dbms/tests/external_models/catboost/helpers/train.py new file mode 100644 index 00000000000..df81d719553 --- /dev/null +++ b/dbms/tests/external_models/catboost/helpers/train.py @@ -0,0 +1,28 @@ +import os +import sys +from pandas import DataFrame + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +CATBOOST_ROOT = os.path.dirname(SCRIPT_DIR) +CATBOOST_PYTHON_DIR = os.path.join(CATBOOST_ROOT, 'data', 'python-package') + +if CATBOOST_PYTHON_DIR not in sys.path: + sys.path.append(CATBOOST_PYTHON_DIR) + + +import catboost +from catboost import CatBoostClassifier + + +def train_catboost_model(df, target, cat_features, params, verbose=True): + + if not isinstance(df, DataFrame): + raise Exception('DataFrame object expected, but got ' + repr(df)) + + print 'features:', df.columns.tolist() + + cat_features_index = list(df.columns.get_loc(feature) for feature in cat_features) + print 'cat features:', cat_features_index + model = CatBoostClassifier(**params) + model.fit(df, target, cat_features=cat_features_index, verbose=verbose) + return model diff --git a/dbms/tests/external_models/catboost/pytest.ini b/dbms/tests/external_models/catboost/pytest.ini new file mode 100644 index 00000000000..a40472347fb --- /dev/null +++ b/dbms/tests/external_models/catboost/pytest.ini @@ -0,0 +1,3 @@ +[pytest] +python_files = test.py +norecursedirs=data diff --git a/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py b/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py new file mode 100644 index 00000000000..c38e92636b7 --- /dev/null +++ b/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py @@ -0,0 +1,236 @@ +from helpers.server_with_models import ClickHouseServerWithCatboostModels +from helpers.generate import generate_uniform_string_column, generate_uniform_float_column, generate_uniform_int_column +from helpers.train import train_catboost_model +import os +import numpy as np +from pandas import DataFrame + + +PORT = int(os.environ.get('CLICKHOUSE_TESTS_PORT', '9000')) +CLICKHOUSE_TESTS_SERVER_BIN_PATH = os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse') + + +def add_noise_to_target(target, seed, threshold=0.05): + col = generate_uniform_float_column(len(target), 0., 1., seed + 1) < threshold + return target * (1 - col) + (1 - target) * col + + +def check_predictions(test_name, target, pred_python, pred_ch, acc_threshold): + ch_class = pred_ch.astype(int) + python_class = pred_python.astype(int) + if not np.array_equal(ch_class, python_class): + raise Exception('Got different results:\npython:\n' + str(python_class) + '\nClickHouse:\n' + str(ch_class)) + + acc = 1 - np.sum(np.abs(ch_class - np.array(target))) / (len(target) + .0) + assert acc >= acc_threshold + print test_name, 'accuracy: {:.10f}'.format(acc) + + +def test_apply_float_features_only(): + + name = 'test_apply_float_features_only' + + train_size = 10000 + test_size = 10000 + + def gen_data(size, seed): + data = { + 'a': generate_uniform_float_column(size, 0., 1., seed + 1), + 'b': generate_uniform_float_column(size, 0., 1., seed + 2), + 'c': generate_uniform_float_column(size, 0., 1., seed + 3) + } + return DataFrame.from_dict(data) + + def get_target(df): + def target_filter(row): + return 1 if (row['a'] > .3 and row['b'] > .3) or (row['c'] < .4 and row['a'] * row['b'] > 0.1) else 0 + return df.apply(target_filter, axis=1).as_matrix() + + train_df = gen_data(train_size, 42) + test_df = gen_data(test_size, 43) + + train_target = get_target(train_df) + test_target = get_target(test_df) + + print + print 'train target', train_target + print 'test target', test_target + + params = { + 'iterations': 4, + 'depth': 2, + 'learning_rate': 1, + 'loss_function': 'Logloss' + } + + model = train_catboost_model(train_df, train_target, [], params) + pred_python = model.predict(test_df) + + server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) + server.add_model(name, model, 3, 0) + with server: + pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) + + print 'python predictions', pred_python + print 'clickhouse predictions', pred_ch + + check_predictions(name, test_target, pred_python, pred_ch, 0.95) + + +def test_apply_float_features_with_string_cat_features(): + + name = 'test_apply_float_features_with_string_cat_features' + + train_size = 10000 + test_size = 10000 + + def gen_data(size, seed): + data = { + 'a': generate_uniform_float_column(size, 0., 1., seed + 1), + 'b': generate_uniform_float_column(size, 0., 1., seed + 2), + 'c': generate_uniform_string_column(size, ['a', 'b', 'c'], seed + 3), + 'd': generate_uniform_string_column(size, ['e', 'f', 'g'], seed + 4) + } + return DataFrame.from_dict(data) + + def get_target(df): + def target_filter(row): + return 1 if (row['a'] > .3 and row['b'] > .3 and row['c'] != 'a') \ + or (row['a'] * row['b'] > 0.1 and row['c'] != 'b' and row['d'] != 'e') else 0 + return df.apply(target_filter, axis=1).as_matrix() + + train_df = gen_data(train_size, 42) + test_df = gen_data(test_size, 43) + + train_target = get_target(train_df) + test_target = get_target(test_df) + + print + print 'train target', train_target + print 'test target', test_target + + params = { + 'iterations': 6, + 'depth': 2, + 'learning_rate': 1, + 'loss_function': 'Logloss' + } + + model = train_catboost_model(train_df, train_target, ['c', 'd'], params) + pred_python = model.predict(test_df) + + server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) + server.add_model(name, model, 2, 2) + with server: + pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) + + print 'python predictions', pred_python + print 'clickhouse predictions', pred_ch + + check_predictions(name, test_target, pred_python, pred_ch, 0.95) + + +def test_apply_float_features_with_int_cat_features(): + + name = 'test_apply_float_features_with_int_cat_features' + + train_size = 10000 + test_size = 10000 + + def gen_data(size, seed): + data = { + 'a': generate_uniform_float_column(size, 0., 1., seed + 1), + 'b': generate_uniform_float_column(size, 0., 1., seed + 2), + 'c': generate_uniform_int_column(size, 1, 4, seed + 3), + 'd': generate_uniform_int_column(size, 1, 4, seed + 4) + } + return DataFrame.from_dict(data) + + def get_target(df): + def target_filter(row): + return 1 if (row['a'] > .3 and row['b'] > .3 and row['c'] != 1) \ + or (row['a'] * row['b'] > 0.1 and row['c'] != 2 and row['d'] != 3) else 0 + return df.apply(target_filter, axis=1).as_matrix() + + train_df = gen_data(train_size, 42) + test_df = gen_data(test_size, 43) + + train_target = get_target(train_df) + test_target = get_target(test_df) + + print + print 'train target', train_target + print 'test target', test_target + + params = { + 'iterations': 6, + 'depth': 4, + 'learning_rate': 1, + 'loss_function': 'Logloss' + } + + model = train_catboost_model(train_df, train_target, ['c', 'd'], params) + pred_python = model.predict(test_df) + + server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) + server.add_model(name, model, 2, 2) + with server: + pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) + + print 'python predictions', pred_python + print 'clickhouse predictions', pred_ch + + check_predictions(name, test_target, pred_python, pred_ch, 0.95) + + +def test_apply_float_features_with_mixed_cat_features(): + + name = 'test_apply_float_features_with_mixed_cat_features' + + train_size = 10000 + test_size = 10000 + + def gen_data(size, seed): + data = { + 'a': generate_uniform_float_column(size, 0., 1., seed + 1), + 'b': generate_uniform_float_column(size, 0., 1., seed + 2), + 'c': generate_uniform_string_column(size, ['a', 'b', 'c'], seed + 3), + 'd': generate_uniform_int_column(size, 1, 4, seed + 4) + } + return DataFrame.from_dict(data) + + def get_target(df): + def target_filter(row): + return 1 if (row['a'] > .3 and row['b'] > .3 and row['c'] != 'a') \ + or (row['a'] * row['b'] > 0.1 and row['c'] != 'b' and row['d'] != 2) else 0 + return df.apply(target_filter, axis=1).as_matrix() + + train_df = gen_data(train_size, 42) + test_df = gen_data(test_size, 43) + + train_target = get_target(train_df) + test_target = get_target(test_df) + + print + print 'train target', train_target + print 'test target', test_target + + params = { + 'iterations': 6, + 'depth': 4, + 'learning_rate': 1, + 'loss_function': 'Logloss' + } + + model = train_catboost_model(train_df, train_target, ['c', 'd'], params) + pred_python = model.predict(test_df) + + server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) + server.add_model(name, model, 2, 2) + with server: + pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) + + print 'python predictions', pred_python + print 'clickhouse predictions', pred_ch + + check_predictions(name, test_target, pred_python, pred_ch, 0.95) From fd8eb13dcc45841b0285f9f0be30bd64408c89bc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 15:18:37 +0300 Subject: [PATCH 18/32] refactoring [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 34 ++++++++++++------------- dbms/src/Dictionaries/CatBoostModel.h | 14 +++++----- dbms/src/Dictionaries/IDictionary.h | 5 ++-- 3 files changed, 27 insertions(+), 26 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index cf0e4a14609..2a6df227892 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -23,7 +23,7 @@ extern const int CANNOT_APPLY_CATBOOST_MODEL; /// CatBoost wrapper interface functions. -struct CatBoostWrapperApi +struct CatBoostWrapperAPI { typedef void ModelCalcerHandle; @@ -61,21 +61,21 @@ namespace class CatBoostModelHolder { private: - CatBoostWrapperApi::ModelCalcerHandle * handle; - const CatBoostWrapperApi * api; + CatBoostWrapperAPI::ModelCalcerHandle * handle; + const CatBoostWrapperAPI * api; public: - explicit CatBoostModelHolder(const CatBoostWrapperApi * api) : api(api) { handle = api->ModelCalcerCreate(); } + explicit CatBoostModelHolder(const CatBoostWrapperAPI * api) : api(api) { handle = api->ModelCalcerCreate(); } ~CatBoostModelHolder() { api->ModelCalcerDelete(handle); } - CatBoostWrapperApi::ModelCalcerHandle * get() { return handle; } - explicit operator CatBoostWrapperApi::ModelCalcerHandle * () { return handle; } + CatBoostWrapperAPI::ModelCalcerHandle * get() { return handle; } + explicit operator CatBoostWrapperAPI::ModelCalcerHandle * () { return handle; } }; class CatBoostModelImpl : public ICatBoostModel { public: - CatBoostModelImpl(const CatBoostWrapperApi * api, const std::string & model_path) : api(api) + CatBoostModelImpl(const CatBoostWrapperAPI * api, const std::string & model_path) : api(api) { auto handle_ = std::make_unique(api); if (!handle_) @@ -91,7 +91,7 @@ public: handle = std::move(handle_); } - ColumnPtr eval(const Columns & columns, size_t float_features_count, size_t cat_features_count) const override + ColumnPtr evaluate(const Columns & columns, size_t float_features_count, size_t cat_features_count) const override { if (columns.empty()) throw Exception("Got empty columns list for CatBoost model.", ErrorCodes::BAD_ARGUMENTS); @@ -143,7 +143,7 @@ public: private: std::unique_ptr handle; - const CatBoostWrapperApi * api; + const CatBoostWrapperAPI * api; /// Buffer should be allocated with features_count * column->size() elements. /// Place column elements in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] @@ -379,20 +379,20 @@ private: /// Holds CatBoost wrapper library and provides wrapper interface. -class CatBoostLibHolder: public CatBoostWrapperApiProvider +class CatBoostLibHolder: public CatBoostWrapperAPIProvider { public: - explicit CatBoostLibHolder(const std::string & lib_path) : lib_path(lib_path), lib(lib_path) { initApi(); } + explicit CatBoostLibHolder(const std::string & lib_path) : lib_path(lib_path), lib(lib_path) { initAPI(); } - const CatBoostWrapperApi & getApi() const override { return api; } + const CatBoostWrapperAPI & getAPI() const override { return api; } const std::string & getCurrentPath() const { return lib_path; } private: - CatBoostWrapperApi api; + CatBoostWrapperAPI api; std::string lib_path; boost::dll::shared_library lib; - void initApi(); + void initAPI(); template void load(T& func, const std::string & name) @@ -402,7 +402,7 @@ private: } }; -void CatBoostLibHolder::initApi() +void CatBoostLibHolder::initAPI() { load(api.ModelCalcerCreate, "ModelCalcerCreate"); load(api.ModelCalcerDelete, "ModelCalcerDelete"); @@ -455,7 +455,7 @@ CatBoostModel::CatBoostModel(const std::string & name, const std::string & model void CatBoostModel::init(const std::string & lib_path) { api_provider = getCatBoostWrapperHolder(lib_path); - api = &api_provider->getApi(); + api = &api_provider->getAPI(); model = std::make_unique(api, model_path); } @@ -488,7 +488,7 @@ ColumnPtr CatBoostModel::evaluate(const Columns & columns) const { if (!model) throw Exception("CatBoost model was not loaded.", ErrorCodes::LOGICAL_ERROR); - return model->eval(columns, float_features_count, cat_features_count); + return model->evaluate(columns, float_features_count, cat_features_count); } } diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index 884f367f25b..db00ba36bb3 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -8,12 +8,12 @@ namespace DB { /// CatBoost wrapper interface functions. -struct CatBoostWrapperApi; -class CatBoostWrapperApiProvider +struct CatBoostWrapperAPI; +class CatBoostWrapperAPIProvider { public: - virtual ~CatBoostWrapperApiProvider() = default; - virtual const CatBoostWrapperApi & getApi() const = 0; + virtual ~CatBoostWrapperAPIProvider() = default; + virtual const CatBoostWrapperAPI & getAPI() const = 0; }; /// CatBoost model interface. @@ -23,7 +23,7 @@ public: virtual ~ICatBoostModel() = default; /// Evaluate model. Use first `float_features_count` columns as float features, /// the others `cat_features_count` as categorical features. - virtual ColumnPtr eval(const Columns & columns, size_t float_features_count, size_t cat_features_count) const = 0; + virtual ColumnPtr evaluate(const Columns & columns, size_t float_features_count, size_t cat_features_count) const = 0; }; /// General ML model evaluator interface. @@ -65,8 +65,8 @@ private: std::string lib_path; ExternalLoadableLifetime lifetime; std::exception_ptr creation_exception; - std::shared_ptr api_provider; - const CatBoostWrapperApi * api; + std::shared_ptr api_provider; + const CatBoostWrapperAPI * api; std::unique_ptr model; diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index 48b8206fbca..77bc04a30b9 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -56,14 +56,15 @@ struct IDictionaryBase : public IExternalLoadable bool supportUpdates() const override { return !isCached(); } bool isModified() const override - { auto source = getSource(); + { + auto source = getSource(); return source && source->isModified(); } std::unique_ptr cloneObject() const override { return clone(); - }; + } std::shared_ptr shared_from_this() { From c3d104f53ba554a059535563a3286dffc056741e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 16:19:58 +0300 Subject: [PATCH 19/32] changed IDictionary clone [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CacheDictionary.h | 2 +- dbms/src/Dictionaries/ComplexKeyCacheDictionary.h | 2 +- dbms/src/Dictionaries/ComplexKeyHashedDictionary.h | 2 +- dbms/src/Dictionaries/FlatDictionary.h | 2 +- dbms/src/Dictionaries/HashedDictionary.h | 2 +- dbms/src/Dictionaries/IDictionary.h | 2 +- dbms/src/Dictionaries/RangeHashedDictionary.h | 2 +- dbms/src/Dictionaries/TrieDictionary.h | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index ac0854fc84e..942b4cb633c 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -54,7 +54,7 @@ public: bool isCached() const override { return true; } - DictionaryPtr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override { return std::make_unique(*this); } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index 078192cee1e..2e90d3187b3 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -98,7 +98,7 @@ public: return true; } - DictionaryPtr clone() const override + std::unique_ptr clone() const override { return std::make_unique(*this); } diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h index 1140e53cf09..7524533a049 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -46,7 +46,7 @@ public: bool isCached() const override { return false; } - DictionaryPtr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override { return std::make_unique(*this); } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/FlatDictionary.h b/dbms/src/Dictionaries/FlatDictionary.h index 0a28896a941..cdeabea1e3c 100644 --- a/dbms/src/Dictionaries/FlatDictionary.h +++ b/dbms/src/Dictionaries/FlatDictionary.h @@ -41,7 +41,7 @@ public: bool isCached() const override { return false; } - DictionaryPtr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override { return std::make_unique(*this); } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index 8c0e01dfce3..55bc7523b67 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -40,7 +40,7 @@ public: bool isCached() const override { return false; } - DictionaryPtr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override { return std::make_unique(*this); } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index 77bc04a30b9..21083677a12 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -41,7 +41,7 @@ struct IDictionaryBase : public IExternalLoadable virtual bool isCached() const = 0; - virtual DictionaryPtr clone() const = 0; + virtual std::unique_ptr clone() const = 0; virtual const IDictionarySource * getSource() const = 0; diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.h b/dbms/src/Dictionaries/RangeHashedDictionary.h index 38a53c7d3df..1aa7ce893f9 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.h +++ b/dbms/src/Dictionaries/RangeHashedDictionary.h @@ -41,7 +41,7 @@ public: bool isCached() const override { return false; } - DictionaryPtr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override { return std::make_unique(*this); } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/TrieDictionary.h b/dbms/src/Dictionaries/TrieDictionary.h index 0b548bc1c9e..2a958c2f74a 100644 --- a/dbms/src/Dictionaries/TrieDictionary.h +++ b/dbms/src/Dictionaries/TrieDictionary.h @@ -50,7 +50,7 @@ public: bool isCached() const override { return false; } - DictionaryPtr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override { return std::make_unique(*this); } const IDictionarySource * getSource() const override { return source_ptr.get(); } From 2e6b1dfd929ffa7fc6696463667ea54ae7732100 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 16:36:01 +0300 Subject: [PATCH 20/32] renamed cloneObject to clone in IExternalLoadable [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 2 +- dbms/src/Dictionaries/CatBoostModel.h | 2 +- dbms/src/Dictionaries/IDictionary.h | 7 ------- dbms/src/Interpreters/ExternalLoader.cpp | 4 ++-- dbms/src/Interpreters/IExternalLoadable.h | 2 +- 5 files changed, 5 insertions(+), 12 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 2a6df227892..de948791fa3 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -469,7 +469,7 @@ bool CatBoostModel::isModified() const return true; } -std::unique_ptr CatBoostModel::cloneObject() const +std::unique_ptr CatBoostModel::clone() const { return std::make_unique(name, model_path, lib_path, lifetime, float_features_count, cat_features_count); } diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index db00ba36bb3..f4248930aa2 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -55,7 +55,7 @@ public: bool isModified() const override; - std::unique_ptr cloneObject() const override; + std::unique_ptr clone() const override; std::exception_ptr getCreationException() const override { return creation_exception; } diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index 21083677a12..cba1a173833 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -41,8 +41,6 @@ struct IDictionaryBase : public IExternalLoadable virtual bool isCached() const = 0; - virtual std::unique_ptr clone() const = 0; - virtual const IDictionarySource * getSource() const = 0; virtual const DictionaryStructure & getStructure() const = 0; @@ -61,11 +59,6 @@ struct IDictionaryBase : public IExternalLoadable return source && source->isModified(); } - std::unique_ptr cloneObject() const override - { - return clone(); - } - std::shared_ptr shared_from_this() { return std::static_pointer_cast(IExternalLoadable::shared_from_this()); diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 9b784639366..3bde003771b 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -127,7 +127,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error) try { - auto loadable_ptr = failed_loadable_object.second.loadable->cloneObject(); + auto loadable_ptr = failed_loadable_object.second.loadable->clone(); if (const auto exception_ptr = loadable_ptr->getCreationException()) { /// recalculate next attempt time @@ -211,7 +211,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error) if (current->isModified()) { /// create new version of loadable object - auto new_version = current->cloneObject(); + auto new_version = current->clone(); if (const auto exception_ptr = new_version->getCreationException()) std::rethrow_exception(exception_ptr); diff --git a/dbms/src/Interpreters/IExternalLoadable.h b/dbms/src/Interpreters/IExternalLoadable.h index 9e52b9e6864..c16f1538e09 100644 --- a/dbms/src/Interpreters/IExternalLoadable.h +++ b/dbms/src/Interpreters/IExternalLoadable.h @@ -35,7 +35,7 @@ public: virtual bool isModified() const = 0; - virtual std::unique_ptr cloneObject() const = 0; + virtual std::unique_ptr clone() const = 0; virtual std::exception_ptr getCreationException() const = 0; }; From 4204a94b20713d52214d23e40bd5c7a048733d5e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 17:08:05 +0300 Subject: [PATCH 21/32] changed CatBoost model evaluate signature [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 38 +++++++++++++------------ dbms/src/Dictionaries/CatBoostModel.h | 6 ++-- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index de948791fa3..9401df9a40f 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -91,7 +91,8 @@ public: handle = std::move(handle_); } - ColumnPtr evaluate(const Columns & columns, size_t float_features_count, size_t cat_features_count) const override + ColumnPtr evaluate(const ConstColumnPlainPtrs & columns, + size_t float_features_count, size_t cat_features_count) const override { if (columns.empty()) throw Exception("Got empty columns list for CatBoost model.", ErrorCodes::BAD_ARGUMENTS); @@ -123,11 +124,11 @@ public: bool cat_features_are_strings = true; for (size_t i = float_features_count; i < float_features_count + cat_features_count; ++i) { - const auto & column = columns[i]; + auto column = columns[i]; if (column->isNumeric()) cat_features_are_strings = false; - else if (!(typeid_cast(column.get()) - || typeid_cast(column.get()))) + else if (!(typeid_cast(column) + || typeid_cast(column))) { std::string msg; { @@ -148,7 +149,7 @@ private: /// Buffer should be allocated with features_count * column->size() elements. /// Place column elements in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] template - void placeColumnAsNumber(const ColumnPtr & column, T * buffer, size_t features_count) const + void placeColumnAsNumber(const IColumn * column, T * buffer, size_t features_count) const { size_t size = column->size(); FieldVisitorConvertToNumber visitor; @@ -200,7 +201,8 @@ private: /// Place columns into buffer, returns column which holds placed data. Buffer should contains column->size() values. template - ColumnPtr placeNumericColumns(const Columns & columns, size_t offset, size_t size, const T** buffer) const + ColumnPtr placeNumericColumns(const ConstColumnPlainPtrs & columns, + size_t offset, size_t size, const T** buffer) const { if (size == 0) return nullptr; @@ -209,7 +211,7 @@ private: T* data = data_column->getData().data(); for (size_t i = 0; i < size; ++i) { - const auto & column = columns[offset + i]; + auto column = columns[offset + i]; if (column->isNumeric()) placeColumnAsNumber(column, data + i, size); } @@ -227,7 +229,7 @@ private: /// Place columns into buffer, returns data which was used for fixed string columns. /// Buffer should contains column->size() values, each value contains size strings. std::vector> placeStringColumns( - const Columns & columns, size_t offset, size_t size, const char ** buffer) const + const ConstColumnPlainPtrs & columns, size_t offset, size_t size, const char ** buffer) const { if (size == 0) return {}; @@ -235,10 +237,10 @@ private: std::vector> data; for (size_t i = 0; i < size; ++i) { - const auto & column = columns[offset + i]; - if (auto column_string = typeid_cast(column.get())) + auto column = columns[offset + i]; + if (auto column_string = typeid_cast(column)) placeStringColumn(*column_string, buffer + i, size); - else if (auto column_fixed_string = typeid_cast(column.get())) + else if (auto column_fixed_string = typeid_cast(column)) data.push_back(placeFixedStringColumn(*column_fixed_string, buffer + i, size)); else throw Exception("Cannot place string column.", ErrorCodes::LOGICAL_ERROR); @@ -273,7 +275,7 @@ private: /// buffer contains column->size() rows and size columns. /// For int cat features calc hash inplace. /// For string cat features calc hash from column rows. - void calcHashes(const Columns & columns, size_t offset, size_t size, const int ** buffer) const + void calcHashes(const ConstColumnPlainPtrs & columns, size_t offset, size_t size, const int ** buffer) const { if (size == 0) return; @@ -282,10 +284,10 @@ private: std::vector> data; for (size_t i = 0; i < size; ++i) { - const auto & column = columns[offset + i]; - if (auto column_string = typeid_cast(column.get())) + auto column = columns[offset + i]; + if (auto column_string = typeid_cast(column)) calcStringHashes(column_string, i, buffer); - else if (auto column_fixed_string = typeid_cast(column.get())) + else if (auto column_fixed_string = typeid_cast(column)) calcStringHashes(column_fixed_string, i, buffer); else calcIntHashes(column_size, i, buffer); @@ -308,7 +310,7 @@ private: /// * CalcModelPredictionFlat if no cat features /// * CalcModelPrediction if all cat features are strings /// * CalcModelPredictionWithHashedCatFeatures if has int cat features. - ColumnPtr evalImpl(const Columns & columns, size_t float_features_count, size_t cat_features_count, + ColumnPtr evalImpl(const ConstColumnPlainPtrs & columns, size_t float_features_count, size_t cat_features_count, bool cat_features_are_strings) const { std::string error_msg = "Error occurred while applying CatBoost model: "; @@ -344,7 +346,7 @@ private: auto cat_features_buf = cat_features.data(); fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count); - /// Fixed strings are stored without termination zero, so have to copy data into fixed_strings_data + /// Fixed strings are stored without termination zero, so have to copy data into fixed_strings_data. auto fixed_strings_data = placeStringColumns(columns, float_features_count, cat_features_count, cat_features_holder.data()); @@ -484,7 +486,7 @@ size_t CatBoostModel::getCatFeaturesCount() const return cat_features_count; } -ColumnPtr CatBoostModel::evaluate(const Columns & columns) const +ColumnPtr CatBoostModel::evaluate(const ConstColumnPlainPtrs & columns) const { if (!model) throw Exception("CatBoost model was not loaded.", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index f4248930aa2..4c70e81594f 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -23,14 +23,14 @@ public: virtual ~ICatBoostModel() = default; /// Evaluate model. Use first `float_features_count` columns as float features, /// the others `cat_features_count` as categorical features. - virtual ColumnPtr evaluate(const Columns & columns, size_t float_features_count, size_t cat_features_count) const = 0; + virtual ColumnPtr evaluate(const ConstColumnPlainPtrs & columns, size_t float_features_count, size_t cat_features_count) const = 0; }; /// General ML model evaluator interface. class IModel : public IExternalLoadable { public: - virtual ColumnPtr evaluate(const Columns & columns) const = 0; + virtual ColumnPtr evaluate(const ConstColumnPlainPtrs & columns) const = 0; }; class CatBoostModel : public IModel @@ -40,7 +40,7 @@ public: const std::string & lib_path, const ExternalLoadableLifetime & lifetime, size_t float_features_count, size_t cat_features_count); - ColumnPtr evaluate(const Columns & columns) const override; + ColumnPtr evaluate(const ConstColumnPlainPtrs & columns) const override; size_t getFloatFeaturesCount() const; size_t getCatFeaturesCount() const; From 092567af39b6f2ad898b342c7668d31fdf9a36a0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 17:10:51 +0300 Subject: [PATCH 22/32] changed CatBoost model evaluate signature [#CLICKHOUSE-3305] --- dbms/src/Functions/FunctionsExternalModels.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index f6843a178a8..332f7d5fe46 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -47,10 +47,10 @@ void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arg auto model = models.getModel(name_col->getValue()); - Columns columns; + ConstColumnPlainPtrs columns; columns.reserve(arguments.size()); for (auto i : ext::range(1, arguments.size())) - columns.push_back(block.getByPosition(arguments[i]).column); + columns.push_back(block.getByPosition(arguments[i]).column.get()); block.getByPosition(result).column = model->evaluate(columns); } From 371e234f01e626252bfef159241f290a72fc8a06 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 17:26:03 +0300 Subject: [PATCH 23/32] changed ExternalLoader::getObjectsMap return type [#CLICKHOUSE-3305] --- dbms/src/Databases/DatabaseDictionary.cpp | 8 ++++---- dbms/src/Interpreters/ExternalLoader.cpp | 4 ++-- dbms/src/Interpreters/ExternalLoader.h | 12 +++++++++++- .../Storages/System/StorageSystemDictionaries.cpp | 2 +- 4 files changed, 18 insertions(+), 8 deletions(-) diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index 0dc471c9e0a..f41b23c9008 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -27,7 +27,7 @@ void DatabaseDictionary::loadTables(Context & context, ThreadPool * thread_pool, Tables DatabaseDictionary::loadTables() { auto objects_map = external_dictionaries.getObjectsMap(); - const auto & dictionaries = std::get<1>(objects_map); + const auto & dictionaries = objects_map.get(); Tables tables; for (const auto & pair : dictionaries) @@ -52,7 +52,7 @@ bool DatabaseDictionary::isTableExist( const String & table_name) const { auto objects_map = external_dictionaries.getObjectsMap(); - const auto & dictionaries = std::get<1>(objects_map); + const auto & dictionaries = objects_map.get(); return dictionaries.count(table_name) && !deleted_tables.count(table_name); } @@ -61,7 +61,7 @@ StoragePtr DatabaseDictionary::tryGetTable( const String & table_name) { auto objects_map = external_dictionaries.getObjectsMap(); - const auto & dictionaries = std::get<1>(objects_map); + const auto & dictionaries = objects_map.get(); if (deleted_tables.count(table_name)) return {}; @@ -90,7 +90,7 @@ DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context) bool DatabaseDictionary::empty(const Context & context) const { auto objects_map = external_dictionaries.getObjectsMap(); - const auto & dictionaries = std::get<1>(objects_map); + const auto & dictionaries = objects_map.get(); for (const auto & pair : dictionaries) if (pair.second.loadable && !deleted_tables.count(pair.first)) return false; diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 3bde003771b..3515ec6b2cd 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -425,9 +425,9 @@ ExternalLoader::LoadablePtr ExternalLoader::getLoadable(const std::string & name return it->second.loadable; } -std::tuple, const ExternalLoader::ObjectsMap &> ExternalLoader::getObjectsMap() const +ExternalLoader::LockedObjectsMap ExternalLoader::getObjectsMap() const { - return std::make_tuple(std::unique_lock(map_mutex), std::cref(loadable_objects)); + return LockedObjectsMap(map_mutex, loadable_objects); } } diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index 672c801424e..d42e9f07877 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -101,8 +101,18 @@ protected: virtual std::unique_ptr create(const std::string & name, const Configuration & config, const std::string & config_prefix) = 0; + class LockedObjectsMap + { + public: + LockedObjectsMap(std::mutex & mutex, const ObjectsMap & objectsMap) : lock(mutex), objectsMap(objectsMap) {} + const ObjectsMap & get() { return objectsMap; } + private: + std::unique_lock lock; + const ObjectsMap & objectsMap; + }; + /// Direct access to objects. - std::tuple, const ObjectsMap &> getObjectsMap() const; + LockedObjectsMap getObjectsMap() const; /// Should be called in derived constructor (to avoid pure virtual call). void init(bool throw_on_error); diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index 02a8c7a44aa..64d633d2d6b 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -77,7 +77,7 @@ BlockInputStreams StorageSystemDictionaries::read( const auto & external_dictionaries = context.getExternalDictionaries(); auto objects_map = external_dictionaries.getObjectsMap(); - const auto & dictionaries = std::get<1>(objects_map); + const auto & dictionaries = objects_map.get(); for (const auto & dict_info : dictionaries) { From 10bbf0871b05aa9dd0abc010b5bc8bdbf83f11b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 21:30:28 +0300 Subject: [PATCH 24/32] added comments [#CLICKHOUSE-3305] --- dbms/src/Interpreters/ExternalModels.h | 2 +- dbms/src/Interpreters/IExternalLoadable.h | 8 +++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/ExternalModels.h b/dbms/src/Interpreters/ExternalModels.h index 889cd80e8e1..bbe23ad28b3 100644 --- a/dbms/src/Interpreters/ExternalModels.h +++ b/dbms/src/Interpreters/ExternalModels.h @@ -11,7 +11,7 @@ namespace DB class Context; -/// Manages user-defined dictionaries. +/// Manages user-defined models. class ExternalModels : public ExternalLoader { public: diff --git a/dbms/src/Interpreters/IExternalLoadable.h b/dbms/src/Interpreters/IExternalLoadable.h index c16f1538e09..6423ddd19bd 100644 --- a/dbms/src/Interpreters/IExternalLoadable.h +++ b/dbms/src/Interpreters/IExternalLoadable.h @@ -22,6 +22,8 @@ struct ExternalLoadableLifetime final ExternalLoadableLifetime(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); }; + +/// Basic interface for external loadable objects. Is used in ExternalLoader. class IExternalLoadable : public std::enable_shared_from_this { public: @@ -30,11 +32,11 @@ public: virtual const ExternalLoadableLifetime & getLifetime() const = 0; virtual std::string getName() const = 0; - + /// True if object can be updated when lifetime exceeded. virtual bool supportUpdates() const = 0; - + /// If lifetime exceeded and isModified() ExternalLoader replace current object with the result of clone(). virtual bool isModified() const = 0; - + /// Returns new object with the same configuration. Is used to update modified object when lifetime exceeded. virtual std::unique_ptr clone() const = 0; virtual std::exception_ptr getCreationException() const = 0; From b66577e38a42c7a97521cc9aaea2377a0985f620 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 21:39:55 +0300 Subject: [PATCH 25/32] changed ExternalLoaderUpdateSettings [#CLICKHOUSE-3305] --- .../src/Interpreters/ExternalDictionaries.cpp | 17 +-------- dbms/src/Interpreters/ExternalLoader.h | 6 +++ dbms/src/Interpreters/ExternalModels.cpp | 38 +++++++------------ 3 files changed, 21 insertions(+), 40 deletions(-) diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 4c1e563e1f2..65c0aaf2031 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -7,20 +7,7 @@ namespace DB namespace { - const ExternalLoaderUpdateSettings & getExternalDictionariesUpdateSettings() - { - static ExternalLoaderUpdateSettings settings; - static std::once_flag flag; - - std::call_once(flag, [] { - settings.check_period_sec = 5; - settings.backoff_initial_sec = 5; - /// 10 minutes - settings.backoff_max_sec = 10 * 60; - }); - - return settings; - } + const ExternalLoaderUpdateSettings externalDictionariesUpdateSettings; const ExternalLoaderConfigSettings & getExternalDictionariesConfigSettings() { @@ -41,7 +28,7 @@ namespace ExternalDictionaries::ExternalDictionaries(Context & context, bool throw_on_error) : ExternalLoader(context.getConfigRef(), - getExternalDictionariesUpdateSettings(), + externalDictionariesUpdateSettings, getExternalDictionariesConfigSettings(), &Logger::get("ExternalDictionaries"), "external dictionary"), diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index d42e9f07877..526a2457ea1 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -24,6 +24,12 @@ struct ExternalLoaderUpdateSettings UInt64 backoff_initial_sec = 5; /// 10 minutes UInt64 backoff_max_sec = 10 * 60; + + ExternalLoaderUpdateSettings() = default; + ExternalLoaderUpdateSettings(UInt64 check_period_sec, UInt64 backoff_initial_sec, UInt64 backoff_max_sec) + : check_period_sec(check_period_sec), + backoff_initial_sec(backoff_initial_sec), + backoff_max_sec(backoff_max_sec) {} }; diff --git a/dbms/src/Interpreters/ExternalModels.cpp b/dbms/src/Interpreters/ExternalModels.cpp index 1a848fa4353..84f3d363a6e 100644 --- a/dbms/src/Interpreters/ExternalModels.cpp +++ b/dbms/src/Interpreters/ExternalModels.cpp @@ -11,41 +11,29 @@ namespace ErrorCodes namespace { -const ExternalLoaderUpdateSettings & getExternalModelsUpdateSettings() -{ - static ExternalLoaderUpdateSettings settings; - static std::once_flag flag; - std::call_once(flag, [] { - settings.check_period_sec = 5; - settings.backoff_initial_sec = 5; - /// 10 minutes - settings.backoff_max_sec = 10 * 60; - }); + const ExternalLoaderUpdateSettings externalModelsUpdateSettings; - return settings; -} + const ExternalLoaderConfigSettings & getExternalModelsConfigSettings() + { + static ExternalLoaderConfigSettings settings; + static std::once_flag flag; -const ExternalLoaderConfigSettings & getExternalModelsConfigSettings() -{ - static ExternalLoaderConfigSettings settings; - static std::once_flag flag; + std::call_once(flag, [] { + settings.external_config = "model"; + settings.external_name = "name"; - std::call_once(flag, [] { - settings.external_config = "model"; - settings.external_name = "name"; + settings.path_setting_name = "models_config"; + }); - settings.path_setting_name = "models_config"; - }); - - return settings; -} + return settings; + } } ExternalModels::ExternalModels(Context & context, bool throw_on_error) : ExternalLoader(context.getConfigRef(), - getExternalModelsUpdateSettings(), + externalModelsUpdateSettings, getExternalModelsConfigSettings(), &Logger::get("ExternalModels"), "external model"), From 3757c9682e73d3e437ea419bc2191ef0c12634d1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 21:49:23 +0300 Subject: [PATCH 26/32] style fixes [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/IDictionary.h | 1 + dbms/src/Functions/FunctionsExternalModels.cpp | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index cba1a173833..6490fdd6833 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -63,6 +63,7 @@ struct IDictionaryBase : public IExternalLoadable { return std::static_pointer_cast(IExternalLoadable::shared_from_this()); } + std::shared_ptr shared_from_this() const { return std::static_pointer_cast(IExternalLoadable::shared_from_this()); diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index 332f7d5fe46..fc8f3ac97fe 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -34,7 +34,6 @@ DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const DataTypes & arguments throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(); } From 46dfa7bf8d283bf44509e0cd887824d098302611 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 22:00:27 +0300 Subject: [PATCH 27/32] changed boost::dll to SharedLibrary; removed boost::dll [#CLICKHOUSE-3305] --- .../libboost/boost_1_65_0/boost/dll/alias.hpp | 264 --------- .../boost/dll/detail/aggressive_ptr_cast.hpp | 135 ----- .../boost/dll/detail/ctor_dtor.hpp | 192 ------ .../dll/detail/demangling/demangle_symbol.hpp | 108 ---- .../boost/dll/detail/demangling/itanium.hpp | 326 ---------- .../demangling/mangled_storage_base.hpp | 120 ---- .../boost/dll/detail/demangling/msvc.hpp | 439 -------------- .../boost/dll/detail/elf_info.hpp | 285 --------- .../boost/dll/detail/get_mem_fn_type.hpp | 40 -- .../dll/detail/import_mangled_helpers.hpp | 290 --------- .../boost/dll/detail/macho_info.hpp | 321 ---------- .../boost_1_65_0/boost/dll/detail/pe_info.hpp | 430 -------------- .../dll/detail/posix/path_from_handle.hpp | 169 ------ .../detail/posix/program_location_impl.hpp | 140 ----- .../dll/detail/posix/shared_library_impl.hpp | 215 ------- .../boost/dll/detail/system_error.hpp | 56 -- .../boost/dll/detail/type_info.hpp | 83 --- .../dll/detail/windows/path_from_handle.hpp | 62 -- .../detail/windows/shared_library_impl.hpp | 177 ------ .../boost/dll/detail/x_info_interface.hpp | 32 - .../boost_1_65_0/boost/dll/import.hpp | 277 --------- .../boost_1_65_0/boost/dll/import_class.hpp | 558 ------------------ .../boost_1_65_0/boost/dll/import_mangled.hpp | 309 ---------- .../boost_1_65_0/boost/dll/library_info.hpp | 181 ------ .../boost/dll/runtime_symbol_info.hpp | 237 -------- .../boost_1_65_0/boost/dll/shared_library.hpp | 550 ----------------- .../boost/dll/shared_library_load_mode.hpp | 249 -------- .../boost_1_65_0/boost/dll/smart_library.hpp | 462 --------------- dbms/src/Dictionaries/CatBoostModel.cpp | 4 +- 29 files changed, 2 insertions(+), 6709 deletions(-) delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/alias.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/import.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp delete mode 100644 contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp diff --git a/contrib/libboost/boost_1_65_0/boost/dll/alias.hpp b/contrib/libboost/boost_1_65_0/boost/dll/alias.hpp deleted file mode 100644 index b8ee45ffa33..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/alias.hpp +++ /dev/null @@ -1,264 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_ALIAS_HPP -#define BOOST_DLL_ALIAS_HPP - -#include -#include -#include -#include -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -/// \file boost/dll/alias.hpp -/// \brief Includes alias methods and macro. You can include this header or -/// boost/dll/shared_library.hpp to reduce dependencies -/// in case you do not use the refcountable functions. - -namespace boost { namespace dll { - -#ifdef BOOST_DLL_DOXYGEN -/// Define this macro to explicitly specify translation unit in which alias must be instantiated. -/// See section 'Limitations' for more info. You may find usage examples in source codes of almost each tutorial. -/// Must be used in code, when \forcedmacrolink{BOOST_DLL_FORCE_NO_WEAK_EXPORTS} is defined -#define BOOST_DLL_FORCE_ALIAS_INSTANTIATION - -/// Define this macro to disable exporting weak symbols and start using the \forcedmacrolink{BOOST_DLL_FORCE_ALIAS_INSTANTIATION}. -/// This may be usefull for working around linker problems or to test your program for compatability with linkers that do not support export of weak symbols. -#define BOOST_DLL_FORCE_NO_WEAK_EXPORTS -#endif - -#if BOOST_COMP_MSVC || (BOOST_COMP_INTEL && BOOST_OS_WINDOWS) - -#define BOOST_DLL_SELECTANY __declspec(selectany) - -#define BOOST_DLL_SECTION(SectionName, Permissions) \ - BOOST_STATIC_ASSERT_MSG( \ - sizeof(#SectionName) < 10, \ - "Some platforms require section names to be at most 8 bytest" \ - ); \ - __pragma(section(#SectionName, Permissions)) __declspec(allocate(#SectionName)) \ - /**/ - -#else // #if BOOST_COMP_MSVC - - -#if BOOST_OS_WINDOWS || BOOST_OS_ANDROID || BOOST_COMP_IBM -// There are some problems with mixing `__dllexport__` and `weak` using MinGW -// See https://sourceware.org/bugzilla/show_bug.cgi?id=17480 -// -// Android had an issue with exporting weak symbols -// https://code.google.com/p/android/issues/detail?id=70206 -#define BOOST_DLL_SELECTANY -#else // #if BOOST_OS_WINDOWS -/*! -* \brief Macro that allows linker to select any occurrence of this symbol instead of -* failing with 'multiple definitions' error at linktime. -* -* This macro does not work on Android, IBM XL C/C++ and MinGW+Windows -* because of linker problems with exporting weak symbols -* (See https://code.google.com/p/android/issues/detail?id=70206, https://sourceware.org/bugzilla/show_bug.cgi?id=17480) -*/ -#define BOOST_DLL_SELECTANY __attribute__((weak)) -#endif // #if BOOST_OS_WINDOWS - -// TODO: improve section permissions using following info: -// http://stackoverflow.com/questions/6252812/what-does-the-aw-flag-in-the-section-attribute-mean - -#if !BOOST_OS_MACOS && !BOOST_OS_IOS -/*! -* \brief Macro that puts symbol to a specific section. On MacOS all the sections are put into "__DATA" segment. -* \param SectionName Name of the section. Must be a valid C identifier without quotes not longer than 8 bytes. -* \param Permissions Can be "read" or "write" (without quotes!). -*/ -#define BOOST_DLL_SECTION(SectionName, Permissions) \ - BOOST_STATIC_ASSERT_MSG( \ - sizeof(#SectionName) < 10, \ - "Some platforms require section names to be at most 8 bytest" \ - ); \ - __attribute__ ((section (#SectionName))) \ - /**/ -#else // #if !BOOST_OS_MACOS && !BOOST_OS_IOS - -#define BOOST_DLL_SECTION(SectionName, Permissions) \ - BOOST_STATIC_ASSERT_MSG( \ - sizeof(#SectionName) < 10, \ - "Some platforms require section names to be at most 8 bytest" \ - ); \ - __attribute__ ((section ( "__DATA," #SectionName))) \ - /**/ - -#endif // #if #if !BOOST_OS_MACOS && !BOOST_OS_IOS - -#endif // #if BOOST_COMP_MSVC - - -// Alias - is just a variable that pointers to original data -// -// A few attempts were made to avoid additional indirection: -// 1) -// // Does not work on Windows, work on Linux -// extern "C" BOOST_SYMBOL_EXPORT void AliasName() { -// reinterpret_cast(Function)(); -// } -// -// 2) -// // Does not work on Linux (changes permissions of .text section and produces incorrect DSO) -// extern "C" BOOST_SYMBOL_EXPORT void* __attribute__ ((section(".text#"))) -// func_ptr = *reinterpret_cast(&foo::bar); -// -// 3) // requires mangled name of `Function` -// // AliasName() __attribute__ ((weak, alias ("Function"))) -// -// // hard to use -// `#pragma comment(linker, "/alternatename:_pWeakValue=_pDefaultWeakValue")` - -/*! -* \brief Makes an alias name for exported function or variable. -* -* This macro is useful in cases of long mangled C++ names. For example some `void boost::foo(std::sting)` -* function name will change to something like `N5boostN3foosE` after mangling. -* Importing function by `N5boostN3foosE` name does not looks user friendly, especially assuming the fact -* that different compilers have different mangling schemes. AliasName is the name that won't be mangled -* and can be used as a portable import name. -* -* -* Can be used in any namespace, including global. FunctionOrVar must be fully qualified, -* so that address of it could be taken. Multiple different aliases for a single variable/function -* are allowed. -* -* Make sure that AliasNames are unique per library/executable. Functions or variables -* in global namespace must not have names same as AliasNames. -* -* Same AliasName in different translation units must point to the same FunctionOrVar. -* -* Puts all the aliases into the \b "boostdll" read only section of the binary. Equal to -* \forcedmacrolink{BOOST_DLL_ALIAS_SECTIONED}(FunctionOrVar, AliasName, boostdll). -* -* \param FunctionOrVar Function or variable for which an alias must be made. -* \param AliasName Name of the alias. Must be a valid C identifier. -* -* \b Example: -* \code -* namespace foo { -* void bar(std::string&); -* -* BOOST_DLL_ALIAS(foo::bar, foo_bar) -* } -* -* BOOST_DLL_ALIAS(foo::bar, foo_bar_another_alias_name) -* \endcode -* -* \b See: \forcedmacrolink{BOOST_DLL_ALIAS_SECTIONED} for making alias in a specific section. -*/ -#define BOOST_DLL_ALIAS(FunctionOrVar, AliasName) \ - BOOST_DLL_ALIAS_SECTIONED(FunctionOrVar, AliasName, boostdll) \ - /**/ - - -#if ((BOOST_COMP_GNUC && BOOST_OS_WINDOWS) || BOOST_OS_ANDROID || BOOST_COMP_IBM || defined(BOOST_DLL_FORCE_NO_WEAK_EXPORTS)) \ - && !defined(BOOST_DLL_FORCE_ALIAS_INSTANTIATION) && !defined(BOOST_DLL_DOXYGEN) - -#define BOOST_DLL_ALIAS_SECTIONED(FunctionOrVar, AliasName, SectionName) \ - namespace _autoaliases { \ - extern "C" BOOST_SYMBOL_EXPORT const void *AliasName; \ - } /* namespace _autoaliases */ \ - /**/ - -#define BOOST_DLL_AUTO_ALIAS(FunctionOrVar) \ - namespace _autoaliases { \ - extern "C" BOOST_SYMBOL_EXPORT const void *FunctionOrVar; \ - } /* namespace _autoaliases */ \ - /**/ -#else -// Note: we can not use `aggressive_ptr_cast` here, because in that case GCC applies -// different permissions to the section and it causes Segmentation fault. -// Note: we can not use `boost::addressof()` here, because in that case GCC -// may optimize away the FunctionOrVar instance and we'll get a pointer to unexisting symbol. -/*! -* \brief Same as \forcedmacrolink{BOOST_DLL_ALIAS} but puts alias name into the user specified section. -* -* \param FunctionOrVar Function or variable for which an alias must be made. -* \param AliasName Name of the alias. Must be a valid C identifier. -* \param SectionName Name of the section. Must be a valid C identifier without quotes not longer than 8 bytes. -* -* \b Example: -* \code -* namespace foo { -* void bar(std::string&); -* -* BOOST_DLL_ALIAS_SECTIONED(foo::bar, foo_bar, sect_1) // section "sect_1" now exports "foo_bar" -* } -* \endcode -* -*/ -#define BOOST_DLL_ALIAS_SECTIONED(FunctionOrVar, AliasName, SectionName) \ - namespace _autoaliases { \ - extern "C" BOOST_SYMBOL_EXPORT const void *AliasName; \ - BOOST_DLL_SECTION(SectionName, read) BOOST_DLL_SELECTANY \ - const void * AliasName = reinterpret_cast(reinterpret_cast( \ - &FunctionOrVar \ - )); \ - } /* namespace _autoaliases */ \ - /**/ - -/*! -* \brief Exports variable or function with unmangled alias name. -* -* This macro is useful in cases of long mangled C++ names. For example some `void boost::foo(std::sting)` -* function name will change to something like `N5boostN3foosE` after mangling. -* Importing function by `N5boostN3foosE` name does not looks user friendly, especially assuming the fact -* that different compilers have different mangling schemes.* -* -* Must be used in scope where FunctionOrVar declared. FunctionOrVar must be a valid C name, which means that -* it must not contain `::`. -* -* Functions or variables -* in global namespace must not have names same as FunctionOrVar. -* -* Puts all the aliases into the \b "boostdll" read only section of the binary. Almost same as -* \forcedmacrolink{BOOST_DLL_ALIAS}(FunctionOrVar, FunctionOrVar). -* -* \param FunctionOrVar Function or variable for which an unmangled alias must be made. -* -* \b Example: -* \code -* namespace foo { -* void bar(std::string&); -* BOOST_DLL_AUTO_ALIAS(bar) -* } -* -* \endcode -* -* \b See: \forcedmacrolink{BOOST_DLL_ALIAS} for making an alias with different names. -*/ - -#define BOOST_DLL_AUTO_ALIAS(FunctionOrVar) \ - namespace _autoaliases { \ - BOOST_DLL_SELECTANY const void * dummy_ ## FunctionOrVar \ - = reinterpret_cast(reinterpret_cast( \ - &FunctionOrVar \ - )); \ - extern "C" BOOST_SYMBOL_EXPORT const void *FunctionOrVar; \ - BOOST_DLL_SECTION(boostdll, read) BOOST_DLL_SELECTANY \ - const void * FunctionOrVar = dummy_ ## FunctionOrVar; \ - } /* namespace _autoaliases */ \ - /**/ - - -#endif - - -}} // namespace boost::dll - - -#endif // BOOST_DLL_ALIAS_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp deleted file mode 100644 index 8d1ecc112bc..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/aggressive_ptr_cast.hpp +++ /dev/null @@ -1,135 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015-2017 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_AGGRESSIVE_PTR_CAST_HPP -#define BOOST_DLL_DETAIL_AGGRESSIVE_PTR_CAST_HPP - -#include -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include // std::memcpy - -#if defined(__GNUC__) && defined(__GNUC_MINOR__) && (__GNUC__ * 100 + __GNUC_MINOR__ > 301) -# pragma GCC system_header -#endif - -namespace boost { namespace dll { namespace detail { - -// GCC warns when reinterpret_cast between function pointer and object pointer occur. -// This method suppress the warnings and ensures that such casts are safe. -template -BOOST_FORCEINLINE typename boost::disable_if_c::value || boost::is_reference::value || boost::is_member_pointer::value, To>::type - aggressive_ptr_cast(From v) BOOST_NOEXCEPT -{ - BOOST_STATIC_ASSERT_MSG( - boost::is_pointer::value && boost::is_pointer::value, - "`agressive_ptr_cast` function must be used only for pointer casting." - ); - - BOOST_STATIC_ASSERT_MSG( - boost::is_void< typename boost::remove_pointer::type >::value - || boost::is_void< typename boost::remove_pointer::type >::value, - "`agressive_ptr_cast` function must be used only for casting to or from void pointers." - ); - - BOOST_STATIC_ASSERT_MSG( - sizeof(v) == sizeof(To), - "Pointer to function and pointer to object differ in size on your platform." - ); - - return reinterpret_cast(v); -} - -#ifdef BOOST_MSVC -# pragma warning(push) -# pragma warning(disable: 4172) // "returning address of local variable or temporary" but **v is not local! -#endif - -template -BOOST_FORCEINLINE typename boost::disable_if_c::value || boost::is_member_pointer::value, To>::type - aggressive_ptr_cast(From v) BOOST_NOEXCEPT -{ - BOOST_STATIC_ASSERT_MSG( - boost::is_pointer::value, - "`agressive_ptr_cast` function must be used only for pointer casting." - ); - - BOOST_STATIC_ASSERT_MSG( - boost::is_void< typename boost::remove_pointer::type >::value, - "`agressive_ptr_cast` function must be used only for casting to or from void pointers." - ); - - BOOST_STATIC_ASSERT_MSG( - sizeof(v) == sizeof(typename boost::remove_reference::type*), - "Pointer to function and pointer to object differ in size on your platform." - ); - return static_cast( - **reinterpret_cast::type**>( - v - ) - ); -} - -#ifdef BOOST_MSVC -# pragma warning(pop) -#endif - -template -BOOST_FORCEINLINE typename boost::disable_if_c::value || boost::is_member_pointer::value, To>::type - aggressive_ptr_cast(From v) BOOST_NOEXCEPT -{ - BOOST_STATIC_ASSERT_MSG( - boost::is_pointer::value, - "`agressive_ptr_cast` function must be used only for pointer casting." - ); - - BOOST_STATIC_ASSERT_MSG( - boost::is_void< typename boost::remove_pointer::type >::value, - "`agressive_ptr_cast` function must be used only for casting to or from void pointers." - ); - - To res = 0; - std::memcpy(&res, &v, sizeof(From)); - return res; -} - -template -BOOST_FORCEINLINE typename boost::disable_if_c::value || !boost::is_member_pointer::value, To>::type - aggressive_ptr_cast(From /* v */) BOOST_NOEXCEPT -{ - BOOST_STATIC_ASSERT_MSG( - boost::is_pointer::value, - "`agressive_ptr_cast` function must be used only for pointer casting." - ); - - BOOST_STATIC_ASSERT_MSG( - boost::is_void< typename boost::remove_pointer::type >::value, - "`agressive_ptr_cast` function must be used only for casting to or from void pointers." - ); - - BOOST_STATIC_ASSERT_MSG( - !sizeof(From), - "Casting from member pointers to void pointer is not implemnted in `agressive_ptr_cast`." - ); - - return 0; -} - -}}} // boost::dll::detail - -#endif // BOOST_DLL_DETAIL_AGGRESSIVE_PTR_CAST_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp deleted file mode 100644 index 3da8395ff5a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/ctor_dtor.hpp +++ /dev/null @@ -1,192 +0,0 @@ -// Copyright 2016 Klemens Morgenstern, Antony Polukhin -// -// 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) - -// For more information, see http://www.boost.org - -#ifndef BOOST_DLL_DETAIL_CTOR_DTOR_HPP_ -#define BOOST_DLL_DETAIL_CTOR_DTOR_HPP_ - -#include -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#include -#include - -#if defined(BOOST_MSVC) || defined(BOOST_MSVC_VER) -# include -#else -# include -#endif - - -namespace boost { namespace dll { namespace detail { - -/*! - * This class stores a constructor. - * - * In some compilers there are several constructors in code, which may include an allocating one. - * This can be used if the imported class shall be put on the heap, which is why the class provied both types. - */ -template -struct constructor; - -template -struct constructor { - typedef typename detail::get_mem_fn_type::mem_fn standard_t; - typedef Class*(*allocating_t)(Args...); - - - //! The standard, i.e. not allocating constructor. @warning May differ with the compiler. Use @ref constructor::call_standard instead. - standard_t standard; - //! The allocating constructor. @warning May differ with the compiler. Use @ref constructor::call_allocating instead. - allocating_t allocating; - - //! Call the standard contructor - void call_standard (Class * const ptr, Args...args){ (ptr->*standard)(static_cast(args)...); } - - //! Call the deleting destructor - Class * call_allocating(Args...args){ return allocating(static_cast(args)...); } - - - //! True if a allocating constructor could be loaded. - bool has_allocating() const { return allocating != nullptr; } - - //! True if a standard constructor could be loaded. - bool has_standard() const { return standard != nullptr; } - - //! False if neither the allocating nor the standard constructor is available. - bool is_empty() const { return (allocating == nullptr) && (standard == nullptr) ; } - - constructor() = delete; - constructor(const constructor &) = default; - - explicit constructor(standard_t standard, allocating_t allocating = nullptr) - : standard(standard) - , allocating(allocating) - {} -}; - - - -template -struct destructor { -#if !defined(_WIN32) - typedef void(*type)(Class* const); -#elif !defined(_WIN64) - typedef void(__thiscall * type)(Class* const); -#else - typedef void(__cdecl * type)(Class* const); -#endif - - typedef type standard_t; - typedef type deleting_t; - - //! The standard, i.e. not deleting destructor. @warning May differ with the compiler. Use @ref destructor::call_standard instead. - standard_t standard; - //! The deleting destructor. @warning May differ with the compiler. Use @ref destructor::call_deallocating instead. - deleting_t deleting; - - //! Call the standard contructor - void call_standard(Class * const ptr){ standard(ptr); } - - //! Call the deleting destructor - void call_deleting(Class * const ptr){ deleting(ptr); } - - //! True if a deleting destructor could be loaded. - bool has_deleting() const { return deleting != nullptr; } - - //! True if a standard destructor could be loaded. - bool has_standard() const { return standard != nullptr; } - - //! False if neither the deleting nor the standard destructor is available. - bool is_empty() const { return (deleting == nullptr) && (standard == nullptr) ; } - destructor() = delete; - - //! Copy destructor. - destructor(const destructor &) = default; - - //! Construct it from both the standard destructor and the allocating destructor - explicit destructor(const standard_t &standard, const deleting_t &deleting = nullptr) - : standard(standard) - , deleting(deleting) - {} -}; - -#if defined(BOOST_MSVC) || defined(BOOST_MSVC_VER) -template -constructor load_ctor(Lib & lib, const mangled_storage_impl::ctor_sym & ct) { - typedef typename constructor::standard_t standard_t; - standard_t ctor = lib.template get(ct); - return constructor(ctor); -} - -template -destructor load_dtor(Lib & lib, const mangled_storage_impl::dtor_sym & dt) { - typedef typename destructor::standard_t standard_t; - //@apolukhin That does NOT work this way with MSVC-14 x32 via memcpy. The x64 is different. - //standard_t dtor = &lib.template get< typename boost::remove_pointer::type >(dt); - void * buf = &lib.template get(dt); - standard_t dtor; - std::memcpy(&dtor, &buf, sizeof(dtor)); - return destructor(dtor); -} - -#else - -template -constructor load_ctor(Lib & lib, const mangled_storage_impl::ctor_sym & ct) { - typedef typename constructor::standard_t stand; - typedef typename constructor::allocating_t alloc; - - stand s = nullptr; - alloc a = nullptr; - - //see here for the abi http://mentorembedded.github.io/cxx-abi/abi.html#mangling-special-ctor-dtor - - if (!ct.C1.empty()) - { - //the only way this works on mingw/win. - //For some reason there is always an 0xA in the following poniter, which screws with the this pointer. - void *buf = &lib.template get(ct.C1); - std::memcpy(&s, &buf, sizeof(void*)); - } - if (!ct.C3.empty()) - { - void *buf = &lib.template get(ct.C3); - std::memcpy(&a, &buf, sizeof(void*)); - } - - return constructor(s,a); -} - -template -destructor load_dtor(Lib & lib, const mangled_storage_impl::dtor_sym & dt) { - typedef typename destructor::standard_t stand; - typedef typename destructor::deleting_t delet; - - stand s = nullptr; - delet d = nullptr; - - //see here for the abi http://mentorembedded.github.io/cxx-abi/abi.html#mangling-special-ctor-dtor - if (!dt.D1.empty()) { - s = &lib.template get< typename boost::remove_pointer::type >(dt.D1); - } - - if (!dt.D0.empty()) { - d = &lib.template get< typename boost::remove_pointer::type >(dt.D0); - } - - return destructor(s,d); - -} - -#endif - -}}} // namespace boost::dll::detail - -#endif /* BOOST_DLL_DETAIL_CTOR_DTOR_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp deleted file mode 100644 index 30525f6b317..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/demangle_symbol.hpp +++ /dev/null @@ -1,108 +0,0 @@ -// Copyright 2015 Klemens Morgenstern -// -// This file provides a demangling for function names, i.e. entry points of a dll. -// -// Distributed under the Boost Software License, Version 1.0. -// See http://www.boost.org/LICENSE_1_0.txt - -#ifndef BOOST_DLL_DEMANGLE_SYMBOL_HPP_ -#define BOOST_DLL_DEMANGLE_SYMBOL_HPP_ - -#include -#include -#include - -#if defined(BOOST_MSVC) || defined(BOOST_MSVC_FULL_VER) - -namespace boost -{ -namespace dll -{ -namespace detail -{ - -typedef void * (__cdecl * allocation_function)(std::size_t); -typedef void (__cdecl * free_function)(void *); - -extern "C" char* __unDName( char* outputString, - const char* name, - int maxStringLength, // Note, COMMA is leading following optional arguments - allocation_function pAlloc, - free_function pFree, - unsigned short disableFlags - ); - - -inline std::string demangle_symbol(const char *mangled_name) -{ - - allocation_function alloc = [](std::size_t size){return static_cast(new char[size]);}; - free_function free_f = [](void* p){delete [] static_cast(p);}; - - - - std::unique_ptr name { __unDName( - nullptr, - mangled_name, - 0, - alloc, - free_f, - static_cast(0))}; - - return std::string(name.get()); -} -inline std::string demangle_symbol(const std::string& mangled_name) -{ - return demangle_symbol(mangled_name.c_str()); -} - - -}}} -#else - -#include - -namespace boost -{ -namespace dll -{ -namespace detail -{ - -inline std::string demangle_symbol(const char *mangled_name) -{ - - if (*mangled_name == '_') - { - //because it start's with an underline _ - auto dm = boost::core::demangle(mangled_name); - if (!dm.empty()) - return dm; - else - return (mangled_name); - } - - //could not demangled - return ""; - - -} - -//for my personal convinience -inline std::string demangle_symbol(const std::string& mangled_name) -{ - return demangle_symbol(mangled_name.c_str()); -} - - -} -namespace experimental -{ -using ::boost::dll::detail::demangle_symbol; -} - -}} - -#endif - -#endif /* BOOST_DEMANGLE_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp deleted file mode 100644 index 2701a0390b8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/itanium.hpp +++ /dev/null @@ -1,326 +0,0 @@ -// Copyright 2016 Klemens Morgenstern -// -// 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) - -#ifndef BOOST_DLL_DETAIL_DEMANGLING_ITANIUM_HPP_ -#define BOOST_DLL_DETAIL_DEMANGLING_ITANIUM_HPP_ - -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace boost { namespace dll { namespace detail { - - - -class mangled_storage_impl : public mangled_storage_base -{ - template - struct dummy {}; - - template - std::vector get_func_params(dummy) const - { - return {get_name()...}; - } - template - std::string get_return_type(dummy) const - { - return get_name(); - } -public: - using mangled_storage_base::mangled_storage_base; - struct ctor_sym - { - std::string C1; - std::string C2; - std::string C3; - - bool empty() const - { - return C1.empty() && C2.empty() && C3.empty(); - } - }; - - struct dtor_sym - { - std::string D0; - std::string D1; - std::string D2; - bool empty() const - { - return D0.empty() && D1.empty() && D2.empty(); - } - }; - - template - std::string get_variable(const std::string &name) const; - - template - std::string get_function(const std::string &name) const; - - template - std::string get_mem_fn(const std::string &name) const; - - template - ctor_sym get_constructor() const; - - template - dtor_sym get_destructor() const; - - template - std::string get_type_info() const; - - template - std::vector get_related() const; - -}; - - - -namespace parser -{ - - inline std::string const_rule_impl(true_type ) {return " const";} - inline std::string const_rule_impl(false_type) {return "";} - template - std::string const_rule() {using t = is_const::type>; return const_rule_impl(t());} - - inline std::string volatile_rule_impl(true_type ) {return " volatile";} - inline std::string volatile_rule_impl(false_type) {return "";} - template - std::string volatile_rule() {using t = is_volatile::type>; return volatile_rule_impl(t());} - - inline std::string reference_rule_impl(false_type, false_type) {return "";} - inline std::string reference_rule_impl(true_type, false_type) {return "&" ;} - inline std::string reference_rule_impl(false_type, true_type ) {return "&&";} - - - template - std::string reference_rule() {using t_l = is_lvalue_reference; using t_r = is_rvalue_reference; return reference_rule_impl(t_l(), t_r());} - - //it takes a string, because it may be overloaded. - template - std::string type_rule(const std::string & type_name) - { - using namespace std; - - return type_name + - const_rule() + - volatile_rule() + - reference_rule(); - } - - - template - std::string arg_list(const mangled_storage_impl & ms, Return (*)(Arg)) - { - using namespace std; - auto str = ms.get_name(); - return type_rule(str); - } - - template - std::string arg_list(const mangled_storage_impl & ms, Return (*)(First, Second, Args...)) - { - auto st = ms.get_name(); - - using next_type = Return (*)(Second, Args...); - return type_rule(st) + ", " + arg_list(ms, next_type()); - } - - template - std::string arg_list(const mangled_storage_impl &, Return (*)()) - { - return ""; - } -} - - - -template std::string mangled_storage_impl::get_variable(const std::string &name) const -{ - auto found = std::find_if(storage_.begin(), storage_.end(), - [&](const entry& e) {return e.demangled == name;}); - - if (found != storage_.end()) - return found->mangled; - else - return ""; -} - -template std::string mangled_storage_impl::get_function(const std::string &name) const -{ - using func_type = Func*; - - auto matcher = name + '(' + parser::arg_list(*this, func_type()) + ')'; - - auto found = std::find_if(storage_.begin(), storage_.end(), [&](const entry& e) {return e.demangled == matcher;}); - if (found != storage_.end()) - return found->mangled; - else - return ""; - -} - -template -std::string mangled_storage_impl::get_mem_fn(const std::string &name) const -{ - using namespace parser; - - using func_type = Func*; - - std::string cname = get_name(); - - auto matcher = cname + "::" + name + - '(' + parser::arg_list(*this, func_type()) + ')' - + const_rule() + volatile_rule(); - - auto found = std::find_if(storage_.begin(), storage_.end(), [&](const entry& e) {return e.demangled == matcher;}); - - if (found != storage_.end()) - return found->mangled; - else - return ""; - -} - - -template -auto mangled_storage_impl::get_constructor() const -> ctor_sym -{ - using namespace parser; - - using func_type = Signature*; - - std::string ctor_name; // = class_name + "::" + name; - std::string unscoped_cname; //the unscoped class-name - { - auto class_name = get_return_type(dummy()); - auto pos = class_name.rfind("::"); - if (pos == std::string::npos) - { - ctor_name = class_name+ "::" +class_name ; - unscoped_cname = class_name; - } - else - { - unscoped_cname = class_name.substr(pos+2) ; - ctor_name = class_name+ "::" + unscoped_cname; - } - } - - auto matcher = - ctor_name + '(' + parser::arg_list(*this, func_type()) + ')'; - - - std::vector findings; - std::copy_if(storage_.begin(), storage_.end(), - std::back_inserter(findings), [&](const entry& e) {return e.demangled == matcher;}); - - ctor_sym ct; - - for (auto & e : findings) - { - - if (e.mangled.find(unscoped_cname +"C1E") != std::string::npos) - ct.C1 = e.mangled; - else if (e.mangled.find(unscoped_cname +"C2E") != std::string::npos) - ct.C2 = e.mangled; - else if (e.mangled.find(unscoped_cname +"C3E") != std::string::npos) - ct.C3 = e.mangled; - } - return ct; -} - -template -auto mangled_storage_impl::get_destructor() const -> dtor_sym -{ - std::string dtor_name; // = class_name + "::" + name; - std::string unscoped_cname; //the unscoped class-name - { - auto class_name = get_name(); - auto pos = class_name.rfind("::"); - if (pos == std::string::npos) - { - dtor_name = class_name+ "::~" + class_name + "()"; - unscoped_cname = class_name; - } - else - { - unscoped_cname = class_name.substr(pos+2) ; - dtor_name = class_name+ "::~" + unscoped_cname + "()"; - } - } - - auto d0 = unscoped_cname + "D0Ev"; - auto d1 = unscoped_cname + "D1Ev"; - auto d2 = unscoped_cname + "D2Ev"; - - dtor_sym dt; - //this is so simple, i don#t need a predicate - for (auto & s : storage_) - { - //alright, name fits - if (s.demangled == dtor_name) - { - if (s.mangled.find(d0) != std::string::npos) - dt.D0 = s.mangled; - else if (s.mangled.find(d1) != std::string::npos) - dt.D1 = s.mangled; - else if (s.mangled.find(d2) != std::string::npos) - dt.D2 = s.mangled; - - } - } - return dt; - -} - -template -std::string mangled_storage_impl::get_type_info() const -{ - std::string id = "typeinfo for " + get_name(); - - - auto predicate = [&](const mangled_storage_base::entry & e) - { - return e.demangled == id; - }; - - auto found = std::find_if(storage_.begin(), storage_.end(), predicate); - - - if (found != storage_.end()) - return found->mangled; - else - return ""; -} - -template -std::vector mangled_storage_impl::get_related() const -{ - std::vector ret; - auto name = get_name(); - - for (auto & c : storage_) - { - if (c.demangled.find(name) != std::string::npos) - ret.push_back(c.demangled); - } - - return ret; -} - -}}} - - -#endif /* BOOST_DLL_DETAIL_DEMANGLING_ITANIUM_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp deleted file mode 100644 index b9cfd64fa26..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/mangled_storage_base.hpp +++ /dev/null @@ -1,120 +0,0 @@ -// Copyright 2016 Klemens Morgenstern -// -// 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) - -#ifndef BOOST_DLL_DETAIL_MANGLE_STORAGE_BASE_HPP_ -#define BOOST_DLL_DETAIL_MANGLE_STORAGE_BASE_HPP_ - -#include -#include -#include -#include -#include -#include -#include - -namespace boost { namespace dll { namespace detail { - -///stores the mangled names with the demangled name. -struct mangled_storage_base -{ - struct entry - { - std::string mangled; - std::string demangled; - entry() = default; - entry(const std::string & m, const std::string &d) : mangled(m), demangled(d) {} - entry(const entry&) = default; - entry(entry&&) = default; - entry &operator= (const entry&) = default; - entry &operator= (entry&&) = default; - }; -protected: - std::vector storage_; - ///if a unknown class is imported it can be overloaded by this type - std::map aliases_; -public: - void assign(const mangled_storage_base & storage) - { - aliases_ = storage.aliases_; - storage_ = storage.storage_; - } - void swap( mangled_storage_base & storage) - { - aliases_.swap(storage.aliases_); - storage_.swap(storage.storage_); - } - void clear() - { - storage_.clear(); - aliases_.clear(); - } - std::vector & get_storage() {return storage_;}; - template - std::string get_name() const - { - using boost::typeindex::ctti_type_index; - auto tx = ctti_type_index::type_id(); - auto val = (aliases_.count(tx) > 0) ? aliases_.at(tx) : tx.pretty_name(); - return val; - } - - mangled_storage_base() = default; - mangled_storage_base(mangled_storage_base&&) = default; - mangled_storage_base(const mangled_storage_base&) = default; - - mangled_storage_base(const std::vector & symbols) { add_symbols(symbols);} - - explicit mangled_storage_base(library_info & li) : mangled_storage_base(li.symbols()) {} - - explicit mangled_storage_base( - const boost::filesystem::path& library_path, - bool throw_if_not_native_format = true) - : mangled_storage_base(library_info(library_path, throw_if_not_native_format).symbols()) - { - - } - - void load(library_info & li) { storage_.clear(); add_symbols(li.symbols()); }; - void load(const boost::filesystem::path& library_path, - bool throw_if_not_native_format = true) - { - storage_.clear(); - add_symbols(library_info(library_path, throw_if_not_native_format).symbols()); - }; - - /*! Allows do add a class as alias, if the class imported is not known - * in this binary. - * @tparam Alias The Alias type - * @param The name to create the alias for. - * - * @note There can be multiple aliases, this is on purpose. - */ - template void add_alias(const std::string& name) - { - aliases_.emplace( - boost::typeindex::ctti_type_index::type_id(), - name - ); - } - void add_symbols(const std::vector & symbols) - { - for (auto & sym : symbols) - { - auto dm = demangle_symbol(sym); - if (!dm.empty()) - storage_.emplace_back(sym, dm); - else - storage_.emplace_back(sym, sym); - } - } - - -}; - - -}}} - -#endif /* BOOST_DLL_DETAIL_MANGLE_STORAGE_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp deleted file mode 100644 index c1abd6a185c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/demangling/msvc.hpp +++ /dev/null @@ -1,439 +0,0 @@ -// Copyright 2016 Klemens Morgenstern -// -// 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) - -#ifndef BOOST_DLL_DETAIL_DEMANGLING_MSVC_HPP_ -#define BOOST_DLL_DETAIL_DEMANGLING_MSVC_HPP_ - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace boost { namespace dll { namespace detail { - -class mangled_storage_impl : public mangled_storage_base -{ - template - struct dummy {}; - - template - std::vector get_func_params(dummy) const - { - return {get_name()...}; - } - template - std::string get_return_type(dummy) const - { - return get_name(); - } - //function to remove preceeding 'class ' or 'struct ' if the are given in this format. - - inline static void trim_typename(std::string & val); -public: - using ctor_sym = std::string; - using dtor_sym = std::string; - - using mangled_storage_base::mangled_storage_base; - - template - std::string get_variable(const std::string &name) const; - - template - std::string get_function(const std::string &name) const; - - template - std::string get_mem_fn(const std::string &name) const; - - template - ctor_sym get_constructor() const; - - template - dtor_sym get_destructor() const; - - template //overload, does not need to virtual. - std::string get_name() const - { - auto nm = mangled_storage_base::get_name(); - trim_typename(nm); - return nm; - } - - template - std::string get_vtable() const; - - template - std::vector get_related() const; - -}; - -void mangled_storage_impl::trim_typename(std::string & val) -{ - //remove preceeding class or struct, because you might want to use a struct as class, et vice versa - if (val.size() >= 6) - { - using namespace std; - static constexpr char class_ [7] = "class "; - static constexpr char struct_[8] = "struct "; - - if (equal(begin(class_), end(class_)-1, val.begin())) //aklright, starts with 'class ' - val.erase(0, 6); - else if (val.size() >= 7) - if (equal(begin(struct_), end(struct_)-1, val.begin())) - val.erase(0, 7); - } -} - - -namespace parser -{ - namespace x3 = spirit::x3; - - auto ptr_rule_impl(std::integral_constant) - { - return -((-x3::space) >> "__ptr32"); - } - auto ptr_rule_impl(std::integral_constant) - { - return -((-x3::space) >> "__ptr64"); - } - - auto ptr_rule() { return ptr_rule_impl(std::integral_constant());} - - auto const visibility = ("public:" | x3::lit("protected:") | "private:"); - auto const virtual_ = x3::space >> "virtual"; - auto const static_ = x3::space >> x3::lit("static") ; - - auto const_rule_impl(true_type ) {return x3::space >> "const";}; - auto const_rule_impl(false_type) {return x3::eps;}; - template - auto const_rule() {using t = is_const::type>; return const_rule_impl(t());} - - auto volatile_rule_impl(true_type ) {return x3::space >> "volatile";}; - auto volatile_rule_impl(false_type) {return x3::eps;}; - template - auto volatile_rule() {using t = is_volatile::type>; return volatile_rule_impl(t());} - - - auto inv_const_rule_impl(true_type ) {return "const" >> x3::space ;}; - auto inv_const_rule_impl(false_type) {return x3::eps;}; - template - auto inv_const_rule() {using t = is_const::type>; return inv_const_rule_impl(t());} - - auto inv_volatile_rule_impl(true_type ) {return "volatile" >> x3::space;}; - auto inv_volatile_rule_impl(false_type) {return x3::eps;}; - template - auto inv_volatile_rule() {using t = is_volatile::type>; return inv_volatile_rule_impl(t());} - - - auto reference_rule_impl(false_type, false_type) {return x3::eps;} - auto reference_rule_impl(true_type, false_type) {return x3::space >>"&" ;} - auto reference_rule_impl(false_type, true_type ) {return x3::space >>"&&" ;} - - - template - auto reference_rule() {using t_l = is_lvalue_reference; using t_r = is_rvalue_reference; return reference_rule_impl(t_l(), t_r());} - - auto const class_ = ("class" | x3::lit("struct")); - - //it takes a string, because it may be overloaded. - template - auto type_rule(const std::string & type_name) - { - using namespace std; - - return -(class_ >> x3::space)>> x3::string(type_name) >> - const_rule() >> - volatile_rule() >> - reference_rule() >> - ptr_rule(); - } - template<> - auto type_rule(const std::string &) { return x3::string("void"); }; - - auto const cdecl_ = "__cdecl" >> x3::space; - auto const stdcall = "__stdcall" >> x3::space; -#if defined(_WIN64)//seems to be necessary by msvc 14-x64 - auto const thiscall = "__cdecl" >> x3::space; -#else - auto const thiscall = "__thiscall" >> x3::space; -#endif - - template - auto arg_list(const mangled_storage_impl & ms, Return (*)(Arg)) - { - using namespace std; - - return type_rule(ms.get_name()); - } - - template - auto arg_list(const mangled_storage_impl & ms, Return (*)(First, Second, Args...)) - { - - using next_type = Return (*)(Second, Args...); - return type_rule(ms.get_name()) >> x3::char_(',') >> arg_list(ms, next_type()); - } - - template - auto arg_list(const mangled_storage_impl& /*ms*/, Return (*)()) - { - return x3::string("void"); - } -} - - -template std::string mangled_storage_impl::get_variable(const std::string &name) const -{ - using namespace std; - using namespace boost; - - namespace x3 = spirit::x3; - using namespace parser; - - auto type_name = get_name(); - - auto matcher = - -(visibility >> static_ >> x3::space) >> //it may be a static class-member - parser::type_rule(type_name) >> x3::space >> - name; - - auto predicate = [&](const mangled_storage_base::entry & e) - { - if (e.demangled == name)//maybe not mangled, - return true; - - auto itr = e.demangled.begin(); - auto end = e.demangled.end(); - auto res = x3::parse(itr, end, matcher); - return res && (itr == end); - }; - - auto found = std::find_if(storage_.begin(), storage_.end(), predicate); - - if (found != storage_.end()) - return found->mangled; - else - return ""; -} - -template std::string mangled_storage_impl::get_function(const std::string &name) const -{ - namespace x3 = spirit::x3; - using namespace parser; - using func_type = Func*; - using return_type = typename function_traits::result_type; - std::string return_type_name = get_name(); - - - auto matcher = - -(visibility >> static_ >> x3::space) >> //it may be a static class-member, which does however not have the static attribute. - parser::type_rule(return_type_name) >> x3::space >> - cdecl_ >> //cdecl declaration for methods. stdcall cannot be - name >> x3::lit('(') >> parser::arg_list(*this, func_type()) >> x3::lit(')') >> parser::ptr_rule(); - - - auto predicate = [&](const mangled_storage_base::entry & e) - { - if (e.demangled == name)//maybe not mangled, - return true; - - auto itr = e.demangled.begin(); - auto end = e.demangled.end(); - auto res = x3::parse(itr, end, matcher); - - return res && (itr == end); - }; - - auto found = std::find_if(storage_.begin(), storage_.end(), predicate); - - if (found != storage_.end()) - return found->mangled; - else - return ""; - -} - -template -std::string mangled_storage_impl::get_mem_fn(const std::string &name) const -{ - namespace x3 = spirit::x3; - using namespace parser; - using func_type = Func*; - using return_type = typename function_traits::result_type; - auto return_type_name = get_name(); - - - auto cname = get_name(); - - auto matcher = - visibility >> -virtual_ >> x3::space >> - parser::type_rule(return_type_name) >> x3::space >> - thiscall >> //cdecl declaration for methods. stdcall cannot be - cname >> "::" >> name >> - x3::lit('(') >> parser::arg_list(*this, func_type()) >> x3::lit(')') >> - inv_const_rule() >> inv_volatile_rule() >> parser::ptr_rule(); - - auto predicate = [&](const mangled_storage_base::entry & e) - { - auto itr = e.demangled.begin(); - auto end = e.demangled.end(); - auto res = x3::parse(itr, end, matcher); - - return res && (itr == end); - }; - - auto found = std::find_if(storage_.begin(), storage_.end(), predicate); - - if (found != storage_.end()) - return found->mangled; - else - return ""; -} - - -template -auto mangled_storage_impl::get_constructor() const -> ctor_sym -{ - namespace x3 = spirit::x3; - using namespace parser; - - using func_type = Signature*; - - - std::string ctor_name; // = class_name + "::" + name; - std::string unscoped_cname; //the unscoped class-name - { - auto class_name = get_return_type(dummy()); - auto pos = class_name.rfind("::"); - if (pos == std::string::npos) - { - ctor_name = class_name+ "::" + class_name ; - unscoped_cname = class_name; - } - else - { - unscoped_cname = class_name.substr(pos+2) ; - ctor_name = class_name+ "::" + unscoped_cname; - } - } - - auto matcher = - visibility >> x3::space >> - thiscall >> //cdecl declaration for methods. stdcall cannot be - ctor_name >> - x3::lit('(') >> parser::arg_list(*this, func_type()) >> x3::lit(')') >> parser::ptr_rule(); - - - auto predicate = [&](const mangled_storage_base::entry & e) - { - auto itr = e.demangled.begin(); - auto end = e.demangled.end(); - auto res = x3::parse(itr, end, matcher); - - return res && (itr == end); - }; - - auto f = std::find_if(storage_.begin(), storage_.end(), predicate); - - if (f != storage_.end()) - return f->mangled; - else - return ""; -} - -template -auto mangled_storage_impl::get_destructor() const -> dtor_sym -{ - namespace x3 = spirit::x3; - using namespace parser; - std::string dtor_name; // = class_name + "::" + name; - std::string unscoped_cname; //the unscoped class-name - { - auto class_name = get_name(); - auto pos = class_name.rfind("::"); - if (pos == std::string::npos) - { - dtor_name = class_name+ "::~" + class_name + "(void)"; - unscoped_cname = class_name; - } - else - { - unscoped_cname = class_name.substr(pos+2) ; - dtor_name = class_name+ "::~" + unscoped_cname + "(void)"; - } - } - - auto matcher = - visibility >> -virtual_ >> x3::space >> - thiscall >> //cdecl declaration for methods. stdcall cannot be - dtor_name >> parser::ptr_rule(); - - - auto predicate = [&](const mangled_storage_base::entry & e) - { - auto itr = e.demangled.begin(); - auto end = e.demangled.end(); - auto res = x3::parse(itr, end, matcher); - - return res && (itr == end); - }; - - auto found = std::find_if(storage_.begin(), storage_.end(), predicate); - - - if (found != storage_.end()) - return found->mangled; - else - return ""; -} - -template -std::string mangled_storage_impl::get_vtable() const -{ - std::string id = "const " + get_name() + "::`vftable'"; - - auto predicate = [&](const mangled_storage_base::entry & e) - { - return e.demangled == id; - }; - - auto found = std::find_if(storage_.begin(), storage_.end(), predicate); - - - if (found != storage_.end()) - return found->mangled; - else - return ""; -} - -template -std::vector mangled_storage_impl::get_related() const -{ - std::vector ret; - auto name = get_name(); - - for (auto & c : storage_) - { - if (c.demangled.find(name) != std::string::npos) - ret.push_back(c.demangled); - } - - return ret; -} - - -}}} - - - -#endif /* BOOST_DLL_DETAIL_DEMANGLING_MSVC_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp deleted file mode 100644 index 325df9cb10f..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/elf_info.hpp +++ /dev/null @@ -1,285 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_POSIX_ELF_INFO_HPP -#define BOOST_DLL_DETAIL_POSIX_ELF_INFO_HPP - -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#include -#include -#include - -namespace boost { namespace dll { namespace detail { - -template -struct Elf_Ehdr_template { - unsigned char e_ident[16]; /* Magic number and other info */ - boost::uint16_t e_type; /* Object file type */ - boost::uint16_t e_machine; /* Architecture */ - boost::uint32_t e_version; /* Object file version */ - AddressOffsetT e_entry; /* Entry point virtual address */ - AddressOffsetT e_phoff; /* Program header table file offset */ - AddressOffsetT e_shoff; /* Section header table file offset */ - boost::uint32_t e_flags; /* Processor-specific flags */ - boost::uint16_t e_ehsize; /* ELF header size in bytes */ - boost::uint16_t e_phentsize; /* Program header table entry size */ - boost::uint16_t e_phnum; /* Program header table entry count */ - boost::uint16_t e_shentsize; /* Section header table entry size */ - boost::uint16_t e_shnum; /* Section header table entry count */ - boost::uint16_t e_shstrndx; /* Section header string table index */ -}; - -typedef Elf_Ehdr_template Elf32_Ehdr_; -typedef Elf_Ehdr_template Elf64_Ehdr_; - -template -struct Elf_Shdr_template { - boost::uint32_t sh_name; /* Section name (string tbl index) */ - boost::uint32_t sh_type; /* Section type */ - AddressOffsetT sh_flags; /* Section flags */ - AddressOffsetT sh_addr; /* Section virtual addr at execution */ - AddressOffsetT sh_offset; /* Section file offset */ - AddressOffsetT sh_size; /* Section size in bytes */ - boost::uint32_t sh_link; /* Link to another section */ - boost::uint32_t sh_info; /* Additional section information */ - AddressOffsetT sh_addralign; /* Section alignment */ - AddressOffsetT sh_entsize; /* Entry size if section holds table */ -}; - -typedef Elf_Shdr_template Elf32_Shdr_; -typedef Elf_Shdr_template Elf64_Shdr_; - -template -struct Elf_Sym_template; - -template <> -struct Elf_Sym_template { - typedef boost::uint32_t AddressOffsetT; - - boost::uint32_t st_name; /* Symbol name (string tbl index) */ - AddressOffsetT st_value; /* Symbol value */ - AddressOffsetT st_size; /* Symbol size */ - unsigned char st_info; /* Symbol type and binding */ - unsigned char st_other; /* Symbol visibility */ - boost::uint16_t st_shndx; /* Section index */ -}; - -template <> -struct Elf_Sym_template { - typedef boost::uint64_t AddressOffsetT; - - boost::uint32_t st_name; /* Symbol name (string tbl index) */ - unsigned char st_info; /* Symbol type and binding */ - unsigned char st_other; /* Symbol visibility */ - boost::uint16_t st_shndx; /* Section index */ - AddressOffsetT st_value; /* Symbol value */ - AddressOffsetT st_size; /* Symbol size */ -}; - - -typedef Elf_Sym_template Elf32_Sym_; -typedef Elf_Sym_template Elf64_Sym_; - -template -class elf_info: public x_info_interface { - boost::filesystem::ifstream& f_; - - typedef boost::dll::detail::Elf_Ehdr_template header_t; - typedef boost::dll::detail::Elf_Shdr_template section_t; - typedef boost::dll::detail::Elf_Sym_template symbol_t; - - BOOST_STATIC_CONSTANT(boost::uint32_t, SHT_SYMTAB_ = 2); - BOOST_STATIC_CONSTANT(boost::uint32_t, SHT_STRTAB_ = 3); - - BOOST_STATIC_CONSTANT(unsigned char, STB_LOCAL_ = 0); /* Local symbol */ - BOOST_STATIC_CONSTANT(unsigned char, STB_GLOBAL_ = 1); /* Global symbol */ - BOOST_STATIC_CONSTANT(unsigned char, STB_WEAK_ = 2); /* Weak symbol */ - - /* Symbol visibility specification encoded in the st_other field. */ - BOOST_STATIC_CONSTANT(unsigned char, STV_DEFAULT_ = 0); /* Default symbol visibility rules */ - BOOST_STATIC_CONSTANT(unsigned char, STV_INTERNAL_ = 1); /* Processor specific hidden class */ - BOOST_STATIC_CONSTANT(unsigned char, STV_HIDDEN_ = 2); /* Sym unavailable in other modules */ - BOOST_STATIC_CONSTANT(unsigned char, STV_PROTECTED_ = 3); /* Not preemptible, not exported */ - -public: - static bool parsing_supported(boost::filesystem::ifstream& f) { - const unsigned char magic_bytes[5] = { - 0x7f, 'E', 'L', 'F', sizeof(boost::uint32_t) == sizeof(AddressOffsetT) ? 1 : 2 - }; - - unsigned char ch; - f.seekg(0); - for (std::size_t i = 0; i < sizeof(magic_bytes); ++i) { - f >> ch; - if (ch != magic_bytes[i]) { - return false; - } - } - - return true; - } - - explicit elf_info(boost::filesystem::ifstream& f) BOOST_NOEXCEPT - : f_(f) - {} - - std::vector sections() { - std::vector ret; - std::vector names; - sections_names_raw(names); - - const char* name_begin = &names[0]; - const char* const name_end = name_begin + names.size(); - ret.reserve(header().e_shnum); - do { - ret.push_back(name_begin); - name_begin += ret.back().size() + 1; - } while (name_begin != name_end); - - return ret; - } - -private: - template - inline void read_raw(T& value, std::size_t size = sizeof(T)) const { - f_.read(reinterpret_cast(&value), size); - } - - inline header_t header() { - header_t elf; - - f_.seekg(0); - read_raw(elf); - - return elf; - } - - void sections_names_raw(std::vector& sections) { - const header_t elf = header(); - - section_t section_names_section; - f_.seekg(elf.e_shoff + elf.e_shstrndx * sizeof(section_t)); - read_raw(section_names_section); - - sections.resize(static_cast(section_names_section.sh_size)); - f_.seekg(section_names_section.sh_offset); - read_raw(sections[0], static_cast(section_names_section.sh_size)); - } - - void symbols_text(std::vector& symbols, std::vector& text) { - const header_t elf = header(); - f_.seekg(elf.e_shoff); - - for (std::size_t i = 0; i < elf.e_shnum; ++i) { - section_t section; - read_raw(section); - - if (section.sh_type == SHT_SYMTAB_) { - symbols.resize(static_cast(section.sh_size / sizeof(symbol_t))); - - const boost::filesystem::ifstream::pos_type pos = f_.tellg(); - f_.seekg(section.sh_offset); - read_raw(symbols[0], static_cast(section.sh_size - (section.sh_size % sizeof(symbol_t))) ); - f_.seekg(pos); - } else if (section.sh_type == SHT_STRTAB_) { - text.resize(static_cast(section.sh_size)); - - const boost::filesystem::ifstream::pos_type pos = f_.tellg(); - f_.seekg(section.sh_offset); - read_raw(text[0], static_cast(section.sh_size)); - f_.seekg(pos); - } - } - } - - static bool is_visible(const symbol_t& sym) BOOST_NOEXCEPT { - // `(sym.st_info >> 4) != STB_LOCAL_ && !!sym.st_size` check also workarounds the - // GCC's issue https://sourceware.org/bugzilla/show_bug.cgi?id=13621 - return (sym.st_other & 0x03) == STV_DEFAULT_ && (sym.st_info >> 4) != STB_LOCAL_ && !!sym.st_size; - } - -public: - std::vector symbols() { - std::vector ret; - - std::vector symbols; - std::vector text; - symbols_text(symbols, text); - - ret.reserve(symbols.size()); - for (std::size_t i = 0; i < symbols.size(); ++i) { - if (is_visible(symbols[i])) { - ret.push_back(&text[0] + symbols[i].st_name); - if (ret.back().empty()) { - ret.pop_back(); // Do not show empty names - } - } - } - - return ret; - } - - std::vector symbols(const char* section_name) { - std::vector ret; - - std::size_t index = 0; - std::size_t ptrs_in_section_count = 0; - { - std::vector names; - sections_names_raw(names); - - const header_t elf = header(); - - for (; index < elf.e_shnum; ++index) { - section_t section; - f_.seekg(elf.e_shoff + index * sizeof(section_t)); - read_raw(section); - - if (!std::strcmp(&names[0] + section.sh_name, section_name)) { - if (!section.sh_entsize) { - section.sh_entsize = 1; - } - ptrs_in_section_count = static_cast(section.sh_size / section.sh_entsize); - break; - } - } - } - - std::vector symbols; - std::vector text; - symbols_text(symbols, text); - - if (ptrs_in_section_count < symbols.size()) { - ret.reserve(ptrs_in_section_count); - } else { - ret.reserve(symbols.size()); - } - - for (std::size_t i = 0; i < symbols.size(); ++i) { - if (symbols[i].st_shndx == index && is_visible(symbols[i])) { - ret.push_back(&text[0] + symbols[i].st_name); - if (ret.back().empty()) { - ret.pop_back(); // Do not show empty names - } - } - } - - return ret; - } -}; - -typedef elf_info elf_info32; -typedef elf_info elf_info64; - -}}} // namespace boost::dll::detail - -#endif // BOOST_DLL_DETAIL_POSIX_ELF_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp deleted file mode 100644 index b9dffdf4fe2..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/get_mem_fn_type.hpp +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright 2016 Klemens Morgenstern -// -// 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) - -// For more information, see http://www.boost.org - -#ifndef BOOST_DLL_DETAIL_GET_MEM_FN_TYPE_HPP_ -#define BOOST_DLL_DETAIL_GET_MEM_FN_TYPE_HPP_ - -namespace boost { namespace dll { namespace detail { - -template -struct get_mem_fn_type; - -template -struct get_mem_fn_type { - typedef Return (Class::*mem_fn)(Args...); -}; - -template -struct get_mem_fn_type { - typedef Return (Class::*mem_fn)(Args...) const ; -}; - -template -struct get_mem_fn_type { - typedef Return (Class::*mem_fn)(Args...) volatile; -}; - -template -struct get_mem_fn_type { - typedef Return (Class::*mem_fn)(Args...) const volatile ; -}; - -}}} // namespace boost::dll::detail - - -#endif /* BOOST_DLL_SMART_LIBRARY_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp deleted file mode 100644 index b1a22e70122..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/import_mangled_helpers.hpp +++ /dev/null @@ -1,290 +0,0 @@ -// Copyright 2015-2016 Klemens D. Morgenstern -// -// 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) - -#ifndef BOOST_DLL_DETAIL_IMPORT_MANGLED_HELPERS_HPP_ -#define BOOST_DLL_DETAIL_IMPORT_MANGLED_HELPERS_HPP_ - - -#include -#include -#include -#include -#include - - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { namespace experimental { namespace detail { - -//the following could be done by fusion, though it's simple enough to just declare it here. -template -struct sequence {}; - -template struct push_front; -template -struct push_front> -{ - typedef sequence type; -}; - -template -struct unqalified_is_same : - boost::is_same< - typename boost::remove_cv::type, - typename boost::remove_cv::type - > -{ -}; - -/* ********************************** function sequence type traits ******************************/ - -//determine if it's a sequence of functions. -template struct is_function_seq; - -//type-trait for function overloads -template struct is_function_seq> - : boost::conditional< - boost::is_function::value, - is_function_seq>, - boost::false_type>::type -{}; - -template -struct is_function_seq> : boost::is_function -{ -}; - -template<> -struct is_function_seq> : boost::false_type -{ -}; - -/* ********************************* Function Tuple *************************** */ - -//a tuple of plain functions. -template -struct function_tuple; - -template -struct function_tuple - : function_tuple -{ - Return(*f_)(Args...); - - constexpr function_tuple(Return(* t)(Args...), T2* t2, Ts* ... ts) - : function_tuple(t2, ts...) - , f_(t) - {} - - Return operator()(Args...args) const { - return (*f_)(static_cast(args)...); - } - using function_tuple::operator(); -}; - -template -struct function_tuple { - Return(*f_)(Args...); - - constexpr function_tuple(Return(* t)(Args...)) - : f_(t) - {} - - Return operator()(Args...args) const { - return (*f_)(static_cast(args)...); - } -}; - - -/* ********************************** MemFn sequence type traits ******************************/ - -template -struct mem_fn_def -{ - typedef Class class_type; - typedef Func func_type; - typedef typename boost::dll::detail::get_mem_fn_type::mem_fn mem_fn; -}; - -template -struct make_mem_fn_seq; - -// B: is T1 another version of T0? -template -struct make_mem_fn_seq_getter; - -template -struct make_mem_fn_seq_getter -{ - typedef mem_fn_def type; -}; - -template -struct make_mem_fn_seq_getter -{ - typedef mem_fn_def type; -}; - -template -struct make_mem_fn_seq -{ - typedef mem_fn_def mem_fn; - typedef sequence type; -}; - -template -struct make_mem_fn_seq -{ - typedef sequence<> type; -}; - -template -struct make_mem_fn_seq -{ - /* Since we might have ovls, it might be : - * Class, void(int), void(int, int) //--> just us class for both - * Class, const Class, void(int)//--> ovl class. - * - */ - static_assert(boost::is_object::value, ""); - typedef typename make_mem_fn_seq_getter< - unqalified_is_same::value, T0, T1, T2>::type mem_fn_type; - - typedef typename boost::conditional< - unqalified_is_same::value, - make_mem_fn_seq, - make_mem_fn_seq> ::type next; - - typedef typename push_front::type type; -}; - - - - -/* Ok, this needs to be documented, so here's some pseudo-code: - * - * @code - * - * bool unqalified_is_same(lhs, rhs) - * { - * return remove_cv(lhs) == remove_cv(rhs); - * } - * - * mem_fn make_mem_fn_seq_getter(b, cl, T2, T3) - * { - * if (b) //b means, that T2 is another version of cl, i.e. qualified - * return get_mem_fn_type(T2, T3); - * else //means that T2 is a function. - * return get_mem_fn_type(cl, T2); - * } - * - * sequence make_mem_fn_seq(type cl, type T2, type T3, types...) - * { - * mem_fn = make_mem_fn_seq_getter( - * unqalified_is_same(cl, T2), cl, T2, T3); - * - * next = unqalified_is_same(cl, T2) ? - * make_mem_fn_seq(T2, types...) //because: T2 is another version of cl, hence i use this. T3 was already consumed. - * : - * make_mem_fn_seq(Class, T3, types...) //because: T2 was a function, hence it is consumed and class remains unchanged. - * ; - * return push_front(mem_fn, next) ; - * }; - * @endcode - */ - - - -template -struct is_mem_fn_seq_impl -{ - typedef typename boost::conditional< - boost::is_function::value || boost::dll::experimental::detail::unqalified_is_same::value, - typename is_mem_fn_seq_impl::type, - boost::false_type>::type type; -}; - -template -struct is_mem_fn_seq_impl -{ - typedef typename boost::conditional< - boost::is_function::value && boost::is_object::value, - boost::true_type, boost::false_type>::type type; -}; - -template -struct is_mem_fn_seq_impl -{ - typedef typename boost::conditional< - (boost::is_function::value || boost::dll::experimental::detail::unqalified_is_same::value) - && boost::is_function::value, - boost::true_type, boost::false_type>::type type; -}; - -template struct is_mem_fn_seq : boost::false_type {}; - -//If only two arguments are provided at all. -template -struct is_mem_fn_seq> : boost::conditional< - boost::is_object::value && boost::is_function::value, - boost::true_type, boost::false_type>::type -{ -}; - - -template -struct is_mem_fn_seq> : - boost::conditional< - boost::is_class::value && boost::is_function::value, - typename is_mem_fn_seq_impl::type, - boost::false_type>::type {}; - - -/* ********************************** mem fn sequence tuple ******************************/ - -/* A tuple of member functions - * Unlike for plain functions a sequence here might contain classes as well as functions. - */ -template -struct mem_fn_tuple; - -template -struct mem_fn_tuple, T2, Ts...> - : mem_fn_tuple -{ - typedef typename boost::dll::detail::get_mem_fn_type::mem_fn mem_fn; - mem_fn f_; - - constexpr mem_fn_tuple(mem_fn f, typename T2::mem_fn t2, typename Ts::mem_fn ... ts) - : mem_fn_tuple(t2, ts...) - , f_(f) - {} - - Return operator()(Class* const cl, Args...args) const { - return (cl->*f_)(static_cast(args)...); - } - using mem_fn_tuple::operator(); - -}; - -template -struct mem_fn_tuple> { - typedef typename boost::dll::detail::get_mem_fn_type::mem_fn mem_fn; - mem_fn f_; - - constexpr mem_fn_tuple(mem_fn f) - : f_(f) - {} - - Return operator()(Class * const cl, Args...args) const { - return (cl->*f_)(static_cast(args)...); - } -}; - -}}}} -#endif /* BOOST_DLL_DETAIL_IMPORT_MANGLED_HELPERS_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp deleted file mode 100644 index 69119082383..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/macho_info.hpp +++ /dev/null @@ -1,321 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_MACHO_INFO_HPP -#define BOOST_DLL_DETAIL_MACHO_INFO_HPP - -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#include -#include - -namespace boost { namespace dll { namespace detail { - -typedef int integer_t; -typedef int vm_prot_t; -typedef integer_t cpu_type_t; -typedef integer_t cpu_subtype_t; - -template -struct mach_header_template { - boost::uint32_t magic; - cpu_type_t cputype; - cpu_subtype_t cpusubtype; - boost::uint32_t filetype; - boost::uint32_t ncmds; - boost::uint32_t sizeofcmds; - boost::uint32_t flags[sizeof(AddressOffsetT) / sizeof(uint32_t)]; // Flags and reserved -}; - -typedef mach_header_template mach_header_32_; -typedef mach_header_template mach_header_64_; - -struct load_command_ { - boost::uint32_t cmd; /* type of command */ - boost::uint32_t cmdsize; -}; - -struct load_command_types { - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SEGMENT_ = 0x1); /* segment of this file to be mapped */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SYMTAB_ = 0x2); /* link-edit stab symbol table info */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SYMSEG_ = 0x3); /* link-edit gdb symbol table info (obsolete) */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_THREAD_ = 0x4); /* thread */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_UNIXTHREAD_ = 0x5); /* unix thread (includes a stack) */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOADFVMLIB_ = 0x6); /* load a specified fixed VM shared library */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_IDFVMLIB_ = 0x7); /* fixed VM shared library identification */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_IDENT_ = 0x8); /* object identification info (obsolete) */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_FVMFILE_ = 0x9); /* fixed VM file inclusion (internal use) */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_PREPAGE_ = 0xa); /* prepage command (internal use) */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_DYSYMTAB_ = 0xb); /* dynamic link-edit symbol table info */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOAD_DYLIB_ = 0xc); /* load a dynamically linked shared library */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ID_DYLIB_ = 0xd); /* dynamically linked shared lib ident */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOAD_DYLINKER_ = 0xe); /* load a dynamic linker */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ID_DYLINKER_ = 0xf); /* dynamic linker identification */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_PREBOUND_DYLIB_ = 0x10); /* modules prebound for a dynamically linked shared library */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ROUTINES_ = 0x11); /* image routines */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_FRAMEWORK_ = 0x12); /* sub framework */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_UMBRELLA_ = 0x13); /* sub umbrella */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_CLIENT_ = 0x14); /* sub client */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SUB_LIBRARY_ = 0x15); /* sub library */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_TWOLEVEL_HINTS_ = 0x16); /* two-level namespace lookup hints */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_PREBIND_CKSUM_ = 0x17); /* prebind checksum */ -/* - * After MacOS X 10.1 when a new load command is added that is required to be - * understood by the dynamic linker for the image to execute properly the - * LC_REQ_DYLD bit will be or'ed into the load command constant. If the dynamic - * linker sees such a load command it it does not understand will issue a - * "unknown load command required for execution" error and refuse to use the - * image. Other load commands without this bit that are not understood will - * simply be ignored. - */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_REQ_DYLD_ = 0x80000000); - -/* - * load a dynamically linked shared library that is allowed to be missing - * (all symbols are weak imported). - */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LOAD_WEAK_DYLIB_ = (0x18 | LC_REQ_DYLD_)); - - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SEGMENT_64_ = 0x19); /* 64-bit segment of this file to be mapped */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ROUTINES_64_ = 0x1a); /* 64-bit image routines */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_UUID_ = 0x1b); /* the uuid */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_RPATH_ = (0x1c | LC_REQ_DYLD_)); /* runpath additions */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_CODE_SIGNATURE_ = 0x1d); /* local of code signature */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_SEGMENT_SPLIT_INFO_= 0x1e); /* local of info to split segments */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_REEXPORT_DYLIB_ = (0x1f | LC_REQ_DYLD_)); /* load and re-export dylib */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_LAZY_LOAD_DYLIB_ = 0x20); /* delay load of dylib until first use */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_ENCRYPTION_INFO_ = 0x21); /* encrypted segment information */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_DYLD_INFO_ = 0x22); /* compressed dyld information */ - BOOST_STATIC_CONSTANT(boost::uint32_t, LC_DYLD_INFO_ONLY_ = (0x22|LC_REQ_DYLD_)); /* compressed dyld information only */ -}; - -template -struct segment_command_template { - boost::uint32_t cmd; /* LC_SEGMENT_ */ - boost::uint32_t cmdsize; /* includes sizeof section structs */ - char segname[16]; /* segment name */ - AddressOffsetT vmaddr; /* memory address of this segment */ - AddressOffsetT vmsize; /* memory size of this segment */ - AddressOffsetT fileoff; /* file offset of this segment */ - AddressOffsetT filesize; /* amount to map from the file */ - vm_prot_t maxprot; /* maximum VM protection */ - vm_prot_t initprot; /* initial VM protection */ - boost::uint32_t nsects; /* number of sections in segment */ - boost::uint32_t flags; /* flags */ -}; - -typedef segment_command_template segment_command_32_; -typedef segment_command_template segment_command_64_; - -template -struct section_template { - char sectname[16]; /* name of this section */ - char segname[16]; /* segment this section goes in */ - AddressOffsetT addr; /* memory address of this section */ - AddressOffsetT size; /* size in bytes of this section */ - boost::uint32_t offset; /* file offset of this section */ - boost::uint32_t align; /* section alignment (power of 2) */ - boost::uint32_t reloff; /* file offset of relocation entries */ - boost::uint32_t nreloc; /* number of relocation entries */ - boost::uint32_t flags; /* flags (section type and attributes)*/ - boost::uint32_t reserved[1 + sizeof(AddressOffsetT) / sizeof(uint32_t)]; -}; - -typedef section_template section_32_; -typedef section_template section_64_; - -struct symtab_command_ { - boost::uint32_t cmd; /* LC_SYMTAB_ */ - boost::uint32_t cmdsize; /* sizeof(struct symtab_command) */ - boost::uint32_t symoff; /* symbol table offset */ - boost::uint32_t nsyms; /* number of symbol table entries */ - boost::uint32_t stroff; /* string table offset */ - boost::uint32_t strsize; /* string table size in bytes */ -}; - -template -struct nlist_template { - boost::uint32_t n_strx; - boost::uint8_t n_type; - boost::uint8_t n_sect; - boost::uint16_t n_desc; - AddressOffsetT n_value; -}; - -typedef nlist_template nlist_32_; -typedef nlist_template nlist_64_; - -template -class macho_info: public x_info_interface { - boost::filesystem::ifstream& f_; - - typedef boost::dll::detail::mach_header_template header_t; - typedef boost::dll::detail::load_command_ load_command_t; - typedef boost::dll::detail::segment_command_template segment_t; - typedef boost::dll::detail::section_template section_t; - typedef boost::dll::detail::symtab_command_ symbol_header_t; - typedef boost::dll::detail::nlist_template nlist_t; - - BOOST_STATIC_CONSTANT(boost::uint32_t, SEGMENT_CMD_NUMBER = (sizeof(AddressOffsetT) > 4 ? load_command_types::LC_SEGMENT_64_ : load_command_types::LC_SEGMENT_)); - -public: - static bool parsing_supported(boost::filesystem::ifstream& f) { - static const uint32_t magic_bytes = (sizeof(AddressOffsetT) <= sizeof(uint32_t) ? 0xfeedface : 0xfeedfacf); - - uint32_t magic; - f.seekg(0); - f.read(reinterpret_cast(&magic), sizeof(magic)); - return (magic_bytes == magic); - } - - explicit macho_info(boost::filesystem::ifstream& f) BOOST_NOEXCEPT - : f_(f) - {} - -private: - template - inline void read_raw(T& value, std::size_t size = sizeof(T)) const { - f_.read(reinterpret_cast(&value), size); - } - - template - void command_finder(uint32_t cmd_num, F callback_f) { - const header_t h = header(); - load_command_t command; - f_.seekg(sizeof(header_t)); - for (std::size_t i = 0; i < h.ncmds; ++i) { - const boost::filesystem::ifstream::pos_type pos = f_.tellg(); - read_raw(command); - if (command.cmd != cmd_num) { - f_.seekg(pos + static_cast(command.cmdsize)); - continue; - } - - f_.seekg(pos); - callback_f(*this); - f_.seekg(pos + static_cast(command.cmdsize)); - } - } - - struct section_names_gather { - std::vector& ret; - - void operator()(const macho_info& f) const { - segment_t segment; - f.read_raw(segment); - - section_t section; - ret.reserve(ret.size() + segment.nsects); - for (std::size_t j = 0; j < segment.nsects; ++j) { - f.read_raw(section); - // `segname` goes right after the `sectname`. - // Forcing `sectname` to end on '\0' - section.segname[0] = '\0'; - ret.push_back(section.sectname); - if (ret.back().empty()) { - ret.pop_back(); // Do not show empty names - } - } - } - }; - - struct symbol_names_gather { - std::vector& ret; - std::size_t section_index; - - void operator()(const macho_info& f) const { - symbol_header_t symbh; - f.read_raw(symbh); - ret.reserve(ret.size() + symbh.nsyms); - - nlist_t symbol; - std::string symbol_name; - for (std::size_t j = 0; j < symbh.nsyms; ++j) { - f.f_.seekg(symbh.symoff + j * sizeof(nlist_t)); - f.read_raw(symbol); - if (!symbol.n_strx) { - continue; // Symbol has no name - } - - if ((symbol.n_type & 0x0e) != 0xe || !symbol.n_sect) { - continue; // Symbol has no section - } - - if (section_index && section_index != symbol.n_sect) { - continue; // Not in the required section - } - - f.f_.seekg(symbh.stroff + symbol.n_strx); - getline(f.f_, symbol_name, '\0'); - if (symbol_name.empty()) { - continue; - } - - if (symbol_name[0] == '_') { - // Linker adds additional '_' symbol. Could not find official docs for that case. - ret.push_back(symbol_name.c_str() + 1); - } else { - ret.push_back(symbol_name); - } - } - } - }; - -public: - std::vector sections() { - std::vector ret; - section_names_gather f = { ret }; - command_finder(SEGMENT_CMD_NUMBER, f); - return ret; - } - -private: - inline header_t header() { - header_t h; - - f_.seekg(0); - read_raw(h); - - return h; - } - -public: - std::vector symbols() { - std::vector ret; - symbol_names_gather f = { ret, 0 }; - command_finder(load_command_types::LC_SYMTAB_, f); - return ret; - } - - std::vector symbols(const char* section_name) { - // Not very optimal solution - std::vector ret = sections(); - std::vector::iterator it = std::find(ret.begin(), ret.end(), section_name); - if (it == ret.end()) { - // No section with such name - ret.clear(); - return ret; - } - - // section indexes start from 1 - symbol_names_gather f = { ret, static_cast(1 + (it - ret.begin())) }; - ret.clear(); - command_finder(load_command_types::LC_SYMTAB_, f); - return ret; - } -}; - -typedef macho_info macho_info32; -typedef macho_info macho_info64; - -}}} // namespace boost::dll::detail - -#endif // BOOST_DLL_DETAIL_MACHO_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp deleted file mode 100644 index 72668a075f7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/pe_info.hpp +++ /dev/null @@ -1,430 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_WINDOWS_PE_INFO_HPP -#define BOOST_DLL_DETAIL_WINDOWS_PE_INFO_HPP - -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#include -#include -#include - -namespace boost { namespace dll { namespace detail { - -// reference: -// http://www.joachim-bauch.de/tutorials/loading-a-dll-from-memory/ -// http://msdn.microsoft.com/en-us/magazine/ms809762.aspx -// http://msdn.microsoft.com/en-us/magazine/cc301808.aspx -// - -// Basic Windows typedefs. We can not use header -// because that header must be included only on Windows platform -typedef unsigned char BYTE_; -typedef unsigned short WORD_; -typedef boost::uint32_t DWORD_; -typedef boost::int32_t LONG_; -typedef boost::uint32_t ULONG_; -typedef boost::int64_t LONGLONG_; -typedef boost::uint64_t ULONGLONG_; - -struct IMAGE_DOS_HEADER_ { // 32/64 independent header - boost::dll::detail::WORD_ e_magic; // Magic number - boost::dll::detail::WORD_ e_cblp; // Bytes on last page of file - boost::dll::detail::WORD_ e_cp; // Pages in file - boost::dll::detail::WORD_ e_crlc; // Relocations - boost::dll::detail::WORD_ e_cparhdr; // Size of header in paragraphs - boost::dll::detail::WORD_ e_minalloc; // Minimum extra paragraphs needed - boost::dll::detail::WORD_ e_maxalloc; // Maximum extra paragraphs needed - boost::dll::detail::WORD_ e_ss; // Initial (relative) SS value - boost::dll::detail::WORD_ e_sp; // Initial SP value - boost::dll::detail::WORD_ e_csum; // Checksum - boost::dll::detail::WORD_ e_ip; // Initial IP value - boost::dll::detail::WORD_ e_cs; // Initial (relative) CS value - boost::dll::detail::WORD_ e_lfarlc; // File address of relocation table - boost::dll::detail::WORD_ e_ovno; // Overlay number - boost::dll::detail::WORD_ e_res[4]; // Reserved words - boost::dll::detail::WORD_ e_oemid; // OEM identifier (for e_oeminfo) - boost::dll::detail::WORD_ e_oeminfo; // OEM information; e_oemid specific - boost::dll::detail::WORD_ e_res2[10]; // Reserved words - boost::dll::detail::LONG_ e_lfanew; // File address of new exe header -}; - -struct IMAGE_FILE_HEADER_ { // 32/64 independent header - boost::dll::detail::WORD_ Machine; - boost::dll::detail::WORD_ NumberOfSections; - boost::dll::detail::DWORD_ TimeDateStamp; - boost::dll::detail::DWORD_ PointerToSymbolTable; - boost::dll::detail::DWORD_ NumberOfSymbols; - boost::dll::detail::WORD_ SizeOfOptionalHeader; - boost::dll::detail::WORD_ Characteristics; -}; - -struct IMAGE_DATA_DIRECTORY_ { // 32/64 independent header - boost::dll::detail::DWORD_ VirtualAddress; - boost::dll::detail::DWORD_ Size; -}; - -struct IMAGE_EXPORT_DIRECTORY_ { // 32/64 independent header - boost::dll::detail::DWORD_ Characteristics; - boost::dll::detail::DWORD_ TimeDateStamp; - boost::dll::detail::WORD_ MajorVersion; - boost::dll::detail::WORD_ MinorVersion; - boost::dll::detail::DWORD_ Name; - boost::dll::detail::DWORD_ Base; - boost::dll::detail::DWORD_ NumberOfFunctions; - boost::dll::detail::DWORD_ NumberOfNames; - boost::dll::detail::DWORD_ AddressOfFunctions; - boost::dll::detail::DWORD_ AddressOfNames; - boost::dll::detail::DWORD_ AddressOfNameOrdinals; -}; - -struct IMAGE_SECTION_HEADER_ { // 32/64 independent header - static const std::size_t IMAGE_SIZEOF_SHORT_NAME_ = 8; - - boost::dll::detail::BYTE_ Name[IMAGE_SIZEOF_SHORT_NAME_]; - union { - boost::dll::detail::DWORD_ PhysicalAddress; - boost::dll::detail::DWORD_ VirtualSize; - } Misc; - boost::dll::detail::DWORD_ VirtualAddress; - boost::dll::detail::DWORD_ SizeOfRawData; - boost::dll::detail::DWORD_ PointerToRawData; - boost::dll::detail::DWORD_ PointerToRelocations; - boost::dll::detail::DWORD_ PointerToLinenumbers; - boost::dll::detail::WORD_ NumberOfRelocations; - boost::dll::detail::WORD_ NumberOfLinenumbers; - boost::dll::detail::DWORD_ Characteristics; -}; - - -template -struct IMAGE_OPTIONAL_HEADER_template { - static const std::size_t IMAGE_NUMBEROF_DIRECTORY_ENTRIES_ = 16; - - boost::dll::detail::WORD_ Magic; - boost::dll::detail::BYTE_ MajorLinkerVersion; - boost::dll::detail::BYTE_ MinorLinkerVersion; - boost::dll::detail::DWORD_ SizeOfCode; - boost::dll::detail::DWORD_ SizeOfInitializedData; - boost::dll::detail::DWORD_ SizeOfUninitializedData; - boost::dll::detail::DWORD_ AddressOfEntryPoint; - union { - boost::dll::detail::DWORD_ BaseOfCode; - unsigned char padding_[sizeof(AddressOffsetT) == 8 ? 4 : 8]; // in x64 version BaseOfData does not exist - } BaseOfCode_and_BaseOfData; - - AddressOffsetT ImageBase; - boost::dll::detail::DWORD_ SectionAlignment; - boost::dll::detail::DWORD_ FileAlignment; - boost::dll::detail::WORD_ MajorOperatingSystemVersion; - boost::dll::detail::WORD_ MinorOperatingSystemVersion; - boost::dll::detail::WORD_ MajorImageVersion; - boost::dll::detail::WORD_ MinorImageVersion; - boost::dll::detail::WORD_ MajorSubsystemVersion; - boost::dll::detail::WORD_ MinorSubsystemVersion; - boost::dll::detail::DWORD_ Win32VersionValue; - boost::dll::detail::DWORD_ SizeOfImage; - boost::dll::detail::DWORD_ SizeOfHeaders; - boost::dll::detail::DWORD_ CheckSum; - boost::dll::detail::WORD_ Subsystem; - boost::dll::detail::WORD_ DllCharacteristics; - AddressOffsetT SizeOfStackReserve; - AddressOffsetT SizeOfStackCommit; - AddressOffsetT SizeOfHeapReserve; - AddressOffsetT SizeOfHeapCommit; - boost::dll::detail::DWORD_ LoaderFlags; - boost::dll::detail::DWORD_ NumberOfRvaAndSizes; - IMAGE_DATA_DIRECTORY_ DataDirectory[IMAGE_NUMBEROF_DIRECTORY_ENTRIES_]; -}; - -typedef IMAGE_OPTIONAL_HEADER_template IMAGE_OPTIONAL_HEADER32_; -typedef IMAGE_OPTIONAL_HEADER_template IMAGE_OPTIONAL_HEADER64_; - -template -struct IMAGE_NT_HEADERS_template { - boost::dll::detail::DWORD_ Signature; - IMAGE_FILE_HEADER_ FileHeader; - IMAGE_OPTIONAL_HEADER_template OptionalHeader; -}; - -typedef IMAGE_NT_HEADERS_template IMAGE_NT_HEADERS32_; -typedef IMAGE_NT_HEADERS_template IMAGE_NT_HEADERS64_; - - -template -class pe_info: public x_info_interface { - boost::filesystem::ifstream& f_; - - typedef IMAGE_NT_HEADERS_template header_t; - typedef IMAGE_EXPORT_DIRECTORY_ exports_t; - typedef IMAGE_SECTION_HEADER_ section_t; - typedef IMAGE_DOS_HEADER_ dos_t; - - template - inline void read_raw(T& value, std::size_t size = sizeof(T)) const { - f_.read(reinterpret_cast(&value), size); - } - -public: - static bool parsing_supported(boost::filesystem::ifstream& f) { - dos_t dos; - f.seekg(0); - f.read(reinterpret_cast(&dos), sizeof(dos)); - - // 'MZ' and 'ZM' according to Wikipedia - if (dos.e_magic != 0x4D5A && dos.e_magic != 0x5A4D) { - return false; - } - - header_t h; - f.seekg(dos.e_lfanew); - f.read(reinterpret_cast(&h), sizeof(h)); - - return h.Signature == 0x00004550 // 'PE00' - && h.OptionalHeader.Magic == (sizeof(boost::uint32_t) == sizeof(AddressOffsetT) ? 0x10B : 0x20B); - } - - - explicit pe_info(boost::filesystem::ifstream& f) BOOST_NOEXCEPT - : f_(f) - {} - -private: - inline header_t header() { - header_t h; - - dos_t dos; - f_.seekg(0); - read_raw(dos); - - f_.seekg(dos.e_lfanew); - read_raw(h); - - return h; - } - - inline exports_t exports(const header_t& h) { - exports_t exports; - - static const unsigned int IMAGE_DIRECTORY_ENTRY_EXPORT_ = 0; - const std::size_t exp_virtual_address = h.OptionalHeader.DataDirectory[IMAGE_DIRECTORY_ENTRY_EXPORT_].VirtualAddress; - - const std::size_t real_offset = get_file_offset(exp_virtual_address, h); - BOOST_ASSERT(real_offset); - - f_.seekg(real_offset); - read_raw(exports); - - return exports; - } - - std::size_t get_file_offset(std::size_t virtual_address, const header_t& h) { - section_t image_section_header; - - { // f_.seekg to the beginning on section headers - dos_t dos; - f_.seekg(0); - read_raw(dos); - f_.seekg(dos.e_lfanew + sizeof(header_t)); - } - - for (std::size_t i = 0;i < h.FileHeader.NumberOfSections;++i) { - read_raw(image_section_header); - if (virtual_address >= image_section_header.VirtualAddress - && virtual_address < image_section_header.VirtualAddress + image_section_header.SizeOfRawData) - { - return image_section_header.PointerToRawData + virtual_address - image_section_header.VirtualAddress; - } - } - - return 0; - } - -public: - std::vector sections() { - std::vector ret; - - const header_t h = header(); - ret.reserve(h.FileHeader.NumberOfSections); - - // get names, e.g: .text .rdata .data .rsrc .reloc - section_t image_section_header; - char name_helper[section_t::IMAGE_SIZEOF_SHORT_NAME_ + 1]; - std::memset(name_helper, 0, sizeof(name_helper)); - for (std::size_t i = 0;i < h.FileHeader.NumberOfSections;++i) { - // There is no terminating null character if the string is exactly eight characters long - read_raw(image_section_header); - std::memcpy(name_helper, image_section_header.Name, section_t::IMAGE_SIZEOF_SHORT_NAME_); - - if (name_helper[0] != '/') { - ret.push_back(name_helper); - } else { - // For longer names, image_section_header.Name contains a slash (/) followed by ASCII representation of a decimal number. - // this number is an offset into the string table. - // TODO: fixme - ret.push_back(name_helper); - } - } - - return ret; - } - - std::vector symbols() { - std::vector ret; - - const header_t h = header(); - const exports_t exprt = exports(h); - const std::size_t exported_symbols = exprt.NumberOfNames; - const std::size_t fixed_names_addr = get_file_offset(exprt.AddressOfNames, h); - - ret.reserve(exported_symbols); - boost::dll::detail::DWORD_ name_offset; - std::string symbol_name; - for (std::size_t i = 0;i < exported_symbols;++i) { - f_.seekg(fixed_names_addr + i * sizeof(name_offset)); - read_raw(name_offset); - f_.seekg(get_file_offset(name_offset, h)); - getline(f_, symbol_name, '\0'); - ret.push_back(symbol_name); - } - - return ret; - } - - std::vector symbols(const char* section_name) { - std::vector ret; - - const header_t h = header(); - - std::size_t section_begin_addr = 0; - std::size_t section_end_addr = 0; - - { // getting address range for the section - section_t image_section_header; - char name_helper[section_t::IMAGE_SIZEOF_SHORT_NAME_ + 1]; - std::memset(name_helper, 0, sizeof(name_helper)); - for (std::size_t i = 0;i < h.FileHeader.NumberOfSections;++i) { - // There is no terminating null character if the string is exactly eight characters long - read_raw(image_section_header); - std::memcpy(name_helper, image_section_header.Name, section_t::IMAGE_SIZEOF_SHORT_NAME_); - if (!std::strcmp(section_name, name_helper)) { - section_begin_addr = image_section_header.PointerToRawData; - section_end_addr = section_begin_addr + image_section_header.SizeOfRawData; - } - } - - // returning empty result if section was not found - if(section_begin_addr == 0 || section_end_addr == 0) - return ret; - } - - const exports_t exprt = exports(h); - const std::size_t exported_symbols = exprt.NumberOfFunctions; - const std::size_t fixed_names_addr = get_file_offset(exprt.AddressOfNames, h); - const std::size_t fixed_ordinals_addr = get_file_offset(exprt.AddressOfNameOrdinals, h); - const std::size_t fixed_functions_addr = get_file_offset(exprt.AddressOfFunctions, h); - - ret.reserve(exported_symbols); - boost::dll::detail::DWORD_ ptr; - boost::dll::detail::WORD_ ordinal; - std::string symbol_name; - for (std::size_t i = 0;i < exported_symbols;++i) { - // getting ordinal - f_.seekg(fixed_ordinals_addr + i * sizeof(ordinal)); - read_raw(ordinal); - - // getting function addr - f_.seekg(fixed_functions_addr + ordinal * sizeof(ptr)); - read_raw(ptr); - ptr = static_cast( get_file_offset(ptr, h) ); - - if (ptr >= section_end_addr || ptr < section_begin_addr) { - continue; - } - - f_.seekg(fixed_names_addr + i * sizeof(ptr)); - read_raw(ptr); - f_.seekg(get_file_offset(ptr, h)); - getline(f_, symbol_name, '\0'); - ret.push_back(symbol_name); - } - - return ret; - } - - // a test method to get dependents modules, - // who my plugin imports (1st level only) - /* - e.g. for myself I get: - KERNEL32.dll - MSVCP110D.dll - boost_system-vc-mt-gd-1_56.dll - MSVCR110D.dll - */ - /* - std::vector depend_of(boost::system::error_code &ec) BOOST_NOEXCEPT { - std::vector ret; - - IMAGE_DOS_HEADER* image_dos_header = (IMAGE_DOS_HEADER*)native(); - if(!image_dos_header) { - // ERROR_BAD_EXE_FORMAT - ec = boost::system::error_code( - boost::system::errc::executable_format_error, - boost::system::generic_category() - ); - - return ret; - } - - IMAGE_OPTIONAL_HEADER* image_optional_header = (IMAGE_OPTIONAL_HEADER*)((boost::dll::detail::BYTE_*)native() + image_dos_header->e_lfanew + 24); - if(!image_optional_header) { - // ERROR_BAD_EXE_FORMAT - ec = boost::system::error_code( - boost::system::errc::executable_format_error, - boost::system::generic_category() - ); - - return ret; - } - - IMAGE_IMPORT_DESCRIPTOR* image_import_descriptor = (IMAGE_IMPORT_DESCRIPTOR*)((boost::dll::detail::BYTE_*)native() + image_optional_header->DataDirectory[IMAGE_DIRECTORY_ENTRY_IMPORT].VirtualAddress); - if(!image_import_descriptor) { - // ERROR_BAD_EXE_FORMAT - ec = boost::system::error_code( - boost::system::errc::executable_format_error, - boost::system::generic_category() - ); - - return ret; - } - - while(image_import_descriptor->FirstThunk) { - std::string module_name = reinterpret_cast((boost::dll::detail::BYTE_*)native() + image_import_descriptor->Name); - - if(module_name.size()) { - ret.push_back(module_name); - } - - image_import_descriptor++; - } - - return ret; - } -*/ -}; - -typedef pe_info pe_info32; -typedef pe_info pe_info64; - -}}} // namespace boost::dll::detail - -#endif // BOOST_DLL_DETAIL_WINDOWS_PE_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp deleted file mode 100644 index 65d1a78b96e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/path_from_handle.hpp +++ /dev/null @@ -1,169 +0,0 @@ -// Copyright 2014-2015 Renato Tegon Forti, Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_POSIX_PATH_FROM_HANDLE_HPP -#define BOOST_DLL_DETAIL_POSIX_PATH_FROM_HANDLE_HPP - -#include -#include -#include -#include -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#if BOOST_OS_MACOS || BOOST_OS_IOS - -# include -# include -# include // for std::ptrdiff_t - -namespace boost { namespace dll { namespace detail { - inline void* strip_handle(void* handle) BOOST_NOEXCEPT { - return reinterpret_cast( - (reinterpret_cast(handle) >> 2) << 2 - ); - } - - inline boost::filesystem::path path_from_handle(void* handle, boost::system::error_code &ec) { - handle = strip_handle(handle); - - // Iterate through all images currently in memory - // https://developer.apple.com/library/mac/documentation/Darwin/Reference/ManPages/man3/dyld.3.html - const std::size_t count = _dyld_image_count(); // not thread safe: other thread my [un]load images - for (std::size_t i = 0; i <= count; ++i) { - // on last iteration `i` is equal to `count` which is out of range, so `_dyld_get_image_name` - // will return NULL. `dlopen(NULL, RTLD_LAZY)` call will open the current executable. - const char* image_name = _dyld_get_image_name(i); - - // dlopen/dlclose must not affect `_dyld_image_count()`, because libraries are already loaded and only the internal counter is affected - void* probe_handle = dlopen(image_name, RTLD_LAZY); - dlclose(probe_handle); - - // If the handle is the same as what was passed in (modulo mode bits), return this image name - if (handle == strip_handle(probe_handle)) { - boost::dll::detail::reset_dlerror(); - return image_name; - } - } - - boost::dll::detail::reset_dlerror(); - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - return boost::filesystem::path(); - } - -}}} // namespace boost::dll::detail - -#elif BOOST_OS_ANDROID - -#include - -namespace boost { namespace dll { namespace detail { - - struct soinfo { - // if defined(__work_around_b_24465209__), then an array of char[128] goes here. - // Unfortunately, __work_around_b_24465209__ is visible only during compilation of Android's linker - const void* phdr; - size_t phnum; - void* entry; - void* base; - // ... // Ignoring remaning parts of the structure - }; - - inline boost::filesystem::path path_from_handle(const void* handle, boost::system::error_code &ec) { - static const std::size_t work_around_b_24465209__offset = 128; - const struct soinfo* si = reinterpret_cast( - static_cast(handle) + work_around_b_24465209__offset - ); - boost::filesystem::path ret = boost::dll::symbol_location_ptr(si->base, ec); - - if (ec) { - ec.clear(); - si = static_cast(handle); - return boost::dll::symbol_location_ptr(si->base, ec); - } - - return ret; - } - -}}} // namespace boost::dll::detail - -#else // #if BOOST_OS_MACOS || BOOST_OS_IOS || BOOST_OS_ANDROID - -// for dlinfo -#include - -#if BOOST_OS_QNX -// QNX's copy of and reside in sys folder -# include -#else -# include // struct link_map -#endif - -namespace boost { namespace dll { namespace detail { - -#if BOOST_OS_QNX - // Android and QNX miss struct link_map. QNX misses ElfW macro, so avoiding it. - struct link_map { - void *l_addr; // Base address shared object is loaded at - char *l_name; // Absolute file name object was found in - // ... // Ignoring remaning parts of the structure - }; -#endif // #if BOOST_OS_QNX - - inline boost::filesystem::path path_from_handle(void* handle, boost::system::error_code &ec) { - // RTLD_DI_LINKMAP (RTLD_DI_ORIGIN returns only folder and is not suitable for this case) - // Obtain the Link_map for the handle that is specified. - // The p argument points to a Link_map pointer (Link_map - // **p). The actual storage for the Link_map structure is - // maintained by ld.so.1. - // - // Unfortunately we can not use `dlinfo(handle, RTLD_DI_LINKMAP, &link_map) < 0` - // because it is not supported on MacOS X 10.3, NetBSD 3.0, OpenBSD 3.8, AIX 5.1, - // HP-UX 11, IRIX 6.5, OSF/1 5.1, Cygwin, mingw, Interix 3.5, BeOS. - // Fortunately investigating the sources of open source projects brought the understanding, that - // `handle` is just a `struct link_map*` that contains full library name. - - const struct link_map* link_map = 0; -#if BOOST_OS_BSD_FREE - // FreeBSD has it's own logic http://code.metager.de/source/xref/freebsd/libexec/rtld-elf/rtld.c - // Fortunately it has the dlinfo call. - if (dlinfo(handle, RTLD_DI_LINKMAP, &link_map) < 0) { - link_map = 0; - } -#else - link_map = static_cast(handle); -#endif - if (!link_map) { - boost::dll::detail::reset_dlerror(); - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - return boost::filesystem::path(); - } - - if (!link_map->l_name || *link_map->l_name == '\0') { - return program_location_impl(ec); - } - - return boost::filesystem::path(link_map->l_name); - } - -}}} // namespace boost::dll::detail - -#endif // #if BOOST_OS_MACOS || BOOST_OS_IOS - -#endif // BOOST_DLL_DETAIL_POSIX_PATH_FROM_HANDLE_HPP - - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp deleted file mode 100644 index bca9b8caef9..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/program_location_impl.hpp +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_POSIX_PROGRAM_LOCATION_IMPL_HPP -#define BOOST_DLL_DETAIL_POSIX_PROGRAM_LOCATION_IMPL_HPP - -#include -#include -#include -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#if BOOST_OS_MACOS || BOOST_OS_IOS - -#include - -namespace boost { namespace dll { namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { - ec.clear(); - - char path[1024]; - uint32_t size = sizeof(path); - if (_NSGetExecutablePath(path, &size) == 0) - return boost::filesystem::path(path); - - char *p = new char[size]; - if (_NSGetExecutablePath(p, &size) != 0) { - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - } - - boost::filesystem::path ret(p); - delete[] p; - return ret; - } -}}} // namespace boost::dll::detail - -#elif BOOST_OS_SOLARIS - -#include -namespace boost { namespace dll { namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code& ec) { - ec.clear(); - - return boost::filesystem::path(getexecname()); - } -}}} // namespace boost::dll::detail - -#elif BOOST_OS_BSD_FREE - -#include -#include -#include - -namespace boost { namespace dll { namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code& ec) { - ec.clear(); - - int mib[4]; - mib[0] = CTL_KERN; - mib[1] = KERN_PROC; - mib[2] = KERN_PROC_PATHNAME; - mib[3] = -1; - char buf[10240]; - size_t cb = sizeof(buf); - sysctl(mib, 4, buf, &cb, NULL, 0); - - return boost::filesystem::path(buf); - } -}}} // namespace boost::dll::detail - - - -#elif BOOST_OS_BSD_NET - -#include -namespace boost { namespace dll { namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { - return boost::filesystem::read_symlink("/proc/curproc/exe", ec); - } -}}} // namespace boost::dll::detail - -#elif BOOST_OS_BSD_DRAGONFLY - -#include -namespace boost { namespace dll { namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { - return boost::filesystem::read_symlink("/proc/curproc/file", ec); - } -}}} // namespace boost::dll::detail - -#elif BOOST_OS_QNX - -#include -#include // for std::getline -namespace boost { namespace dll { namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { - ec.clear(); - - std::string s; - std::ifstream ifs("/proc/self/exefile"); - std::getline(ifs, s); - - if (ifs.fail() || s.empty()) { - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - } - - return boost::filesystem::path(s); - } -}}} // namespace boost::dll::detail - -#else // BOOST_OS_LINUX || BOOST_OS_UNIX || BOOST_OS_HPUX || BOOST_OS_ANDROID - -#include -namespace boost { namespace dll { namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code &ec) { - // We can not use - // boost::dll::detail::path_from_handle(dlopen(NULL, RTLD_LAZY | RTLD_LOCAL), ignore); - // because such code returns empty path. - - return boost::filesystem::read_symlink("/proc/self/exe", ec); // Linux specific - } -}}} // namespace boost::dll::detail - -#endif - -#endif // BOOST_DLL_DETAIL_POSIX_PROGRAM_LOCATION_IMPL_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp deleted file mode 100644 index 509e6c74873..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/posix/shared_library_impl.hpp +++ /dev/null @@ -1,215 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015-2016 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_SHARED_LIBRARY_IMPL_HPP -#define BOOST_DLL_SHARED_LIBRARY_IMPL_HPP - -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include // strncmp -#if !BOOST_OS_MACOS && !BOOST_OS_IOS && !BOOST_OS_QNX -# include -#elif BOOST_OS_QNX -// QNX's copy of and reside in sys folder -# include -#endif - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { namespace detail { - -class shared_library_impl { - - BOOST_MOVABLE_BUT_NOT_COPYABLE(shared_library_impl) - -public: - typedef void* native_handle_t; - - shared_library_impl() BOOST_NOEXCEPT - : handle_(NULL) - {} - - ~shared_library_impl() BOOST_NOEXCEPT { - unload(); - } - - shared_library_impl(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT - : handle_(sl.handle_) - { - sl.handle_ = NULL; - } - - shared_library_impl & operator=(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT { - swap(sl); - return *this; - } - - void load(boost::filesystem::path sl, load_mode::type mode, boost::system::error_code &ec) { - typedef int native_mode_t; - unload(); - - // Do not allow opening NULL paths. User must use program_location() instead - if (sl.empty()) { - boost::dll::detail::reset_dlerror(); - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - return; - } - - // Fixing modes - if (!(mode & load_mode::rtld_now)) { - mode |= load_mode::rtld_lazy; - } - - if (!(mode & load_mode::rtld_global)) { - mode |= load_mode::rtld_local; - } - -#if BOOST_OS_LINUX || BOOST_OS_ANDROID - if (!sl.has_parent_path() && !(mode & load_mode::search_system_folders)) { - sl = "." / sl; - } -#else - if (!sl.is_absolute() && !(mode & load_mode::search_system_folders)) { - boost::system::error_code current_path_ec; - boost::filesystem::path prog_loc = boost::filesystem::current_path(current_path_ec); - if (!current_path_ec) { - prog_loc /= sl; - sl.swap(prog_loc); - } - } -#endif - - mode &= ~load_mode::search_system_folders; - - // Trying to open with appended decorations - if (!!(mode & load_mode::append_decorations)) { - mode &= ~load_mode::append_decorations; - - boost::filesystem::path actual_path = ( - std::strncmp(sl.filename().string().c_str(), "lib", 3) - ? (sl.has_parent_path() ? sl.parent_path() / L"lib" : L"lib").native() + sl.filename().native() - : sl - ); - actual_path += suffix(); - - handle_ = dlopen(actual_path.c_str(), static_cast(mode)); - if (handle_) { - boost::dll::detail::reset_dlerror(); - return; - } - } - - // Opening by exactly specified path - handle_ = dlopen(sl.c_str(), static_cast(mode)); - if (handle_) { - boost::dll::detail::reset_dlerror(); - return; - } - - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - // Maybe user wanted to load the executable itself? Checking... - // We assume that usually user wants to load a dynamic library not the executable itself, that's why - // we try this only after traditional load fails. - boost::system::error_code prog_loc_err; - boost::filesystem::path loc = boost::dll::detail::program_location_impl(prog_loc_err); - if (!prog_loc_err && boost::filesystem::equivalent(sl, loc, prog_loc_err) && !prog_loc_err) { - // As is known the function dlopen() loads the dynamic library file - // named by the null-terminated string filename and returns an opaque - // "handle" for the dynamic library. If filename is NULL, then the - // returned handle is for the main program. - ec.clear(); - boost::dll::detail::reset_dlerror(); - handle_ = dlopen(NULL, static_cast(mode)); - if (!handle_) { - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - } - } - } - - bool is_loaded() const BOOST_NOEXCEPT { - return (handle_ != 0); - } - - void unload() BOOST_NOEXCEPT { - if (!is_loaded()) { - return; - } - - dlclose(handle_); - handle_ = 0; - } - - void swap(shared_library_impl& rhs) BOOST_NOEXCEPT { - boost::swap(handle_, rhs.handle_); - } - - boost::filesystem::path full_module_path(boost::system::error_code &ec) const { - return boost::dll::detail::path_from_handle(handle_, ec); - } - - static boost::filesystem::path suffix() { - // https://sourceforge.net/p/predef/wiki/OperatingSystems/ -#if BOOST_OS_MACOS || BOOST_OS_IOS - return ".dylib"; -#else - return ".so"; -#endif - } - - void* symbol_addr(const char* sb, boost::system::error_code &ec) const BOOST_NOEXCEPT { - // dlsym - obtain the address of a symbol from a dlopen object - void* const symbol = dlsym(handle_, sb); - if (symbol == NULL) { - ec = boost::system::error_code( - boost::system::errc::invalid_seek, - boost::system::generic_category() - ); - } - - // If handle does not refer to a valid object opened by dlopen(), - // or if the named symbol cannot be found within any of the objects - // associated with handle, dlsym() shall return NULL. - // More detailed diagnostic information shall be available through dlerror(). - - return symbol; - } - - native_handle_t native() const BOOST_NOEXCEPT { - return handle_; - } - -private: - native_handle_t handle_; -}; - -}}} // boost::dll::detail - -#endif // BOOST_DLL_SHARED_LIBRARY_IMPL_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp deleted file mode 100644 index c032633a0b8..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/system_error.hpp +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_SYSTEM_ERROR_HPP -#define BOOST_DLL_SYSTEM_ERROR_HPP - -#include -#include -#include -#include -#include - -#if !BOOST_OS_WINDOWS -# include -#endif - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { namespace detail { - - inline void reset_dlerror() BOOST_NOEXCEPT { -#if !BOOST_OS_WINDOWS - const char* const error_txt = dlerror(); - (void)error_txt; -#endif - } - - inline void report_error(const boost::system::error_code& ec, const char* message) { -#if !BOOST_OS_WINDOWS - const char* const error_txt = dlerror(); - if (error_txt) { - boost::throw_exception( - boost::system::system_error( - ec, - message + std::string(" (dlerror system message: ") + error_txt + std::string(")") - ) - ); - } -#endif - - boost::throw_exception( - boost::system::system_error( - ec, message - ) - ); - } - -}}} // boost::dll::detail - -#endif // BOOST_DLL_SYSTEM_ERROR_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp deleted file mode 100644 index 798ae20042e..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/type_info.hpp +++ /dev/null @@ -1,83 +0,0 @@ -// Copyright 2016 Klemens Morgenstern, Antony Polukhin -// -// 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) - -// For more information, see http://www.boost.org - -#ifndef BOOST_DLL_DETAIL_TYPE_INFO_HPP_ -#define BOOST_DLL_DETAIL_TYPE_INFO_HPP_ - -#include -#include - -namespace boost { namespace dll { namespace detail { - -#if defined(BOOST_MSVC) || defined(BOOST_MSVC_VER) - -#if defined ( _WIN64 ) - -template -const std::type_info& load_type_info(Lib & lib, Storage & storage) -{ - struct RTTICompleteObjectLocator - { - boost::detail::winapi::DWORD_ signature; //always zero ? - boost::detail::winapi::DWORD_ offset; //offset of this vtable in the complete class - boost::detail::winapi::DWORD_ cdOffset; //constructor displacement offset - boost::detail::winapi::DWORD_ pTypeDescriptorOffset; //TypeDescriptor of the complete class - boost::detail::winapi::DWORD_ pClassDescriptorOffset; //describes inheritance hierarchy (ignored) - }; - - RTTICompleteObjectLocator** vtable_p = &lib.template get(storage.template get_vtable()); - - vtable_p--; - auto vtable = *vtable_p; - - auto nat = reinterpret_cast(lib.native()); - - nat += vtable->pTypeDescriptorOffset; - - return *reinterpret_cast(nat); - -} - -#else - -template -const std::type_info& load_type_info(Lib & lib, Storage & storage) -{ - struct RTTICompleteObjectLocator - { - boost::detail::winapi::DWORD_ signature; //always zero ? - boost::detail::winapi::DWORD_ offset; //offset of this vtable in the complete class - boost::detail::winapi::DWORD_ cdOffset; //constructor displacement offset - const std::type_info* pTypeDescriptor; //TypeDescriptor of the complete class - void* pClassDescriptor; //describes inheritance hierarchy (ignored) - }; - - RTTICompleteObjectLocator** vtable_p = &lib.template get(storage.template get_vtable()); - - vtable_p--; - auto vtable = *vtable_p; - return *vtable->pTypeDescriptor; - -} - -#endif //_WIN64 - -#else - -template -const std::type_info& load_type_info(Lib & lib, Storage & storage) -{ - return lib.template get(storage.template get_type_info()); - -} - -#endif - - -}}} -#endif /* BOOST_DLL_DETAIL_TYPE_INFO_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp deleted file mode 100644 index 37eee86428c..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/path_from_handle.hpp +++ /dev/null @@ -1,62 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_WINDOWS_PATH_FROM_HANDLE_HPP -#define BOOST_DLL_DETAIL_WINDOWS_PATH_FROM_HANDLE_HPP - -#include -#include -#include -#include -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { namespace detail { - - static inline boost::system::error_code last_error_code() BOOST_NOEXCEPT { - boost::detail::winapi::DWORD_ err = boost::detail::winapi::GetLastError(); - return boost::system::error_code( - err, - boost::system::system_category() - ); - } - - inline boost::filesystem::path path_from_handle(boost::detail::winapi::HMODULE_ handle, boost::system::error_code &ec) { - BOOST_STATIC_CONSTANT(boost::detail::winapi::DWORD_, ERROR_INSUFFICIENT_BUFFER_ = 0x7A); - BOOST_STATIC_CONSTANT(boost::detail::winapi::DWORD_, DEFAULT_PATH_SIZE_ = 260); - - // If `handle` parameter is NULL, GetModuleFileName retrieves the path of the - // executable file of the current process. - boost::detail::winapi::WCHAR_ path_hldr[DEFAULT_PATH_SIZE_]; - boost::detail::winapi::GetModuleFileNameW(handle, path_hldr, DEFAULT_PATH_SIZE_); - ec = last_error_code(); - if (!ec) { - return boost::filesystem::path(path_hldr); - } - - for (unsigned i = 2; i < 1025 && static_cast(ec.value()) == ERROR_INSUFFICIENT_BUFFER_; i *= 2) { - std::wstring p(DEFAULT_PATH_SIZE_ * i, L'\0'); - const std::size_t size = boost::detail::winapi::GetModuleFileNameW(handle, &p[0], DEFAULT_PATH_SIZE_ * i); - ec = last_error_code(); - - if (!ec) { - p.resize(size); - return boost::filesystem::path(p); - } - } - - // Error other than ERROR_INSUFFICIENT_BUFFER_ occurred or failed to allocate buffer big enough - return boost::filesystem::path(); - } - -}}} // namespace boost::dll::detail - -#endif // BOOST_DLL_DETAIL_WINDOWS_PATH_FROM_HANDLE_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp deleted file mode 100644 index 48156505f47..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/windows/shared_library_impl.hpp +++ /dev/null @@ -1,177 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015-2016 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_SHARED_LIBRARY_IMPL_HPP -#define BOOST_DLL_SHARED_LIBRARY_IMPL_HPP - -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { namespace detail { - -class shared_library_impl { - BOOST_MOVABLE_BUT_NOT_COPYABLE(shared_library_impl) - -public: - typedef boost::detail::winapi::HMODULE_ native_handle_t; - - shared_library_impl() BOOST_NOEXCEPT - : handle_(NULL) - {} - - ~shared_library_impl() BOOST_NOEXCEPT { - unload(); - } - - shared_library_impl(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT - : handle_(sl.handle_) - { - sl.handle_ = NULL; - } - - shared_library_impl & operator=(BOOST_RV_REF(shared_library_impl) sl) BOOST_NOEXCEPT { - swap(sl); - return *this; - } - - void load(boost::filesystem::path sl, load_mode::type mode, boost::system::error_code &ec) { - typedef boost::detail::winapi::DWORD_ native_mode_t; - unload(); - - if (!sl.is_absolute() && !(mode & load_mode::search_system_folders)) { - - boost::system::error_code current_path_ec; - boost::filesystem::path prog_loc = boost::filesystem::current_path(current_path_ec); - if (!current_path_ec) { - prog_loc /= sl; - sl.swap(prog_loc); - } - } - mode &= ~load_mode::search_system_folders; - - // Trying to open with appended decorations - if (!!(mode & load_mode::append_decorations)) { - mode &= ~load_mode::append_decorations; - - handle_ = boost::detail::winapi::LoadLibraryExW((sl.native() + L".dll").c_str(), 0, static_cast(mode)); - if (!handle_) { - // MinGW loves 'lib' prefix and puts it even on Windows platform - const boost::filesystem::path load_path = (sl.has_parent_path() ? sl.parent_path() / L"lib" : L"lib").native() + sl.filename().native() + L".dll"; - handle_ = boost::detail::winapi::LoadLibraryExW( - load_path.c_str(), - 0, - static_cast(mode) - ); - } - - if (handle_) { - return; - } - } - - // From MSDN: If the string specifies a module name without a path and the - // file name extension is omitted, the function appends the default library - // extension .dll to the module name. - // - // From experiments: Default library extension appended to the module name even if - // we have some path. So we do not check for path, only for extension. We can not be sure that - // such behavior remain across all platforms, so we add L"." by hand. - if (sl.has_extension()) { - handle_ = boost::detail::winapi::LoadLibraryExW(sl.c_str(), 0, static_cast(mode)); - } else { - handle_ = boost::detail::winapi::LoadLibraryExW((sl.native() + L".").c_str(), 0, static_cast(mode)); - } - - // LoadLibraryExW method is capable of self loading from program_location() path. No special actions - // must be taken to allow self loading. - - if (!handle_) { - ec = boost::dll::detail::last_error_code(); - } - } - - bool is_loaded() const BOOST_NOEXCEPT { - return (handle_ != 0); - } - - void unload() BOOST_NOEXCEPT { - if (handle_) { - boost::detail::winapi::FreeLibrary(handle_); - handle_ = 0; - } - } - - void swap(shared_library_impl& rhs) BOOST_NOEXCEPT { - boost::swap(handle_, rhs.handle_); - } - - boost::filesystem::path full_module_path(boost::system::error_code &ec) const { - return boost::dll::detail::path_from_handle(handle_, ec); - } - - static boost::filesystem::path suffix() { - return L".dll"; - } - - void* symbol_addr(const char* sb, boost::system::error_code &ec) const BOOST_NOEXCEPT { - if (is_resource()) { - // `GetProcAddress` could not be called for libraries loaded with - // `LOAD_LIBRARY_AS_DATAFILE`, `LOAD_LIBRARY_AS_DATAFILE_EXCLUSIVE` - // or `LOAD_LIBRARY_AS_IMAGE_RESOURCE`. - ec = boost::system::error_code( - boost::system::errc::operation_not_supported, - boost::system::generic_category() - ); - - return NULL; - } - - // Judging by the documentation of GetProcAddress - // there is no version for UNICODE on desktop/server Windows, because - // names of functions are stored in narrow characters. - void* const symbol = boost::dll::detail::aggressive_ptr_cast( - boost::detail::winapi::get_proc_address(handle_, sb) - ); - if (symbol == NULL) { - ec = boost::dll::detail::last_error_code(); - } - - return symbol; - } - - native_handle_t native() const BOOST_NOEXCEPT { - return handle_; - } - -private: - bool is_resource() const BOOST_NOEXCEPT { - return false; /*!!( - reinterpret_cast(handle_) & static_cast(3) - );*/ - } - - native_handle_t handle_; -}; - -}}} // boost::dll::detail - -#endif // BOOST_DLL_SHARED_LIBRARY_IMPL_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp b/contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp deleted file mode 100644 index e53214e1cd6..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/detail/x_info_interface.hpp +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_DETAIL_X_INFO_INTERFACE_HPP -#define BOOST_DLL_DETAIL_X_INFO_INTERFACE_HPP - -#include -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -#include -#include - -namespace boost { namespace dll { namespace detail { - -class x_info_interface { -public: - virtual std::vector sections() = 0; - virtual std::vector symbols() = 0; - virtual std::vector symbols(const char* section_name) = 0; - - virtual ~x_info_interface() BOOST_NOEXCEPT {} -}; - -}}} // namespace boost::dll::detail - -#endif // BOOST_DLL_DETAIL_X_INFO_INTERFACE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/import.hpp b/contrib/libboost/boost_1_65_0/boost/dll/import.hpp deleted file mode 100644 index dcf3086a181..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/import.hpp +++ /dev/null @@ -1,277 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015-2016 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_IMPORT_HPP -#define BOOST_DLL_IMPORT_HPP - -#include -#include -#include -#include -#include -#include -#include - -#if defined(BOOST_NO_CXX11_TRAILING_RESULT_TYPES) || defined(BOOST_NO_CXX11_DECLTYPE) || defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) || defined(BOOST_NO_CXX11_RVALUE_REFERENCES) -# include -#endif - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -/// \file boost/dll/import.hpp -/// \brief Contains all the boost::dll::import* reference counting -/// functions that hold a shared pointer to the instance of -/// boost::dll::shared_library. - -namespace boost { namespace dll { - - -namespace detail { - - template - class library_function { - // Copying of `boost::dll::shared_library` is very expensive, so we use a `shared_ptr` to make it faster. - boost::shared_ptr f_; - - public: - inline library_function(const boost::shared_ptr& lib, T* func_ptr) BOOST_NOEXCEPT - : f_(lib, func_ptr) - {} - -#if defined(BOOST_NO_CXX11_TRAILING_RESULT_TYPES) || defined(BOOST_NO_CXX11_DECLTYPE) || defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) || defined(BOOST_NO_CXX11_RVALUE_REFERENCES) - operator T*() const BOOST_NOEXCEPT { - return f_.get(); - } -#else - - // Compilation error at this point means that imported function - // was called with unmatching parameters. - // - // Example: - // auto f = dll::import("function", "lib.so"); - // f("Hello"); // error: invalid conversion from 'const char*' to 'int' - // f(1, 2); // error: too many arguments to function - // f(); // error: too few arguments to function - template - inline auto operator()(Args&&... args) const - -> decltype( (*f_)(static_cast(args)...) ) - { - return (*f_)(static_cast(args)...); - } -#endif - }; - - template - struct import_type; - - template - struct import_type >::type> { - typedef boost::dll::detail::library_function base_type; - -#if defined(BOOST_NO_CXX11_TRAILING_RESULT_TYPES) || defined(BOOST_NO_CXX11_DECLTYPE) || defined(BOOST_NO_CXX11_VARIADIC_TEMPLATES) || defined(BOOST_NO_CXX11_RVALUE_REFERENCES) - typedef boost::function type; -#else - typedef boost::dll::detail::library_function type; -#endif - }; - - template - struct import_type >::type> { - typedef boost::shared_ptr base_type; - typedef boost::shared_ptr type; - }; -} // namespace detail - - -#ifndef BOOST_DLL_DOXYGEN -# define BOOST_DLL_IMPORT_RESULT_TYPE inline typename boost::dll::detail::import_type::type -#endif - - -/*! -* Returns callable object or boost::shared_ptr that holds the symbol imported -* from the loaded library. Returned value refcounts usage -* of the loaded shared library, so that it won't get unload until all copies of return value -* are not destroyed. -* -* This call will succeed if call to \forcedlink{shared_library}`::has(const char* )` -* function with the same symbol name returned `true`. -* -* For importing symbols by \b alias names use \forcedlink{import_alias} method. -* -* \b Examples: -* -* \code -* boost::function f = import("test_lib.so", "integer_func_name"); -* -* auto f_cpp11 = import("test_lib.so", "integer_func_name"); -* \endcode -* -* \code -* boost::shared_ptr i = import("test_lib.so", "integer_name"); -* \endcode -* -* \b Template \b parameter \b T: Type of the symbol that we are going to import. Must be explicitly specified. -* -* \param lib Path to shared library or shared library to load function from. -* \param name Null-terminated C or C++ mangled name of the function to import. Can handle std::string, char*, const char*. -* \param mode An mode that will be used on library load. -* -* \return callable object if T is a function type, or boost::shared_ptr if T is an object type. -* -* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. -* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. -*/ -template -BOOST_DLL_IMPORT_RESULT_TYPE import(const boost::filesystem::path& lib, const char* name, - load_mode::type mode = load_mode::default_mode) -{ - typedef typename boost::dll::detail::import_type::base_type type; - - boost::shared_ptr p = boost::make_shared(lib, mode); - return type(p, boost::addressof(p->get(name))); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import(const boost::filesystem::path& lib, const std::string& name, - load_mode::type mode = load_mode::default_mode) -{ - return import(lib, name.c_str(), mode); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import(const shared_library& lib, const char* name) { - typedef typename boost::dll::detail::import_type::base_type type; - - boost::shared_ptr p = boost::make_shared(lib); - return type(p, boost::addressof(p->get(name))); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import(const shared_library& lib, const std::string& name) { - return import(lib, name.c_str()); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import(BOOST_RV_REF(shared_library) lib, const char* name) { - typedef typename boost::dll::detail::import_type::base_type type; - - boost::shared_ptr p = boost::make_shared( - boost::move(lib) - ); - return type(p, boost::addressof(p->get(name))); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import(BOOST_RV_REF(shared_library) lib, const std::string& name) { - return import(boost::move(lib), name.c_str()); -} - - - - -/*! -* Returns callable object or boost::shared_ptr that holds the symbol imported -* from the loaded library. Returned value refcounts usage -* of the loaded shared library, so that it won't get unload until all copies of return value -* are not destroyed. -* -* This call will succeed if call to \forcedlink{shared_library}`::has(const char* )` -* function with the same symbol name returned `true`. -* -* For importing symbols by \b non \b alias names use \forcedlink{import} method. -* -* \b Examples: -* -* \code -* boost::function f = import_alias("test_lib.so", "integer_func_alias_name"); -* -* auto f_cpp11 = import_alias("test_lib.so", "integer_func_alias_name"); -* \endcode -* -* \code -* boost::shared_ptr i = import_alias("test_lib.so", "integer_alias_name"); -* \endcode -* -* \code -* \endcode -* -* \b Template \b parameter \b T: Type of the symbol alias that we are going to import. Must be explicitly specified. -* -* \param lib Path to shared library or shared library to load function from. -* \param name Null-terminated C or C++ mangled name of the function or variable to import. Can handle std::string, char*, const char*. -* \param mode An mode that will be used on library load. -* -* \return callable object if T is a function type, or boost::shared_ptr if T is an object type. -* -* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. -* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. -*/ -template -BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const boost::filesystem::path& lib, const char* name, - load_mode::type mode = load_mode::default_mode) -{ - typedef typename boost::dll::detail::import_type::base_type type; - - boost::shared_ptr p = boost::make_shared(lib, mode); - return type(p, p->get(name)); -} - -//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const boost::filesystem::path& lib, const std::string& name, - load_mode::type mode = load_mode::default_mode) -{ - return import_alias(lib, name.c_str(), mode); -} - -//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const shared_library& lib, const char* name) { - typedef typename boost::dll::detail::import_type::base_type type; - - boost::shared_ptr p = boost::make_shared(lib); - return type(p, p->get(name)); -} - -//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import_alias(const shared_library& lib, const std::string& name) { - return import_alias(lib, name.c_str()); -} - -//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import_alias(BOOST_RV_REF(shared_library) lib, const char* name) { - typedef typename boost::dll::detail::import_type::base_type type; - - boost::shared_ptr p = boost::make_shared( - boost::move(lib) - ); - return type(p, p->get(name)); -} - -//! \overload boost::dll::import_alias(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_IMPORT_RESULT_TYPE import_alias(BOOST_RV_REF(shared_library) lib, const std::string& name) { - return import_alias(boost::move(lib), name.c_str()); -} - -#undef BOOST_DLL_IMPORT_RESULT_TYPE - - -}} // boost::dll - -#endif // BOOST_DLL_IMPORT_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp b/contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp deleted file mode 100644 index 866fc9c0459..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/import_class.hpp +++ /dev/null @@ -1,558 +0,0 @@ -// Copyright 2015-2016 Klemens D. Morgenstern -// -// 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) - -#ifndef BOOST_DLL_IMPORT_CLASS_HPP_ -#define BOOST_DLL_IMPORT_CLASS_HPP_ - -#include -#include -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { namespace experimental { - -namespace detail -{ - -template -struct deleter -{ - destructor dtor; - bool use_deleting; - - deleter(const destructor & dtor, bool use_deleting = false) : - dtor(dtor), use_deleting(use_deleting) {} - - void operator()(T*t) - { - if (use_deleting) - dtor.call_deleting(t); - else - { - dtor.call_standard(t); - //the thing is actually an array, so delete[] - auto p = reinterpret_cast(t); - delete [] p; - } - } -}; - -template -struct mem_fn_call_proxy; - -template -struct mem_fn_call_proxy> -{ - typedef boost::dll::experimental::detail::mangled_library_mem_fn mem_fn_t; - Class* t; - mem_fn_t & mem_fn; - - mem_fn_call_proxy(mem_fn_call_proxy&&) = default; - mem_fn_call_proxy(const mem_fn_call_proxy & ) = delete; - mem_fn_call_proxy(Class * t, mem_fn_t & mem_fn) - : t(t), mem_fn(mem_fn) {} - - template - auto operator()(Args&&...args) const - { - return mem_fn(t, std::forward(args)...); - } - -}; - -template -struct mem_fn_call_proxy -{ - T* t; - const std::string &name; - smart_library &_lib; - - mem_fn_call_proxy(mem_fn_call_proxy&&) = default; - mem_fn_call_proxy(const mem_fn_call_proxy&) = delete; - mem_fn_call_proxy(T *t, const std::string &name, smart_library & _lib) - : t(t), name(name), _lib(_lib) {}; - - Return operator()(Args...args) const - { - auto f = _lib.get_mem_fn(name); - return (t->*f)(static_cast(args)...); - } -}; - -} - -template -class imported_class; - -template imported_class -import_class(const smart_library& lib, Args...args); -template imported_class -import_class(const smart_library& lib, const std::string & alias_name, Args...args); -template imported_class -import_class(const smart_library& lib, std::size_t size, Args...args); -template imported_class -import_class(const smart_library& lib, std::size_t size, - const std::string & alias_name, Args...args); - - -/*! This class represents an imported class. - * - * \note It must be constructed via \ref boost::dll::import_class(const smart_library& lib, std::size_t, Args...) - * - * \tparam The type or type-alias of the imported class. - */ -template -class imported_class -{ - smart_library _lib; - std::unique_ptr> _data; - bool _is_allocating; - std::size_t _size; - const std::type_info& _ti; - - template - inline std::unique_ptr> make_data(const smart_library& lib, Args ... args); - template - inline std::unique_ptr> make_data(const smart_library& lib, std::size_t size, Args...args); - - template - imported_class(detail::sequence *, const smart_library& lib, Args...args); - - template - imported_class(detail::sequence *, const smart_library& lib, std::size_t size, Args...args); - - template - imported_class(detail::sequence *, smart_library&& lib, Args...args); - - template - imported_class(detail::sequence *, smart_library&& lib, std::size_t size, Args...args); -public: - //alias to construct with explicit parameter list - template - static imported_class make(smart_library&& lib, Args...args) - { - typedef detail::sequence *seq; - return imported_class(seq(), boost::move(lib), static_cast(args)...); - } - - template - static imported_class make(smart_library&& lib, std::size_t size, Args...args) - { - typedef detail::sequence *seq; - return imported_class(seq(), boost::move(lib), size, static_cast(args)...); - } - template - static imported_class make(const smart_library& lib, Args...args) - { - typedef detail::sequence *seq; - return imported_class(seq(), lib, static_cast(args)...); - } - - template - static imported_class make(const smart_library& lib, std::size_t size, Args...args) - { - typedef detail::sequence *seq; - return imported_class(seq(), lib, size, static_cast(args)...); - } - - typedef imported_class base_t; - ///Returns a pointer to the underlying class - T* get() {return _data.get();} - imported_class() = delete; - - imported_class(imported_class&) = delete; - imported_class(imported_class&&) = default; /// ().empty();} - ///Check if the imported class is move-assignable - bool is_move_assignable() {return !_lib.symbol_storage().template get_mem_fn ("operator=").empty();} - ///Check if the imported class is copy-constructible - bool is_copy_constructible() {return !_lib.symbol_storage().template get_constructor().empty();} - ///Check if the imported class is copy-assignable - bool is_copy_assignable() {return !_lib.symbol_storage().template get_mem_fn("operator=").empty();} - - imported_class copy() const; /// move(); /// & lhs) const; - ///Invoke the move assignment. \attention Undefined behaviour if the imported object is not move assignable. - void move_assign( imported_class & lhs); - - ///Check if the class is loaded. - explicit operator bool() const {return _data;} - - ///Get a const reference to the std::type_info. - const std::type_info& get_type_info() {return _ti;}; - - /*! Call a member function. This returns a proxy to the function. - * The proxy mechanic mechanic is necessary, so the signaute can be passed. - * - * \b Example - * - * \code - * im_class.call("function_name")("MyString"); - * \endcode - */ - template - const detail::mem_fn_call_proxy call(const std::string& name) - { - return detail::mem_fn_call_proxy(_data.get(), name, _lib); - } - /*! Call a qualified member function, i.e. const and or volatile. - * - * \b Example - * - * \code - * im_class.call("function_name")("MyString"); - * \endcode - */ - template>> - const detail::mem_fn_call_proxy call(const std::string& name) - { - return detail::mem_fn_call_proxy(_data.get(), name, _lib); - } - ///Overload of ->* for an imported method. - template - const detail::mem_fn_call_proxy> - operator->*(detail::mangled_library_mem_fn& mn) - { - return detail::mem_fn_call_proxy>(_data.get(), mn); - } - - ///Import a method of the class. - template - typename boost::dll::experimental::detail::mangled_import_type>::type - import(const std::string & name) - { - return boost::dll::experimental::import_mangled(_lib, name); - } -}; - - - -//helper function, uses the allocating -template -template -inline std::unique_ptr> imported_class::make_data(const smart_library& lib, Args ... args) -{ - constructor ctor = lib.get_constructor(); - destructor dtor = lib.get_destructor (); - - if (!ctor.has_allocating() || !dtor.has_deleting()) - { - boost::system::error_code ec; - - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - // report_error() calls dlsym, do not use it here! - boost::throw_exception( - boost::system::system_error( - ec, "boost::dll::detail::make_data() failed: no allocating ctor or dtor was found" - ) - ); - } - - return std::unique_ptr> ( - ctor.call_allocating(static_cast(args)...), - detail::deleter(dtor, false /* not deleting dtor*/)); -} - -//helper function, using the standard -template -template -inline std::unique_ptr> imported_class::make_data(const smart_library& lib, std::size_t size, Args...args) -{ - constructor ctor = lib.get_constructor(); - destructor dtor = lib.get_destructor (); - - if (!ctor.has_standard() || !dtor.has_standard()) - { - boost::system::error_code ec; - - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - // report_error() calls dlsym, do not use it here! - boost::throw_exception( - boost::system::system_error( - ec, "boost::dll::detail::make_data() failed: no regular ctor or dtor was found" - ) - ); - } - - T *data = reinterpret_cast(new char[size]); - - ctor.call_standard(data, static_cast(args)...); - - return std::unique_ptr> ( - reinterpret_cast(data), - detail::deleter(dtor, false /* not deleting dtor*/)); - -} - - -template -template -imported_class::imported_class(detail::sequence *, const smart_library & lib, Args...args) - : _lib(lib), - _data(make_data(lib, static_cast(args)...)), - _is_allocating(false), - _size(0), - _ti(lib.get_type_info()) -{ - -} - -template -template -imported_class::imported_class(detail::sequence *, const smart_library & lib, std::size_t size, Args...args) - : _lib(lib), - _data(make_data(lib, size, static_cast(args)...)), - _is_allocating(true), - _size(size), - _ti(lib.get_type_info()) -{ - -} - -template -template -imported_class::imported_class(detail::sequence *, smart_library && lib, Args...args) - : _lib(boost::move(lib)), - _data(make_data(lib, static_cast(args)...)), - _is_allocating(false), - _size(0), - _ti(lib.get_type_info()) -{ - -} - -template -template -imported_class::imported_class(detail::sequence *, smart_library && lib, std::size_t size, Args...args) - : _lib(boost::move(lib)), - _data(make_data(lib, size, static_cast(args)...)), - _is_allocating(true), - _size(size), - _ti(lib.get_type_info()) -{ - -} - -template -inline imported_class boost::dll::experimental::imported_class::copy() const -{ - if (this->_is_allocating) - return imported_class::template make(_lib, *_data); - else - return imported_class::template make(_lib, _size, *_data); -} - -template -inline imported_class boost::dll::experimental::imported_class::move() -{ - if (this->_is_allocating) - return imported_class::template make(_lib, *_data); - else - return imported_class::template make(_lib, _size, *_data); -} - -template -inline void boost::dll::experimental::imported_class::copy_assign(const imported_class& lhs) const -{ - this->call("operator=")(*lhs._data); -} - -template -inline void boost::dll::experimental::imported_class::move_assign(imported_class& lhs) -{ - this->call("operator=")(static_cast(*lhs._data)); -} - - - -/*! -* Returns an instance of \ref imported_class which allows to call or import more functions. -* It takes a copy of the smart_libray, so no added type_aliases will be visible, -* for the object. -* -* Few compilers do implement an allocating constructor, which allows the construction -* of the class without knowing the size. That is not portable, so the actual size of the class -* shall always be provided. -* -* \b Example: -* -* \code -* auto import_class(lib, "class_name", 20, "param1", 42); -* \endcode -* -* In this example we construct an instance of the class "class_name" with the size 20, which has "type_alias" as an alias, -* through a constructor which takes a const-ref of std::string and an std::size_t parameter. -* -* \tparam T Class type or alias -* \tparam Args Constructor argument list. -* \param lib Path to shared library or shared library to load function from. -* \param name Null-terminated C or C++ mangled name of the function to import. Can handle std::string, char*, const char*. -* \param mode An mode that will be used on library load. -* -* \return class object. -* -* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. -* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. -*/ -template imported_class -import_class(const smart_library& lib_, std::size_t size, Args...args) -{ - smart_library lib(lib_); - - return imported_class::template make(boost::move(lib), size, static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(const smart_library& lib_, Args...args) -{ - smart_library lib(lib_); - return imported_class::template make(boost::move(lib), static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(const smart_library& lib_, const std::string & alias_name, Args...args) -{ - smart_library lib(lib_); - lib.add_type_alias(alias_name); - return imported_class::template make(boost::move(lib), static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(const smart_library& lib_, std::size_t size, const std::string & alias_name, Args...args) -{ - smart_library lib(lib_); - - lib.add_type_alias(alias_name); - return imported_class::template make(boost::move(lib), size, static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(const smart_library& lib_, const std::string & alias_name, std::size_t size, Args...args) -{ - smart_library lib(lib_); - - lib.add_type_alias(alias_name); - return imported_class::template make(boost::move(lib), size, static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(smart_library && lib, Args...args) -{ - return imported_class::template make(boost::move(lib), static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(smart_library && lib, const std::string & alias_name, Args...args) -{ - lib.add_type_alias(alias_name); - return imported_class::template make(boost::move(lib), static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(smart_library && lib, std::size_t size, Args...args) -{ - return imported_class::template make(boost::move(lib), size, static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(smart_library && lib, std::size_t size, const std::string & alias_name, Args...args) -{ - lib.add_type_alias(alias_name); - return imported_class::template make(boost::move(lib), size, static_cast(args)...); -} - -//! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) -template imported_class -import_class(smart_library && lib, const std::string & alias_name, std::size_t size, Args...args) -{ - lib.add_type_alias(alias_name); - return imported_class::template make(boost::move(lib), size, static_cast(args)...); -} - - - -/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) - * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. - */ - -template imported_class -import_class(smart_library & lib, Args...args) -{ - return imported_class::template make(lib, static_cast(args)...); -} - -/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) - * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. - */ -template imported_class -import_class(smart_library & lib, const std::string & alias_name, Args...args) -{ - lib.add_type_alias(alias_name); - return imported_class::template make(lib, static_cast(args)...); -} - -/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) - * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. - */ -template imported_class -import_class(smart_library & lib, std::size_t size, Args...args) -{ - return imported_class::template make(lib, size, static_cast(args)...); -} - -/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) - * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. - */ -template imported_class -import_class(smart_library & lib, std::size_t size, const std::string & alias_name, Args...args) -{ - lib.add_type_alias(alias_name); - return imported_class::template make(lib, size, static_cast(args)...); -} - -/*! \overload boost::dll::import_class(const smart_library& lib, std::size_t, Args...) - * \note This function does add the type alias to the \ref boost::dll::experimental::smart_library. - */ -template imported_class -import_class(smart_library & lib, const std::string & alias_name, std::size_t size, Args...args) -{ - lib.add_type_alias(alias_name); - return imported_class::template make(lib, size, static_cast(args)...); -} - -} -} -} - - - -#endif /* BOOST_DLL_IMPORT_CLASS_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp b/contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp deleted file mode 100644 index 69490ed19d7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/import_mangled.hpp +++ /dev/null @@ -1,309 +0,0 @@ -// Copyright 2015-2016 Klemens D. Morgenstern -// -// 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) - - -#ifndef BOOST_DLL_IMPORT_MANGLED_HPP_ -#define BOOST_DLL_IMPORT_MANGLED_HPP_ - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { namespace experimental { - -namespace detail -{ - - - -template -class mangled_library_function { - // Copying of `boost::dll::shared_library` is very expensive, so we use a `shared_ptr` to make it faster. - boost::shared_ptr lib_; - function_tuple f_; -public: - constexpr mangled_library_function(const boost::shared_ptr& lib, Ts*... func_ptr) BOOST_NOEXCEPT - : lib_(lib) - , f_(func_ptr...) - {} - - - // Compilation error at this point means that imported function - // was called with unmatching parameters. - // - // Example: - // auto f = dll::import_mangled("function", "lib.so"); - // f("Hello"); // error: invalid conversion from 'const char*' to 'int' - // f(1, 2); // error: too many arguments to function - // f(); // error: too few arguments to function - template - auto operator()(Args&&... args) const - -> decltype( f_(static_cast(args)...) ) - { - return f_(static_cast(args)...); - } -}; - - -template -class mangled_library_mem_fn; - -template -class mangled_library_mem_fn> { - // Copying of `boost::dll::shared_library` is very expensive, so we use a `shared_ptr` to make it faster. - typedef mem_fn_tuple call_tuple_t; - boost::shared_ptr lib_; - call_tuple_t f_; - -public: - constexpr mangled_library_mem_fn(const boost::shared_ptr& lib, typename Ts::mem_fn... func_ptr) BOOST_NOEXCEPT - : lib_(lib) - , f_(func_ptr...) - {} - - template - auto operator()(ClassIn *cl, Args&&... args) const - -> decltype( f_(cl, static_cast(args)...) ) - { - return f_(cl, static_cast(args)...); - } -}; - - - - -// simple enough to be here -template struct is_variable : boost::false_type {}; -template struct is_variable> : boost::is_object {}; - -template ::value, - bool isMemFn = is_mem_fn_seq ::value, - bool isVariable = is_variable ::value> -struct mangled_import_type; - -template -struct mangled_import_type, true,false,false> //is function -{ - typedef boost::dll::experimental::detail::mangled_library_function type; - static type make( - const boost::dll::experimental::smart_library& p, - const std::string& name) - { - return type( - boost::make_shared(p.shared_lib()), - boost::addressof(p.get_function(name))...); - } -}; - -template -struct mangled_import_type, false, true, false> //is member-function -{ - typedef typename boost::dll::experimental::detail::make_mem_fn_seq::type actual_sequence; - typedef typename boost::dll::experimental::detail::mangled_library_mem_fn type; - - - template - static type make_impl( - const boost::dll::experimental::smart_library& p, - const std::string & name, - sequence * ) - { - return type(boost::make_shared(p.shared_lib()), - p.get_mem_fn(name)...); - } - - static type make( - const boost::dll::experimental::smart_library& p, - const std::string& name) - { - return make_impl(p, name, static_cast(nullptr)); - } - -}; - -template -struct mangled_import_type, false, false, true> //is variable -{ - typedef boost::shared_ptr type; - - static type make( - const boost::dll::experimental::smart_library& p, - const std::string& name) - { - return type( - boost::make_shared(p.shared_lib()), - boost::addressof(p.get_variable(name))); - } - -}; - - -} // namespace detail - - -#ifndef BOOST_DLL_DOXYGEN -# define BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE inline typename \ - boost::dll::experimental::detail::mangled_import_type>::type -#endif - -/* - * Variants: - * import_mangled("Stuff"); - * import_mangled("Function"); - * import mangled("Function"); - */ - -/*! -* Returns callable object or boost::shared_ptr that holds the symbol imported -* from the loaded library. Returned value refcounts usage -* of the loaded shared library, so that it won't get unload until all copies of return value -* are not destroyed. -* -* For importing symbols by \b alias names use \forcedlink{import_alias} method. -* -* \b Examples: -* -* \code -* boost::function f = import_mangled("test_lib.so", "integer_func_name"); -* -* auto f_cpp11 = import_mangled("test_lib.so", "integer_func_name"); -* \endcode -* -* \code -* boost::shared_ptr i = import_mangled("test_lib.so", "integer_name"); -* \endcode -* -* Additionally you can also import overloaded symbols, including member-functions. -* -* \code -* auto fp = import_mangled("test_lib.so", "func"); -* \endcode -* -* \code -* auto fp = import_mangled("test_lib.so", "func"); -* \endcode -* -* If qualified member-functions are needed, this can be set by repeating the class name with const or volatile. -* All following signatures after the redifintion will use this, i.e. the latest. -* -* * * \code -* auto fp = import_mangled("test_lib.so", "func"); -* \endcode -* -* \b Template \b parameter \b T: Type of the symbol that we are going to import. Must be explicitly specified. -* -* \param lib Path to shared library or shared library to load function from. -* \param name Null-terminated C or C++ mangled name of the function to import. Can handle std::string, char*, const char*. -* \param mode An mode that will be used on library load. -* -* \return callable object if T is a function type, or boost::shared_ptr if T is an object type. -* -* \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. -* Overload that accepts path also throws std::bad_alloc in case of insufficient memory. -*/ - - -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const boost::filesystem::path& lib, const char* name, - load_mode::type mode = load_mode::default_mode) -{ - typedef typename boost::dll::experimental::detail::mangled_import_type< - boost::dll::experimental::detail::sequence> type; - - boost::dll::experimental::smart_library p(lib, mode); - //the load - return type::make(p, name); -} - - - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const boost::filesystem::path& lib, const std::string& name, - load_mode::type mode = load_mode::default_mode) -{ - return import_mangled(lib, name.c_str(), mode); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const smart_library& lib, const char* name) { - typedef typename boost::dll::experimental::detail::mangled_import_type> type; - - return type::make(lib, name); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const smart_library& lib, const std::string& name) { - return import_mangled(lib, name.c_str()); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(smart_library) lib, const char* name) { - typedef typename boost::dll::experimental::detail::mangled_import_type> type; - - return type::make(lib, name); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(smart_library) lib, const std::string& name) { - return import_mangled(boost::move(lib), name.c_str()); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const shared_library& lib, const char* name) { - typedef typename boost::dll::experimental::detail::mangled_import_type> type; - - boost::shared_ptr p = boost::make_shared(lib); - return type::make(p, name); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(const shared_library& lib, const std::string& name) { - return import_mangled(lib, name.c_str()); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(shared_library) lib, const char* name) { - typedef typename boost::dll::experimental::detail::mangled_import_type> type; - - boost::dll::experimental::smart_library p(boost::move(lib)); - - return type::make(p, name); -} - -//! \overload boost::dll::import(const boost::filesystem::path& lib, const char* name, load_mode::type mode) -template -BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE import_mangled(BOOST_RV_REF(shared_library) lib, const std::string& name) { - return import_mangled(boost::move(lib), name.c_str()); -} - -#undef BOOST_DLL_MANGLED_IMPORT_RESULT_TYPE - -}}} - - -#endif /* BOOST_DLL_IMPORT_MANGLED_HPP_ */ diff --git a/contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp deleted file mode 100644 index 4666a16353a..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/library_info.hpp +++ /dev/null @@ -1,181 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_LIBRARY_INFO_HPP -#define BOOST_DLL_LIBRARY_INFO_HPP - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -/// \file boost/dll/library_info.hpp -/// \brief Contains only the boost::dll::library_info class that is capable of -/// extracting different information from binaries. - -namespace boost { namespace dll { - -/*! -* \brief Class that is capable of extracting different information from a library or binary file. -* Currently understands ELF, MACH-O and PE formats on all the platforms. -*/ -class library_info: private boost::noncopyable { -private: - boost::filesystem::ifstream f_; - - boost::aligned_storage< // making my own std::aligned_union from scratch. TODO: move to TypeTraits - boost::mpl::deref< - boost::mpl::max_element< - boost::mpl::vector_c - >::type - >::type::value - >::type impl_; - - /// @cond - boost::dll::detail::x_info_interface& impl() BOOST_NOEXCEPT { - return *reinterpret_cast(impl_.address()); - } - - inline static void throw_if_in_32bit_impl(boost::true_type /* is_32bit_platform */) { - boost::throw_exception(std::runtime_error("Not native format: 64bit binary")); - } - - inline static void throw_if_in_32bit_impl(boost::false_type /* is_32bit_platform */) BOOST_NOEXCEPT {} - - - inline static void throw_if_in_32bit() { - throw_if_in_32bit_impl( boost::integral_constant() ); - } - - static void throw_if_in_windows() { -#if BOOST_OS_WINDOWS - boost::throw_exception(std::runtime_error("Not native format: not a PE binary")); -#endif - } - - static void throw_if_in_linux() { -#if !BOOST_OS_WINDOWS && !BOOST_OS_MACOS && !BOOST_OS_IOS - boost::throw_exception(std::runtime_error("Not native format: not an ELF binary")); -#endif - } - - static void throw_if_in_macos() { -#if BOOST_OS_MACOS || BOOST_OS_IOS - boost::throw_exception(std::runtime_error("Not native format: not an Mach-O binary")); -#endif - } - - void init(bool throw_if_not_native) { - - if (boost::dll::detail::elf_info32::parsing_supported(f_)) { - if (throw_if_not_native) { throw_if_in_windows(); throw_if_in_macos(); } - - new (impl_.address()) boost::dll::detail::elf_info32(f_); - } else if (boost::dll::detail::elf_info64::parsing_supported(f_)) { - if (throw_if_not_native) { throw_if_in_windows(); throw_if_in_macos(); throw_if_in_32bit(); } - - new (impl_.address()) boost::dll::detail::elf_info64(f_); - } else if (boost::dll::detail::pe_info32::parsing_supported(f_)) { - if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_macos(); } - - new (impl_.address()) boost::dll::detail::pe_info32(f_); - } else if (boost::dll::detail::pe_info64::parsing_supported(f_)) { - if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_macos(); throw_if_in_32bit(); } - - new (impl_.address()) boost::dll::detail::pe_info64(f_); - } else if (boost::dll::detail::macho_info32::parsing_supported(f_)) { - if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_windows(); } - - new (impl_.address()) boost::dll::detail::macho_info32(f_); - } else if (boost::dll::detail::macho_info64::parsing_supported(f_)) { - if (throw_if_not_native) { throw_if_in_linux(); throw_if_in_windows(); throw_if_in_32bit(); } - - new (impl_.address()) boost::dll::detail::macho_info64(f_); - } else { - boost::throw_exception(std::runtime_error("Unsupported binary format")); - } - } - /// @endcond - -public: - /*! - * Opens file with specified path and prepares for information extraction. - * \param library_path Path to the binary file from which the info must be extracted. - * \param throw_if_not_native_format Throw an exception if this file format is not - * supported by OS. - */ - explicit library_info(const boost::filesystem::path& library_path, bool throw_if_not_native_format = true) - : f_(library_path, std::ios_base::in | std::ios_base::binary) - , impl_() - { - f_.exceptions( - boost::filesystem::ifstream::failbit - | boost::filesystem::ifstream::badbit - | boost::filesystem::ifstream::eofbit - ); - - init(throw_if_not_native_format); - } - - /*! - * \return List of sections that exist in binary file. - */ - std::vector sections() { - return impl().sections(); - } - - /*! - * \return List of all the exportable symbols from all the sections that exist in binary file. - */ - std::vector symbols() { - return impl().symbols(); - } - - /*! - * \param section_name Name of the section from which symbol names must be returned. - * \return List of symbols from the specified section. - */ - std::vector symbols(const char* section_name) { - return impl().symbols(section_name); - } - - - //! \overload std::vector symbols(const char* section_name) - std::vector symbols(const std::string& section_name) { - return impl().symbols(section_name.c_str()); - } - - /*! - * \throw Nothing. - */ - ~library_info() BOOST_NOEXCEPT { - typedef boost::dll::detail::x_info_interface T; - impl().~T(); - } -}; - -}} // namespace boost::dll -#endif // BOOST_DLL_LIBRARY_INFO_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp b/contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp deleted file mode 100644 index 3f94e2ee1bf..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/runtime_symbol_info.hpp +++ /dev/null @@ -1,237 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015-2017 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_RUNTIME_SYMBOL_INFO_HPP -#define BOOST_DLL_RUNTIME_SYMBOL_INFO_HPP - -#include -#include -#include -#include -#if BOOST_OS_WINDOWS -# include -# include -#else -# include -# include -#endif - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -/// \file boost/dll/runtime_symbol_info.hpp -/// \brief Provides methods for getting acceptable by boost::dll::shared_library location of symbol, source line or program. -namespace boost { namespace dll { - -#if BOOST_OS_WINDOWS -namespace detail { - inline boost::filesystem::path program_location_impl(boost::system::error_code& ec) { - return boost::dll::detail::path_from_handle(NULL, ec); - } -} // namespace detail -#endif - - /*! - * On success returns full path and name to the binary object that holds symbol pointed by ptr_to_symbol. - * - * \param ptr_to_symbol Pointer to symbol which location is to be determined. - * \param ec Variable that will be set to the result of the operation. - * \return Path to the binary object that holds symbol or empty path in case error. - * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. - * - * \b Examples: - * \code - * int main() { - * dll::symbol_location_ptr(std::set_terminate(0)); // returns "/some/path/libmy_terminate_handler.so" - * dll::symbol_location_ptr(::signal(SIGSEGV, SIG_DFL)); // returns "/some/path/libmy_symbol_handler.so" - * } - * \endcode - */ - template - inline boost::filesystem::path symbol_location_ptr(T ptr_to_symbol, boost::system::error_code& ec) { - BOOST_STATIC_ASSERT_MSG(boost::is_pointer::value, "boost::dll::symbol_location_ptr works only with pointers! `ptr_to_symbol` must be a pointer"); - boost::filesystem::path ret; - if (!ptr_to_symbol) { - ec = boost::system::error_code( - boost::system::errc::bad_address, - boost::system::generic_category() - ); - - return ret; - } - ec.clear(); - - const void* ptr = boost::dll::detail::aggressive_ptr_cast(ptr_to_symbol); - -#if BOOST_OS_WINDOWS - boost::detail::winapi::MEMORY_BASIC_INFORMATION_ mbi; - if (!boost::detail::winapi::VirtualQuery(ptr, &mbi, sizeof(mbi))) { - ec = boost::dll::detail::last_error_code(); - return ret; - } - - return boost::dll::detail::path_from_handle(reinterpret_cast(mbi.AllocationBase), ec); -#else - Dl_info info; - - // Some of the libc headers miss `const` in `dladdr(const void*, Dl_info*)` - const int res = dladdr(const_cast(ptr), &info); - - if (res) { - ret = info.dli_fname; - } else { - boost::dll::detail::reset_dlerror(); - ec = boost::system::error_code( - boost::system::errc::bad_address, - boost::system::generic_category() - ); - } - - return ret; -#endif - } - - //! \overload symbol_location_ptr(const void* ptr_to_symbol, boost::system::error_code& ec) - template - inline boost::filesystem::path symbol_location_ptr(T ptr_to_symbol) { - boost::filesystem::path ret; - boost::system::error_code ec; - ret = boost::dll::symbol_location_ptr(ptr_to_symbol, ec); - - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::symbol_location_ptr(T ptr_to_symbol) failed"); - } - - return ret; - } - - /*! - * On success returns full path and name of the binary object that holds symbol. - * - * \tparam T Type of the symbol, must not be explicitly specified. - * \param symbol Symbol which location is to be determined. - * \param ec Variable that will be set to the result of the operation. - * \return Path to the binary object that holds symbol or empty path in case error. - * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. - * - * \b Examples: - * \code - * int var; - * void foo() {} - * - * int main() { - * dll::symbol_location(var); // returns program location - * dll::symbol_location(foo); // returns program location - * dll::symbol_location(std::cerr); // returns location of libstdc++: "/usr/lib/x86_64-linux-gnu/libstdc++.so.6" - * dll::symbol_location(std::placeholders::_1); // returns location of libstdc++: "/usr/lib/x86_64-linux-gnu/libstdc++.so.6" - * dll::symbol_location(std::puts); // returns location of libc: "/lib/x86_64-linux-gnu/libc.so.6" - * } - * \endcode - */ - template - inline boost::filesystem::path symbol_location(const T& symbol, boost::system::error_code& ec) { - ec.clear(); - return boost::dll::symbol_location_ptr( - boost::dll::detail::aggressive_ptr_cast(boost::addressof(symbol)), - ec - ); - } - -#if BOOST_COMP_MSVC < BOOST_VERSION_NUMBER(14,0,0) - // Without this MSVC 7.1 fails with: - // ..\boost\dll\runtime_symbol_info.hpp(133) : error C2780: 'filesystem::path dll::symbol_location(const T &)' : expects 1 arguments - 2 provided - template - inline boost::filesystem::path symbol_location(const T& symbol, const char* /*workaround*/ = 0) -#else - //! \overload symbol_location(const T& symbol, boost::system::error_code& ec) - template - inline boost::filesystem::path symbol_location(const T& symbol) -#endif - { - boost::filesystem::path ret; - boost::system::error_code ec; - ret = boost::dll::symbol_location_ptr( - boost::dll::detail::aggressive_ptr_cast(boost::addressof(symbol)), - ec - ); - - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::symbol_location(const T& symbol) failed"); - } - - return ret; - } - - /// @cond - // We have anonymous namespace here to make sure that `this_line_location()` method is instantiated in - // current translation unit and is not shadowed by instantiations from other units. - namespace { - /// @endcond - - /*! - * On success returns full path and name of the binary object that holds the current line of code - * (the line in which the `this_line_location()` method was called). - * - * \param ec Variable that will be set to the result of the operation. - * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. - */ - static inline boost::filesystem::path this_line_location(boost::system::error_code& ec) { - typedef boost::filesystem::path(func_t)(boost::system::error_code& ); - func_t& f = this_line_location; - return boost::dll::symbol_location(f, ec); - } - - //! \overload this_line_location(boost::system::error_code& ec) - static inline boost::filesystem::path this_line_location() { - boost::filesystem::path ret; - boost::system::error_code ec; - ret = this_line_location(ec); - - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::this_line_location() failed"); - } - - return ret; - } - - /// @cond - } // anonymous namespace - /// @endcond - - /*! - * On success returns full path and name of the currently running program (the one which contains the `main()` function). - * - * Return value can be used as a parameter for shared_library. See Tutorial "Linking plugin into the executable" - * for usage example. Flag '-rdynamic' must be used when linking the plugin into the executable - * on Linux OS. - * - * \param ec Variable that will be set to the result of the operation. - * \throws std::bad_alloc in case of insufficient memory. Overload that does not accept boost::system::error_code also throws boost::system::system_error. - */ - inline boost::filesystem::path program_location(boost::system::error_code& ec) { - ec.clear(); - return boost::dll::detail::program_location_impl(ec); - } - - //! \overload program_location(boost::system::error_code& ec) { - inline boost::filesystem::path program_location() { - boost::filesystem::path ret; - boost::system::error_code ec; - ret = boost::dll::detail::program_location_impl(ec); - - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::program_location() failed"); - } - - return ret; - } - -}} // namespace boost::dll - -#endif // BOOST_DLL_RUNTIME_SYMBOL_INFO_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp b/contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp deleted file mode 100644 index 86aed9433ed..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/shared_library.hpp +++ /dev/null @@ -1,550 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015-2016 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_SHARED_LIBRARY_HPP -#define BOOST_DLL_SHARED_LIBRARY_HPP - -/// \file boost/dll/shared_library.hpp -/// \brief Contains the boost::dll::shared_library class, core class for all the -/// DLL/DSO operations. - -#include -#include -#include -#include -#include -#include -#include - -#if BOOST_OS_WINDOWS -# include -#else -# include -#endif - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -namespace boost { namespace dll { - -/*! -* \brief This class can be used to load a -* Dynamic link libraries (DLL's) or Shared Libraries, also know -* as dynamic shared objects (DSO's) and get their exported -* symbols (functions and variables). -* -* shared_library instances share reference count to an actual loaded DLL/DSO, so it -* is safe and memory efficient to have multiple instances of shared_library referencing the same DLL/DSO -* even if those instances were loaded using different paths (relative + absolute) referencing the same object. -* -* On Linux/POSIX link with library "dl". "-fvisibility=hidden" flag is also recommended for use on Linux/POSIX. -*/ -class shared_library -/// @cond - : private boost::dll::detail::shared_library_impl -/// @endcond -{ - typedef boost::dll::detail::shared_library_impl base_t; - BOOST_COPYABLE_AND_MOVABLE(shared_library) - -public: -#ifdef BOOST_DLL_DOXYGEN - typedef platform_specific native_handle_t; -#else - typedef shared_library_impl::native_handle_t native_handle_t; -#endif - - /*! - * Creates in anstance that does not reference any DLL/DSO. - * - * \post this->is_loaded() returns false. - * \throw Nothing. - */ - shared_library() BOOST_NOEXCEPT {} - - /*! - * Copy constructor that increments the reference count of an underlying shared library. - * Same as calling constructor with `lib.location()` parameter. - * - * \param lib A library to copy. - * \post lib == *this - * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. - */ - shared_library(const shared_library& lib) - : base_t() - { - assign(lib); - } - - /*! - * Copy constructor that increments the reference count of an underlying shared library. - * Same as calling constructor with `lib.location(), ec` parameters. - * - * \param lib A shared library to copy. - * \param ec Variable that will be set to the result of the operation. - * \post lib == *this - * \throw std::bad_alloc in case of insufficient memory. - */ - shared_library(const shared_library& lib, boost::system::error_code& ec) - : base_t() - { - assign(lib, ec); - } - - /*! - * Move constructor. Does not invalidate existing symbols and functions loaded from lib. - * - * \param lib A shared library to move from. - * \post lib.is_loaded() returns false, this->is_loaded() return true. - * \throw Nothing. - */ - shared_library(BOOST_RV_REF(shared_library) lib) BOOST_NOEXCEPT - : base_t(boost::move(static_cast(lib))) - {} - - /*! - * Loads a library by specified path with a specified mode. - * - * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, - * const wchar_t* or boost::filesystem::path. - * \param mode A mode that will be used on library load. - * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. - */ - explicit shared_library(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { - shared_library::load(lib_path, mode); - } - - /*! - * Loads a library by specified path with a specified mode. - * - * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, - * const wchar_t* or boost::filesystem::path. - * \param mode A mode that will be used on library load. - * \param ec Variable that will be set to the result of the operation. - * \throw std::bad_alloc in case of insufficient memory. - */ - shared_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { - shared_library::load(lib_path, mode, ec); - } - - //! \overload shared_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) - shared_library(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { - shared_library::load(lib_path, mode, ec); - } - - /*! - * Assignment operator. If this->is_loaded() then calls this->unload(). Does not invalidate existing symbols and functions loaded from lib. - * - * \param lib A shared library to assign from. - * \post lib == *this - * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. - */ - shared_library& operator=(BOOST_COPY_ASSIGN_REF(shared_library) lib) { - boost::system::error_code ec; - assign(lib, ec); - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::shared_library::operator= failed"); - } - - return *this; - } - - /*! - * Move assignment operator. If this->is_loaded() then calls this->unload(). Does not invalidate existing symbols and functions loaded from lib. - * - * \param lib A library to move from. - * \post lib.is_loaded() returns false. - * \throw Nothing. - */ - shared_library& operator=(BOOST_RV_REF(shared_library) lib) BOOST_NOEXCEPT { - if (lib.native() != native()) { - swap(lib); - } - - return *this; - } - - /*! - * Destroys the object by calling `unload()`. If library was loaded multiple times - * by different instances, the actual DLL/DSO won't be unloaded until - * there is at least one instance that references the DLL/DSO. - * - * \throw Nothing. - */ - ~shared_library() BOOST_NOEXCEPT {} - - /*! - * Makes *this share the same shared object as lib. If *this is loaded, then unloads it. - * - * \post lib.location() == this->location(), lib == *this - * \param lib A library to copy. - * \param ec Variable that will be set to the result of the operation. - * \throw std::bad_alloc in case of insufficient memory. - */ - shared_library& assign(const shared_library& lib, boost::system::error_code& ec) { - ec.clear(); - - if (native() == lib.native()) { - return *this; - } - - if (!lib) { - unload(); - return *this; - } - - boost::filesystem::path loc = lib.location(ec); - if (ec) { - return *this; - } - - shared_library copy(loc, ec); - if (ec) { - return *this; - } - - swap(copy); - return *this; - } - - /*! - * Makes *this share the same shared object as lib. If *this is loaded, then unloads it. - * - * \param lib A library instance to assign from. - * \post lib.location() == this->location() - * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. - */ - shared_library& assign(const shared_library& lib) { - boost::system::error_code ec; - assign(lib, ec); - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::shared_library::assign() failed"); - } - - return *this; - } - - /*! - * Loads a library by specified path with a specified mode. - * - * Note that if some library is already loaded in this instance, load will - * call unload() and then load the new provided library. - * - * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, - * const wchar_t* or boost::filesystem::path. - * \param mode A mode that will be used on library load. - * \throw boost::system::system_error, std::bad_alloc in case of insufficient memory. - * - */ - void load(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { - boost::system::error_code ec; - - base_t::load(lib_path, mode, ec); - - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::shared_library::load() failed"); - } - } - - /*! - * Loads a library by specified path with a specified mode. - * - * Note that if some library is already loaded in this instance, load will - * call unload() and then load the new provided library. - * - * \param lib_path Library file name. Can handle std::string, const char*, std::wstring, - * const wchar_t* or boost::filesystem::path. - * \param ec Variable that will be set to the result of the operation. - * \param mode A mode that will be used on library load. - * \throw std::bad_alloc in case of insufficient memory. - */ - void load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { - ec.clear(); - base_t::load(lib_path, mode, ec); - } - - //! \overload void load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) - void load(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { - ec.clear(); - base_t::load(lib_path, mode, ec); - } - - /*! - * Unloads a shared library. If library was loaded multiple times - * by different instances, the actual DLL/DSO won't be unloaded until - * there is at least one instance that references the DLL/DSO. - * - * \post this->is_loaded() returns false. - * \throw Nothing. - */ - void unload() BOOST_NOEXCEPT { - base_t::unload(); - } - - /*! - * Check if an library is loaded. - * - * \return true if a library has been loaded. - * \throw Nothing. - */ - bool is_loaded() const BOOST_NOEXCEPT { - return base_t::is_loaded(); - } - - /*! - * Check if an library is not loaded. - * - * \return true if a library has not been loaded. - * \throw Nothing. - */ - bool operator!() const BOOST_NOEXCEPT { - return !is_loaded(); - } - - /*! - * Check if an library is loaded. - * - * \return true if a library has been loaded. - * \throw Nothing. - */ - BOOST_EXPLICIT_OPERATOR_BOOL() - - /*! - * Search for a given symbol on loaded library. Works for all symbols, including alias names. - * - * \param symbol_name Null-terminated symbol name. Can handle std::string, char*, const char*. - * \return `true` if the loaded library contains a symbol with a given name. - * \throw Nothing. - */ - bool has(const char* symbol_name) const BOOST_NOEXCEPT { - boost::system::error_code ec; - return is_loaded() && !!base_t::symbol_addr(symbol_name, ec) && !ec; - } - - //! \overload bool has(const char* symbol_name) const - bool has(const std::string& symbol_name) const BOOST_NOEXCEPT { - return has(symbol_name.c_str()); - } - - /*! - * Returns reference to the symbol (function or variable) with the given name from the loaded library. - * This call will always succeed and throw nothing if call to `has(const char* )` - * member function with the same symbol name returned `true`. - * - * \b Example: - * \code - * int& i0 = lib.get("integer_name"); - * int& i1 = *lib.get("integer_alias_name"); - * \endcode - * - * \tparam T Type of the symbol that we are going to import. Must be explicitly specified. - * \param symbol_name Null-terminated symbol name. Can handle std::string, char*, const char*. - * \return Reference to the symbol. - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - */ - template - inline typename boost::enable_if_c::value || boost::is_reference::value, T>::type get(const std::string& symbol_name) const { - return get(symbol_name.c_str()); - } - - //! \overload T& get(const std::string& symbol_name) const - template - inline typename boost::disable_if_c::value || boost::is_reference::value, T&>::type get(const std::string& symbol_name) const { - return get(symbol_name.c_str()); - } - - //! \overload T& get(const std::string& symbol_name) const - template - inline typename boost::enable_if_c::value || boost::is_reference::value, T>::type get(const char* symbol_name) const { - return boost::dll::detail::aggressive_ptr_cast( - get_void(symbol_name) - ); - } - - //! \overload T& get(const std::string& symbol_name) const - template - inline typename boost::disable_if_c::value || boost::is_reference::value, T&>::type get(const char* symbol_name) const { - return *boost::dll::detail::aggressive_ptr_cast( - get_void(symbol_name) - ); - } - - /*! - * Returns a symbol (function or variable) from a shared library by alias name of the symbol. - * - * \b Example: - * \code - * int& i = lib.get_alias("integer_alias_name"); - * \endcode - * - * \tparam T Type of the symbol that we are going to import. Must be explicitly specified.. - * \param alias_name Null-terminated alias symbol name. Can handle std::string, char*, const char*. - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - */ - template - inline T& get_alias(const char* alias_name) const { - return *get(alias_name); - } - - //! \overload T& get_alias(const char* alias_name) const - template - inline T& get_alias(const std::string& alias_name) const { - return *get(alias_name.c_str()); - } - -private: - /// @cond - // get_void is required to reduce binary size: it does not depend on a template - // parameter and will be instantiated only once. - void* get_void(const char* sb) const { - boost::system::error_code ec; - - if (!is_loaded()) { - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - // report_error() calls dlsym, do not use it here! - boost::throw_exception( - boost::system::system_error( - ec, "boost::dll::shared_library::get() failed: no library was loaded" - ) - ); - } - - void* const ret = base_t::symbol_addr(sb, ec); - if (ec || !ret) { - boost::dll::detail::report_error(ec, "boost::dll::shared_library::get() failed"); - } - - return ret; - } - /// @endcond - -public: - - /*! - * Returns the native handler of the loaded library. - * - * \return Platform-specific handle. - */ - native_handle_t native() const BOOST_NOEXCEPT { - return base_t::native(); - } - - /*! - * Returns full path and name of this shared object. - * - * \b Example: - * \code - * shared_library lib("test_lib.dll"); - * filesystem::path full_path = lib.location(); // C:\Windows\System32\test_lib.dll - * \endcode - * - * \return Full path to the shared library. - * \throw boost::system::system_error, std::bad_alloc. - */ - boost::filesystem::path location() const { - boost::system::error_code ec; - if (!is_loaded()) { - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - boost::throw_exception( - boost::system::system_error( - ec, "boost::dll::shared_library::location() failed (no library was loaded)" - ) - ); - } - - boost::filesystem::path full_path = base_t::full_module_path(ec); - - if (ec) { - boost::dll::detail::report_error(ec, "boost::dll::shared_library::location() failed"); - } - - return full_path; - } - - /*! - * Returns full path and name of shared module. - * - * \b Example: - * \code - * shared_library lib("test_lib.dll"); - * filesystem::path full_path = lib.location(); // C:\Windows\System32\test_lib.dll - * \endcode - * - * \param ec Variable that will be set to the result of the operation. - * \return Full path to the shared library. - * \throw std::bad_alloc. - */ - boost::filesystem::path location(boost::system::error_code& ec) const { - if (!is_loaded()) { - ec = boost::system::error_code( - boost::system::errc::bad_file_descriptor, - boost::system::generic_category() - ); - - return boost::filesystem::path(); - } - - ec.clear(); - return base_t::full_module_path(ec); - } - - /*! - * Returns suffix of shared module: - * in a call to load() or the constructor/load. - * - * \return The suffix od shared module: ".dll" (Windows), ".so" (Unix/Linux/BSD), ".dylib" (MacOS/IOS) - */ - static boost::filesystem::path suffix() { - return base_t::suffix(); - } - - /*! - * Swaps two libraries. Does not invalidate existing symbols and functions loaded from libraries. - * - * \param rhs Library to swap with. - * \throw Nothing. - */ - void swap(shared_library& rhs) BOOST_NOEXCEPT { - base_t::swap(rhs); - } -}; - - - -/// Very fast equality check that compares the actual DLL/DSO objects. Throws nothing. -inline bool operator==(const shared_library& lhs, const shared_library& rhs) BOOST_NOEXCEPT { - return lhs.native() == rhs.native(); -} - -/// Very fast inequality check that compares the actual DLL/DSO objects. Throws nothing. -inline bool operator!=(const shared_library& lhs, const shared_library& rhs) BOOST_NOEXCEPT { - return lhs.native() != rhs.native(); -} - -/// Compare the actual DLL/DSO objects without any guarantee to be stable between runs. Throws nothing. -inline bool operator<(const shared_library& lhs, const shared_library& rhs) BOOST_NOEXCEPT { - return lhs.native() < rhs.native(); -} - -/// Swaps two shared libraries. Does not invalidate symbols and functions loaded from libraries. Throws nothing. -inline void swap(shared_library& lhs, shared_library& rhs) BOOST_NOEXCEPT { - lhs.swap(rhs); -} - -}} // boost::dll - -#endif // BOOST_DLL_SHARED_LIBRARY_HPP - diff --git a/contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp b/contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp deleted file mode 100644 index 5627e42dfd1..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/shared_library_load_mode.hpp +++ /dev/null @@ -1,249 +0,0 @@ -// Copyright 2014 Renato Tegon Forti, Antony Polukhin. -// Copyright 2015-2016 Antony Polukhin. -// -// 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) - -#ifndef BOOST_DLL_SHARED_LIBRARY_MODE_HPP -#define BOOST_DLL_SHARED_LIBRARY_MODE_HPP - -#include -#include -#include - -#if BOOST_OS_WINDOWS -//#include -#include -#else -# include -#endif - -#ifdef BOOST_HAS_PRAGMA_ONCE -# pragma once -#endif - -/// \file boost/dll/shared_library_load_mode.hpp -/// \brief Contains only the boost::dll::load_mode::type enum and operators related to it. - -namespace boost { namespace dll { namespace load_mode { - -/*! Library load modes. -* -* Each of system family provides own modes. Flags not supported by a particular platform will be silently ignored. -* -* For a detailed description of platform specific options see: -* Windows specific options, -* POSIX specific options. -* -*/ - -enum type { -#ifdef BOOST_DLL_DOXYGEN - /*! - * Default open mode. See the \b Default: comments below to find out the flags that are enabled by default. - */ - default_mode, - - /*! - * \b Platforms: Windows - * - * \b Default: disabled - * - * If this value is used, and the executable module is a DLL, the system does - * not call DllMain for process and thread initialization and termination. - * Also, the system does not load additional executable modules that are - * referenced by the specified module. - * - * Note Do not use this value; it is provided only for backward compatibility. - * If you are planning to access only data or resources in the DLL, use - * LOAD_LIBRARY_AS_DATAFILE_EXCLUSIVE or LOAD_LIBRARY_AS_IMAGE_RESOURCE - * or both. - */ - dont_resolve_dll_references, - - /*! - * \b Platforms: Windows - * - * \b Default: disabled - * - * If this value is used, the system does not check AppLocker rules or - * apply Software Restriction Policies for the DLL. - */ - load_ignore_code_authz_level, - - /*! - * \b Platforms: Windows - * - * \b Default: disabled - * - * If this value is used and lpFileName specifies an absolute path, - * the system uses the alternate file search strategy. - * - * This value cannot be combined with any LOAD_LIBRARY_SEARCH flag. - */ - load_with_altered_search_path, - - /*! - * \b Platforms: POSIX - * - * \b Default: enabled - * - * Relocations shall be performed at an implementation-defined time, ranging - * from the time of the dlopen() call until the first reference to a given - * symbol occurs. - * - * Specifying RTLD_LAZY should improve performance on implementations - * supporting dynamic symbol binding as a process may not reference all of - * the functions in any given object. And, for systems supporting dynamic - * symbol resolution for normal process execution, this behavior mimics - * the normal handling of process execution. - */ - rtld_lazy, - - /*! - * \b Platforms: POSIX - * - * \b Default: disabled - * - * All necessary relocations shall be performed when the object is first - * loaded. This may waste some processing if relocations are performed for - * functions that are never referenced. This behavior may be useful for - * plugins that need to know as soon as an object is loaded that all - * symbols referenced during execution are available. - */ - rtld_now, - - /*! - * \b Platforms: POSIX - * - * \b Default: disabled - * - * The object's symbols shall be made available for the relocation - * processing of any other object. In addition, symbol lookup using - * dlopen(0, mode) and an associated dlsym() allows objects loaded - * with this mode to be searched. - */ - rtld_global, - - /*! - * \b Platforms: POSIX - * - * \b Default: enabled - * - * The object's symbols shall not be made available for the relocation - * processing of any other object. - * - * This is a default Windows behavior that can not be changed. - */ - rtld_local, - - /*! - * \b Platforms: POSIX (requires glibc >= 2.3.4) - * - * \b Default: disabled - * - * The object will use its own symbols in preference to global symbols - * with the same name contained in libraries that have already been loaded. - * This flag is not specified in POSIX.1-2001. - */ - rtld_deepbind, - - /*! - * \b Platforms: Windows, POSIX - * - * \b Default: disabled - * - * Append a platform specific extension and prefix to shared library filename before trying to load it. - * If load attempt fails, try to load with exactly specified name. - * - * \b Example: - * \code - * // Opens `./my_plugins/plugin1.dll` on Windows, `./my_plugins/libplugin1.so` on Linux, `./my_plugins/libplugin1.dylib` on MacOS. - * // If that fails, loads `./my_plugins/plugin1` - * boost::dll::shared_library lib("./my_plugins/plugin1", load_mode::append_decorations); - * \endcode - */ - append_decorations, - /*! - * \b Platforms: Windows, POSIX - * - * \b Default: disabled - * - * Allow loading from system folders if path to library contains no parent path. - */ - search_system_folders -#elif BOOST_OS_WINDOWS - default_mode = 0, - dont_resolve_dll_references = boost::detail::winapi::DONT_RESOLVE_DLL_REFERENCES_, - load_ignore_code_authz_level = boost::detail::winapi::LOAD_IGNORE_CODE_AUTHZ_LEVEL_, - load_with_altered_search_path = boost::detail::winapi::LOAD_WITH_ALTERED_SEARCH_PATH_, - rtld_lazy = 0, - rtld_now = 0, - rtld_global = 0, - rtld_local = 0, - rtld_deepbind = 0, - append_decorations = 0x00800000, - search_system_folders = (append_decorations << 1) -#else - default_mode = 0, - dont_resolve_dll_references = 0, - load_ignore_code_authz_level = 0, - load_with_altered_search_path = 0, - rtld_lazy = RTLD_LAZY, - rtld_now = RTLD_NOW, - rtld_global = RTLD_GLOBAL, - rtld_local = RTLD_LOCAL, - -#if BOOST_LIB_C_GNU < BOOST_VERSION_NUMBER(2,3,4) - rtld_deepbind = 0, -#else - rtld_deepbind = RTLD_DEEPBIND, -#endif - - append_decorations = 0x00800000, - search_system_folders = (append_decorations << 1) -#endif -}; - - -/// Free operators for load_mode::type flag manipulation. -BOOST_CONSTEXPR inline type operator|(type left, type right) BOOST_NOEXCEPT { - return static_cast( - static_cast(left) | static_cast(right) - ); -} -BOOST_CXX14_CONSTEXPR inline type& operator|=(type& left, type right) BOOST_NOEXCEPT { - left = left | right; - return left; -} - -BOOST_CONSTEXPR inline type operator&(type left, type right) BOOST_NOEXCEPT { - return static_cast( - static_cast(left) & static_cast(right) - ); -} -BOOST_CXX14_CONSTEXPR inline type& operator&=(type& left, type right) BOOST_NOEXCEPT { - left = left & right; - return left; -} - -BOOST_CONSTEXPR inline type operator^(type left, type right) BOOST_NOEXCEPT { - return static_cast( - static_cast(left) ^ static_cast(right) - ); -} -BOOST_CXX14_CONSTEXPR inline type& operator^=(type& left, type right) BOOST_NOEXCEPT { - left = left ^ right; - return left; -} - -BOOST_CONSTEXPR inline type operator~(type left) BOOST_NOEXCEPT { - return static_cast( - ~static_cast(left) - ); -} - -}}} // boost::dll::load_mode - -#endif // BOOST_DLL_SHARED_LIBRARY_MODE_HPP diff --git a/contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp b/contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp deleted file mode 100644 index c5859251de7..00000000000 --- a/contrib/libboost/boost_1_65_0/boost/dll/smart_library.hpp +++ /dev/null @@ -1,462 +0,0 @@ -// Copyright 2016 Klemens Morgenstern -// -// 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) - -#ifndef BOOST_DLL_SMART_LIBRARY_HPP_ -#define BOOST_DLL_SMART_LIBRARY_HPP_ - -/// \file boost/dll/smart_library.hpp -/// \warning Extremely experimental! Requires C++14! Will change in next version of Boost! boost/dll/smart_library.hpp is not included in boost/dll.hpp -/// \brief Contains the boost::dll::experimental::smart_library class for loading mangled symbols. - -#if BOOST_COMP_GNUC || BOOST_COMP_CLANG || BOOST_COMP_HPACC || BOOST_COMP_IBM -#include -#elif BOOST_COMP_MSVC -#include -#else -#error "Compiler not supported" -#endif - -#include -#include -#include -#include -#include -#include -#include -#include - - - -namespace boost { -namespace dll { -namespace experimental { - -using boost::dll::detail::constructor; -using boost::dll::detail::destructor; - -/*! -* \brief This class is an extension of \ref shared_library, which allows to load C++ symbols. -* -* This class allows type safe loading of overloaded functions, member-functions, constructors and variables. -* It also allows to overwrite classes so they can be loaded, while being declared with different names. -* -* \warning Is still very experimental. -* -* Currently known limitations: -* -* Member functions must be defined outside of the class to be exported. That is: -* \code -* //not exported: -* struct BOOST_SYMBOL_EXPORT my_class { void func() {}}; -* //exported -* struct BOOST_SYMBOL_EXPORT my_class { void func();}; -* void my_class::func() {}; -* \endcode -* -* With the current analysis, the first version does get exported in MSVC. -* MinGW also does export it, BOOST_SYMBOL_EXPORT is written before it. To allow this on windows one can use -* BOOST_DLL_MEMBER_EXPORT for this, so that MinGW and MSVC can provide those functions. This does however not work with gcc on linux. -* -* Direct initialization of members. -* On linux the following member variable i will not be initialized when using the allocating contructor: -* \code -* struct BOOST_SYMBOL_EXPORT my_class { int i; my_class() : i(42) {} }; -* \endcode -* -* This does however not happen when the value is set inside the constructor function. -*/ -class smart_library { - shared_library _lib; - detail::mangled_storage_impl _storage; - -public: - /*! - * Get the underlying shared_library - */ - const shared_library &shared_lib() const {return _lib;} - - using mangled_storage = detail::mangled_storage_impl; - /*! - * Acces to the mangled storage, which is created on construction. - * - * \throw Nothing. - */ - const mangled_storage &symbol_storage() const {return _storage;} - - ///Overload, for current development. - mangled_storage &symbol_storage() {return _storage;} - - //! \copydoc shared_library::shared_library() - smart_library() BOOST_NOEXCEPT {}; - - //! \copydoc shared_library::shared_library(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) - smart_library(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { - _lib.load(lib_path, mode); - _storage.load(lib_path); - } - - //! \copydoc shared_library::shared_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) - smart_library(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { - load(lib_path, mode, ec); - } - - //! \copydoc shared_library::shared_library(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) - smart_library(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { - load(lib_path, mode, ec); - } - /*! - * copy a smart_library object. - * - * \param lib A smart_library to move from. - * - * \throw Nothing. - */ - smart_library(const smart_library & lib) BOOST_NOEXCEPT - : _lib(lib._lib), _storage(lib._storage) - {} - /*! - * Move a smart_library object. - * - * \param lib A smart_library to move from. - * - * \throw Nothing. - */ - smart_library(BOOST_RV_REF(smart_library) lib) BOOST_NOEXCEPT - : _lib(boost::move(lib._lib)), _storage(boost::move(lib._storage)) - {} - - /*! - * Construct from a shared_library object. - * - * \param lib A shared_library to move from. - * - * \throw Nothing. - */ - explicit smart_library(const shared_library & lib) BOOST_NOEXCEPT - : _lib(lib) - { - _storage.load(lib.location()); - } - /*! - * Construct from a shared_library object. - * - * \param lib A shared_library to move from. - * - * \throw Nothing. - */ - explicit smart_library(BOOST_RV_REF(shared_library) lib) BOOST_NOEXCEPT - : _lib(boost::move(static_cast(lib))) - { - _storage.load(lib.location()); - } - - /*! - * Destroys the smart_library. - * `unload()` is called if the DLL/DSO was loaded. If library was loaded multiple times - * by different instances of shared_library, the actual DLL/DSO won't be unloaded until - * there is at least one instance of shared_library. - * - * \throw Nothing. - */ - ~smart_library() BOOST_NOEXCEPT {}; - - //! \copydoc shared_library::load(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) - void load(const boost::filesystem::path& lib_path, load_mode::type mode = load_mode::default_mode) { - boost::system::error_code ec; - _storage.load(lib_path); - _lib.load(lib_path, mode, ec); - - if (ec) { - boost::dll::detail::report_error(ec, "load() failed"); - } - } - - //! \copydoc shared_library::load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) - void load(const boost::filesystem::path& lib_path, boost::system::error_code& ec, load_mode::type mode = load_mode::default_mode) { - ec.clear(); - _storage.load(lib_path); - _lib.load(lib_path, mode, ec); - } - - //! \copydoc shared_library::load(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) - void load(const boost::filesystem::path& lib_path, load_mode::type mode, boost::system::error_code& ec) { - ec.clear(); - _storage.load(lib_path); - _lib.load(lib_path, mode, ec); - } - - /*! - * Load a variable from the referenced library. - * - * Unlinke shared_library::get this function will also load scoped variables, which also includes static class members. - * - * \note When mangled, MSVC will also check the type. - * - * \param name Name of the variable - * \tparam T Type of the variable - * \return A reference to the variable of type T. - * - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - */ - template - T& get_variable(const std::string &name) const { - return _lib.get(_storage.get_variable(name)); - } - - /*! - * Load a function from the referenced library. - * - * \b Example: - * - * \code - * smart_library lib("test_lib.so"); - * typedef int (&add_ints)(int, int); - * typedef double (&add_doubles)(double, double); - * add_ints f1 = lib.get_function ("func_name"); - * add_doubles f2 = lib.get_function("func_name"); - * \endcode - * - * \note When mangled, MSVC will also check the return type. - * - * \param name Name of the function. - * \tparam Func Type of the function, required for determining the overload - * \return A reference to the function of type F. - * - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - */ - template - Func& get_function(const std::string &name) const { - return _lib.get(_storage.get_function(name)); - } - - /*! - * Load a member-function from the referenced library. - * - * \b Example (import class is MyClass, which is available inside the library and the host): - * - * \code - * smart_library lib("test_lib.so"); - * - * typedef int MyClass(*func)(int); - * typedef int MyClass(*func_const)(int) const; - * - * add_ints f1 = lib.get_mem_fn ("MyClass::function"); - * add_doubles f2 = lib.get_mem_fn("MyClass::function"); - * \endcode - * - * \note When mangled, MSVC will also check the return type. - * - * \param name Name of the function. - * \tparam Class The class the function is a member of. If Class is const, the function will be assumed as taking a const this-pointer. The same applies for volatile. - * \tparam Func Signature of the function, required for determining the overload - * \return A pointer to the member-function with the signature provided - * - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - */ - template - typename boost::dll::detail::get_mem_fn_type::mem_fn get_mem_fn(const std::string& name) const { - return _lib.get::mem_fn>( - _storage.get_mem_fn(name) - ); - } - - /*! - * Load a constructor from the referenced library. - * - * \b Example (import class is MyClass, which is available inside the library and the host): - * - * \code - * smart_library lib("test_lib.so"); - * - * constructor(); - * \endcode - * - * \tparam Signature Signature of the function, required for determining the overload. The return type is the class which this is the constructor of. - * \return A constructor object. - * - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - */ - template - constructor get_constructor() const { - return boost::dll::detail::load_ctor(_lib, _storage.get_constructor()); - } - - /*! - * Load a destructor from the referenced library. - * - * \b Example (import class is MyClass, which is available inside the library and the host): - * - * \code - * smart_library lib("test_lib.so"); - * - * destructor f1 = lib.get_mem_fn(); - * \endcode - * - * \tparam Class The class whichs destructor shall be loaded - * \return A destructor object. - * - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - * - */ - template - destructor get_destructor() const { - return boost::dll::detail::load_dtor(_lib, _storage.get_destructor()); - } - /*! - * Load the typeinfo of the given type. - * - * \b Example (import class is MyClass, which is available inside the library and the host): - * - * \code - * smart_library lib("test_lib.so"); - * - * std::type_info &ti = lib.get_Type_info(); - * \endcode - * - * \tparam Class The class whichs typeinfo shall be loaded - * \return A reference to a type_info object. - * - * \throw boost::system::system_error if symbol does not exist or if the DLL/DSO was not loaded. - * - */ - template - const std::type_info& get_type_info() const - { - return boost::dll::detail::load_type_info(_lib, _storage); - } - /** - * This function can be used to add a type alias. - * - * This is to be used, when a class shall be imported, which is not declared on the host side. - * - * Example: - * \code - * smart_library lib("test_lib.so"); - * - * lib.add_type_alias("MyClass"); //when using MyAlias, the library will look for MyClass - * - * //get the destructor of MyClass - * destructor dtor = lib.get_destructor(); - * \endcode - * - * - * \param name Name of the class the alias is for. - * - * \attention If the alias-type is not large enough for the imported class, it will result in undefined behaviour. - * \warning The alias will only be applied for the type signature, it will not replace the token in the scoped name. - */ - template void add_type_alias(const std::string& name) { - this->_storage.add_alias(name); - } - - //! \copydoc shared_library::unload() - void unload() BOOST_NOEXCEPT { - _storage.clear(); - _lib.unload(); - } - - //! \copydoc shared_library::is_loaded() const - bool is_loaded() const BOOST_NOEXCEPT { - return _lib.is_loaded(); - } - - //! \copydoc shared_library::operator!() const - bool operator!() const BOOST_NOEXCEPT { - return !is_loaded(); - } - - //! \copydoc shared_library::operator bool() const - BOOST_EXPLICIT_OPERATOR_BOOL() - - //! \copydoc shared_library::has(const char* symbol_name) const - bool has(const char* symbol_name) const BOOST_NOEXCEPT { - return _lib.has(symbol_name); - } - - //! \copydoc shared_library::has(const std::string& symbol_name) const - bool has(const std::string& symbol_name) const BOOST_NOEXCEPT { - return _lib.has(symbol_name); - } - - //! \copydoc shared_library::assign(const shared_library& lib) - smart_library& assign(const smart_library& lib) { - _lib.assign(lib._lib); - _storage.assign(lib._storage); - return *this; - } - - //! \copydoc shared_library::swap(shared_library& rhs) - void swap(smart_library& rhs) BOOST_NOEXCEPT { - _lib.swap(rhs._lib); - _storage.swap(rhs._storage); - } -}; - -/// Very fast equality check that compares the actual DLL/DSO objects. Throws nothing. -inline bool operator==(const smart_library& lhs, const smart_library& rhs) BOOST_NOEXCEPT { - return lhs.shared_lib().native() == rhs.shared_lib().native(); -} - -/// Very fast inequality check that compares the actual DLL/DSO objects. Throws nothing. -inline bool operator!=(const smart_library& lhs, const smart_library& rhs) BOOST_NOEXCEPT { - return lhs.shared_lib().native() != rhs.shared_lib().native(); -} - -/// Compare the actual DLL/DSO objects without any guarantee to be stable between runs. Throws nothing. -inline bool operator<(const smart_library& lhs, const smart_library& rhs) BOOST_NOEXCEPT { - return lhs.shared_lib().native() < rhs.shared_lib().native(); -} - -/// Swaps two shared libraries. Does not invalidate symbols and functions loaded from libraries. Throws nothing. -inline void swap(smart_library& lhs, smart_library& rhs) BOOST_NOEXCEPT { - lhs.swap(rhs); -} - - -#ifdef BOOST_DLL_DOXYGEN -/** Helper functions for overloads. - * - * Gets either a variable, function or member-function, depending on the signature. - * - * @code - * smart_library sm("lib.so"); - * get(sm, "space::value"); //import a variable - * get(sm, "space::func"); //import a function - * get(sm, "space::class_::mem_fn"); //import a member function - * @endcode - * - * @param sm A reference to the @ref smart_library - * @param name The name of the entity to import - */ -template -void get(const smart_library& sm, const std::string &name); -#endif - -template -T& get(const smart_library& sm, const std::string &name, typename boost::enable_if,T>::type* = nullptr) - -{ - return sm.get_variable(name); -} - -template -auto get(const smart_library& sm, const std::string &name, typename boost::enable_if>::type* = nullptr) -{ - return sm.get_function(name); -} - -template -auto get(const smart_library& sm, const std::string &name) -> typename detail::get_mem_fn_type::mem_fn -{ - return sm.get_mem_fn(name); -} - - -} /* namespace experimental */ -} /* namespace dll */ -} /* namespace boost */ - -#endif /* BOOST_DLL_SMART_LIBRARY_HPP_ */ diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 9401df9a40f..f99a0e0507a 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -9,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -392,7 +392,7 @@ public: private: CatBoostWrapperAPI api; std::string lib_path; - boost::dll::shared_library lib; + SharedLibrary lib; void initAPI(); From 0d356f9a6a534d4df34af33101880df9fb55a081 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 22:14:25 +0300 Subject: [PATCH 28/32] fix build [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index f99a0e0507a..56e9fd34602 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -397,11 +397,7 @@ private: void initAPI(); template - void load(T& func, const std::string & name) - { - using Type = typename std::remove_pointer::type; - func = lib.get(name); - } + void load(T& func, const std::string & name) { func = lib.get(name); } }; void CatBoostLibHolder::initAPI() From 9c3c3231f45bc14046e9c4d5b49d0d2cd6d4cef3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Oct 2017 23:12:40 +0300 Subject: [PATCH 29/32] added optional features count validation [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 31 +++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 56e9fd34602..6c024a5dfe0 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -52,6 +52,10 @@ struct CatBoostWrapperAPI int (* GetStringCatFeatureHash)(const char * data, size_t size); int (* GetIntegerCatFeatureHash)(long long val); + + size_t (* GetFloatFeaturesCount)(ModelCalcerHandle* calcer); + + size_t (* GetCatFeaturesCount)(ModelCalcerHandle* calcer); }; @@ -94,6 +98,8 @@ public: ColumnPtr evaluate(const ConstColumnPlainPtrs & columns, size_t float_features_count, size_t cat_features_count) const override { + checkFeaturesCount(float_features_count, cat_features_count); + if (columns.empty()) throw Exception("Got empty columns list for CatBoost model.", ErrorCodes::BAD_ARGUMENTS); @@ -142,6 +148,25 @@ public: return evalImpl(columns, float_features_count, cat_features_count, cat_features_are_strings); } + void checkFeaturesCount(size_t float_features_count, size_t cat_features_count) const + { + if (api->GetFloatFeaturesCount) + { + size_t float_features_in_model = api->GetCatFeaturesCount(handle->get()); + if (float_features_count != float_features_in_model) + throw Exception("CatBoost model expected " + std::to_string(float_features_in_model) + " float features" + + ", but " + std::to_string(float_features_count) + " was provided."); + } + + if (api->GetCatFeaturesCount) + { + size_t cat_features_in_model = api->GetCatFeaturesCount(handle->get()); + if (cat_features_count != cat_features_in_model) + throw Exception("CatBoost model expected " + std::to_string(cat_features_in_model) + " cat features" + + ", but " + std::to_string(cat_features_count) + " was provided."); + } + } + private: std::unique_ptr handle; const CatBoostWrapperAPI * api; @@ -398,6 +423,9 @@ private: template void load(T& func, const std::string & name) { func = lib.get(name); } + + template + void tryLoad(T& func, const std::string & name) { func = lib.get(name); } }; void CatBoostLibHolder::initAPI() @@ -411,6 +439,9 @@ void CatBoostLibHolder::initAPI() load(api.CalcModelPredictionWithHashedCatFeatures, "CalcModelPredictionWithHashedCatFeatures"); load(api.GetStringCatFeatureHash, "GetStringCatFeatureHash"); load(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash"); + + tryLoad(api.GetFloatFeaturesCount, "GetFloatFeaturesCount"); + tryLoad(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash"); } std::shared_ptr getCatBoostWrapperHolder(const std::string & lib_path) From fea2bed3da3f42f9e61461bb0bbec1712c3606fe Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Oct 2017 11:39:54 +0300 Subject: [PATCH 30/32] added optional features count validation [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 6c024a5dfe0..eb2e70d98c2 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -425,7 +425,7 @@ private: void load(T& func, const std::string & name) { func = lib.get(name); } template - void tryLoad(T& func, const std::string & name) { func = lib.get(name); } + void tryLoad(T& func, const std::string & name) { func = lib.tryGet(name); } }; void CatBoostLibHolder::initAPI() From 8f84d51a5d560753432169b8355a54e2c7e6ed38 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Oct 2017 15:23:26 +0300 Subject: [PATCH 31/32] added optional features count validation [#CLICKHOUSE-3305] --- dbms/src/Dictionaries/CatBoostModel.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index eb2e70d98c2..9573002b57c 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -152,7 +152,7 @@ public: { if (api->GetFloatFeaturesCount) { - size_t float_features_in_model = api->GetCatFeaturesCount(handle->get()); + size_t float_features_in_model = api->GetFloatFeaturesCount(handle->get()); if (float_features_count != float_features_in_model) throw Exception("CatBoost model expected " + std::to_string(float_features_in_model) + " float features" + ", but " + std::to_string(float_features_count) + " was provided."); @@ -441,7 +441,7 @@ void CatBoostLibHolder::initAPI() load(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash"); tryLoad(api.GetFloatFeaturesCount, "GetFloatFeaturesCount"); - tryLoad(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash"); + tryLoad(api.GetCatFeaturesCount, "GetCatFeaturesCount"); } std::shared_ptr getCatBoostWrapperHolder(const std::string & lib_path) From c1cbfdc2add7131d0b02221ee9bee0a63295660e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Oct 2017 15:41:08 +0300 Subject: [PATCH 32/32] decreased accuracy threshold for catboost python tests [#CLICKHOUSE-3305] --- .../catboost/test_apply_catboost_model/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py b/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py index c38e92636b7..f2fa040a77f 100644 --- a/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py +++ b/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py @@ -74,7 +74,7 @@ def test_apply_float_features_only(): print 'python predictions', pred_python print 'clickhouse predictions', pred_ch - check_predictions(name, test_target, pred_python, pred_ch, 0.95) + check_predictions(name, test_target, pred_python, pred_ch, 0.9) def test_apply_float_features_with_string_cat_features(): @@ -127,7 +127,7 @@ def test_apply_float_features_with_string_cat_features(): print 'python predictions', pred_python print 'clickhouse predictions', pred_ch - check_predictions(name, test_target, pred_python, pred_ch, 0.95) + check_predictions(name, test_target, pred_python, pred_ch, 0.9) def test_apply_float_features_with_int_cat_features(): @@ -180,7 +180,7 @@ def test_apply_float_features_with_int_cat_features(): print 'python predictions', pred_python print 'clickhouse predictions', pred_ch - check_predictions(name, test_target, pred_python, pred_ch, 0.95) + check_predictions(name, test_target, pred_python, pred_ch, 0.9) def test_apply_float_features_with_mixed_cat_features(): @@ -233,4 +233,4 @@ def test_apply_float_features_with_mixed_cat_features(): print 'python predictions', pred_python print 'clickhouse predictions', pred_ch - check_predictions(name, test_target, pred_python, pred_ch, 0.95) + check_predictions(name, test_target, pred_python, pred_ch, 0.9)