diff --git a/dbms/include/DB/AggregateFunctions/ReservoirSamplerDeterministic.h b/dbms/include/DB/AggregateFunctions/ReservoirSamplerDeterministic.h index dd5798b71a6..2aca1df59e0 100644 --- a/dbms/include/DB/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/dbms/include/DB/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index 38305049e58..65cd950c7c2 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/dbms/include/DB/Common/Arena.h b/dbms/include/DB/Common/Arena.h index a1b0211452e..28d40fed635 100644 --- a/dbms/include/DB/Common/Arena.h +++ b/dbms/include/DB/Common/Arena.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Common/AutoArray.h b/dbms/include/DB/Common/AutoArray.h index 3e08409d1b3..9fdfa936230 100644 --- a/dbms/include/DB/Common/AutoArray.h +++ b/dbms/include/DB/Common/AutoArray.h @@ -2,7 +2,7 @@ #include -#include +#include namespace DB diff --git a/dbms/include/DB/Common/Collator.h b/dbms/include/DB/Common/Collator.h index a6bb90e7e51..012694e932a 100644 --- a/dbms/include/DB/Common/Collator.h +++ b/dbms/include/DB/Common/Collator.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Common/FileChecker.h b/dbms/include/DB/Common/FileChecker.h index 47789635133..d76736734dd 100644 --- a/dbms/include/DB/Common/FileChecker.h +++ b/dbms/include/DB/Common/FileChecker.h @@ -1,15 +1,8 @@ #pragma once #include -#include -#include -#include +#include #include -#include -#include -#include -#include -#include namespace DB @@ -23,146 +16,21 @@ private: using Map = std::map; public: - FileChecker(const std::string & file_info_path_) : - files_info_path(file_info_path_) - { - Poco::Path path(files_info_path); - tmp_files_info_path = path.parent().toString() + "tmp_" + path.getFileName(); - } - - void setPath(const std::string & file_info_path_) - { - files_info_path = file_info_path_; - } - using Files = std::vector; - void update(const Poco::File & file) - { - initialize(); - updateImpl(file); - save(); - } - - void update(const Files::iterator & begin, const Files::iterator & end) - { - initialize(); - for (auto it = begin; it != end; ++it) - updateImpl(*it); - save(); - } + FileChecker(const std::string & file_info_path_); + void setPath(const std::string & file_info_path_); + void update(const Poco::File & file); + void update(const Files::const_iterator & begin, const Files::const_iterator & end); /// Проверяем файлы, параметры которых указаны в sizes.json - bool check() const - { - /** Читаем файлы заново при каждом вызове check - чтобы не нарушать константность. - * Метод check вызывается редко. - */ - Map local_map; - load(local_map); - - if (local_map.empty()) - return true; - - for (const auto & name_size : local_map) - { - Poco::File file(Poco::Path(files_info_path).parent().toString() + "/" + name_size.first); - if (!file.exists()) - { - LOG_ERROR(log, "File " << file.path() << " doesn't exist"); - return false; - } - - size_t real_size = file.getSize(); - if (real_size != name_size.second) - { - LOG_ERROR(log, "Size of " << file.path() << " is wrong. Size is " << real_size << " but should be " << name_size.second); - return false; - } - } - - return true; - } + bool check() const; private: - void initialize() - { - if (initialized) - return; - - load(map); - initialized = true; - } - - void updateImpl(const Poco::File & file) - { - map[Poco::Path(file.path()).getFileName()] = file.getSize(); - } - - void save() const - { - { - WriteBufferFromFile out(tmp_files_info_path); - - /// Столь сложная структура JSON-а - для совместимости со старым форматом. - writeCString("{\"yandex\":{", out); - - for (auto it = map.begin(); it != map.end(); ++it) - { - if (it != map.begin()) - writeString(",", out); - - /// escapeForFileName на самом деле не нужен. Но он оставлен для совместимости со старым кодом. - writeJSONString(escapeForFileName(it->first), out); - writeString(":{\"size\":\"", out); - writeIntText(it->second, out); - writeString("\"}", out); - } - - writeCString("}}", out); - out.next(); - } - - Poco::File current_file(files_info_path); - - if (current_file.exists()) - { - std::string old_file_name = files_info_path + ".old"; - current_file.renameTo(old_file_name); - Poco::File(tmp_files_info_path).renameTo(files_info_path); - Poco::File(old_file_name).remove(); - } - else - Poco::File(tmp_files_info_path).renameTo(files_info_path); - } - - void load(Map & map) const - { - map.clear(); - - if (!Poco::File(files_info_path).exists()) - return; - - std::string content; - { - ReadBufferFromFile in(files_info_path); - WriteBufferFromString out(content); - - /// Библиотека JSON не поддерживает пробельные символы. Удаляем их. Неэффективно. - while (!in.eof()) - { - char c; - readChar(c, in); - if (!isspace(c)) - writeChar(c, out); - } - } - JSON json(content); - - JSON files = json["yandex"]; - for (const auto & name_value : files) - map[unescapeForFileName(name_value.getName())] = name_value.getValue()["size"].toUInt(); - } + void initialize(); + void updateImpl(const Poco::File & file); + void save() const; + void load(Map & map) const; std::string files_info_path; std::string tmp_files_info_path; @@ -173,4 +41,5 @@ private: Logger * log = &Logger::get("FileChecker"); }; + } diff --git a/dbms/include/DB/Common/HashTable/HashTable.h b/dbms/include/DB/Common/HashTable/HashTable.h index 4d9edc52ad4..5d3713e95bc 100644 --- a/dbms/include/DB/Common/HashTable/HashTable.h +++ b/dbms/include/DB/Common/HashTable/HashTable.h @@ -9,7 +9,7 @@ #include -#include +#include #include diff --git a/dbms/include/DB/Common/MemoryTracker.h b/dbms/include/DB/Common/MemoryTracker.h index 18132397eb7..966993b27c5 100644 --- a/dbms/include/DB/Common/MemoryTracker.h +++ b/dbms/include/DB/Common/MemoryTracker.h @@ -1,6 +1,6 @@ #pragma once -#include +#include /** Отслеживает потребление памяти. diff --git a/dbms/include/DB/Common/PODArray.h b/dbms/include/DB/Common/PODArray.h index 7f0ef5758da..063f4025086 100644 --- a/dbms/include/DB/Common/PODArray.h +++ b/dbms/include/DB/Common/PODArray.h @@ -8,8 +8,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/dbms/include/DB/Common/SimpleCache.h b/dbms/include/DB/Common/SimpleCache.h index d17dffc5850..6a2bd544e1d 100644 --- a/dbms/include/DB/Common/SimpleCache.h +++ b/dbms/include/DB/Common/SimpleCache.h @@ -21,7 +21,7 @@ template class SimpleCache { private: - using Key = typename function_traits::arguments_remove_reference; + using Key = typename function_traits::arguments_decay; using Result = typename function_traits::result; std::map cache; diff --git a/dbms/include/DB/Core/Types.h b/dbms/include/DB/Core/Types.h index 3ec2c2093f1..fac272c7f86 100644 --- a/dbms/include/DB/Core/Types.h +++ b/dbms/include/DB/Core/Types.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB diff --git a/dbms/include/DB/Core/toField.h b/dbms/include/DB/Core/toField.h index 1f2f36d006c..5e776ca035f 100644 --- a/dbms/include/DB/Core/toField.h +++ b/dbms/include/DB/Core/toField.h @@ -1,6 +1,6 @@ #pragma once -#include /// VisitID_t +#include /// VisitID_t #include diff --git a/dbms/include/DB/DataStreams/AggregatingSortedBlockInputStream.h b/dbms/include/DB/DataStreams/AggregatingSortedBlockInputStream.h index c0b078575ba..4ce623b774a 100644 --- a/dbms/include/DB/DataStreams/AggregatingSortedBlockInputStream.h +++ b/dbms/include/DB/DataStreams/AggregatingSortedBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h b/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h index 6692a586de4..1e692766560 100644 --- a/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h +++ b/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include diff --git a/dbms/include/DB/DataStreams/CollapsingSortedBlockInputStream.h b/dbms/include/DB/DataStreams/CollapsingSortedBlockInputStream.h index fd9eef68152..e19748b4eba 100644 --- a/dbms/include/DB/DataStreams/CollapsingSortedBlockInputStream.h +++ b/dbms/include/DB/DataStreams/CollapsingSortedBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/dbms/include/DB/DataStreams/MergeSortingBlockInputStream.h b/dbms/include/DB/DataStreams/MergeSortingBlockInputStream.h index 1e9a38a8e79..31674d164e2 100644 --- a/dbms/include/DB/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/include/DB/DataStreams/MergeSortingBlockInputStream.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/DataStreams/MergingSortedBlockInputStream.h b/dbms/include/DB/DataStreams/MergingSortedBlockInputStream.h index 35df7cdd891..5ed5ebd73da 100644 --- a/dbms/include/DB/DataStreams/MergingSortedBlockInputStream.h +++ b/dbms/include/DB/DataStreams/MergingSortedBlockInputStream.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/dbms/include/DB/DataStreams/ParallelInputsProcessor.h b/dbms/include/DB/DataStreams/ParallelInputsProcessor.h index 99e11582bea..8ce7bfe8f3d 100644 --- a/dbms/include/DB/DataStreams/ParallelInputsProcessor.h +++ b/dbms/include/DB/DataStreams/ParallelInputsProcessor.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h index c3ba9f87c12..4ec6bfe359f 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h index 39f93e671a6..c438ba4da5e 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h b/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h index 18ad5740fbb..468555b5aed 100644 --- a/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/include/DB/DataStreams/UnionBlockInputStream.h b/dbms/include/DB/DataStreams/UnionBlockInputStream.h index 4c5a440bce7..31d7e274bf6 100644 --- a/dbms/include/DB/DataStreams/UnionBlockInputStream.h +++ b/dbms/include/DB/DataStreams/UnionBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/include/DB/DataTypes/DataTypeFactory.h b/dbms/include/DB/DataTypes/DataTypeFactory.h index 5924370e1a0..16120454bcb 100644 --- a/dbms/include/DB/DataTypes/DataTypeFactory.h +++ b/dbms/include/DB/DataTypes/DataTypeFactory.h @@ -2,7 +2,7 @@ #include -#include +#include #include diff --git a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h index c193b021edb..aeffaffe6ff 100644 --- a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h +++ b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/dbms/include/DB/Functions/FunctionFactory.h b/dbms/include/DB/Functions/FunctionFactory.h index 8b63ac4552e..65cdd132144 100644 --- a/dbms/include/DB/Functions/FunctionFactory.h +++ b/dbms/include/DB/Functions/FunctionFactory.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/dbms/include/DB/Functions/FunctionsMiscellaneous.h b/dbms/include/DB/Functions/FunctionsMiscellaneous.h index 27b1da01066..10c0cc262f5 100644 --- a/dbms/include/DB/Functions/FunctionsMiscellaneous.h +++ b/dbms/include/DB/Functions/FunctionsMiscellaneous.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/ReadBufferFromHTTP.h b/dbms/include/DB/IO/ReadBufferFromHTTP.h index 7ea806452e0..5c9178d12b9 100644 --- a/dbms/include/DB/IO/ReadBufferFromHTTP.h +++ b/dbms/include/DB/IO/ReadBufferFromHTTP.h @@ -9,7 +9,7 @@ #include #include -#include +#include #define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800 #define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1 diff --git a/dbms/include/DB/IO/ReadHelpers.h b/dbms/include/DB/IO/ReadHelpers.h index 12302a33436..ceb7ab310d2 100644 --- a/dbms/include/DB/IO/ReadHelpers.h +++ b/dbms/include/DB/IO/ReadHelpers.h @@ -6,8 +6,8 @@ #include -#include -#include +#include +#include #include #include diff --git a/dbms/include/DB/IO/RemoteWriteBuffer.h b/dbms/include/DB/IO/RemoteWriteBuffer.h index 16fe0b56bf0..8123633dd38 100644 --- a/dbms/include/DB/IO/RemoteWriteBuffer.h +++ b/dbms/include/DB/IO/RemoteWriteBuffer.h @@ -11,7 +11,7 @@ #include #include -#include +#include #define DEFAULT_REMOTE_WRITE_BUFFER_CONNECTION_TIMEOUT 1 #define DEFAULT_REMOTE_WRITE_BUFFER_RECEIVE_TIMEOUT 1800 diff --git a/dbms/include/DB/IO/WriteHelpers.h b/dbms/include/DB/IO/WriteHelpers.h index 1479ae7c92d..28531a0c723 100644 --- a/dbms/include/DB/IO/WriteHelpers.h +++ b/dbms/include/DB/IO/WriteHelpers.h @@ -5,8 +5,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/dbms/include/DB/IO/WriteIntText.h b/dbms/include/DB/IO/WriteIntText.h index 08ff03f35f0..8354a31ff01 100644 --- a/dbms/include/DB/IO/WriteIntText.h +++ b/dbms/include/DB/IO/WriteIntText.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Interpreters/Aggregator.h b/dbms/include/DB/Interpreters/Aggregator.h index a16276c29e1..4c6bbd53c20 100644 --- a/dbms/include/DB/Interpreters/Aggregator.h +++ b/dbms/include/DB/Interpreters/Aggregator.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Interpreters/Compiler.h b/dbms/include/DB/Interpreters/Compiler.h index 8d2dac0ef05..13b74afa418 100644 --- a/dbms/include/DB/Interpreters/Compiler.h +++ b/dbms/include/DB/Interpreters/Compiler.h @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Interpreters/Dictionaries.h b/dbms/include/DB/Interpreters/Dictionaries.h index 17da95d11fd..5cef75de12c 100644 --- a/dbms/include/DB/Interpreters/Dictionaries.h +++ b/dbms/include/DB/Interpreters/Dictionaries.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include #include #include #include diff --git a/dbms/include/DB/Interpreters/ExternalDictionaries.h b/dbms/include/DB/Interpreters/ExternalDictionaries.h index 5a8730326c6..2cef0e615c0 100644 --- a/dbms/include/DB/Interpreters/ExternalDictionaries.h +++ b/dbms/include/DB/Interpreters/ExternalDictionaries.h @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/dbms/include/DB/Interpreters/Join.h b/dbms/include/DB/Interpreters/Join.h index 8d36c22f7ce..552676cfd49 100644 --- a/dbms/include/DB/Interpreters/Join.h +++ b/dbms/include/DB/Interpreters/Join.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/dbms/include/DB/Interpreters/LogicalExpressionsOptimizer.h b/dbms/include/DB/Interpreters/LogicalExpressionsOptimizer.h index 653dc80d5d7..57b20f2c24f 100644 --- a/dbms/include/DB/Interpreters/LogicalExpressionsOptimizer.h +++ b/dbms/include/DB/Interpreters/LogicalExpressionsOptimizer.h @@ -29,7 +29,7 @@ public: /** Заменить все довольно длинные однородные OR-цепочки expr = x1 OR ... OR expr = xN * на выражения expr IN (x1, ..., xN). */ - void optimizeDisjunctiveEqualityChains(); + void perform(); LogicalExpressionsOptimizer(const LogicalExpressionsOptimizer &) = delete; LogicalExpressionsOptimizer & operator=(const LogicalExpressionsOptimizer &) = delete; @@ -83,7 +83,6 @@ private: private: ASTSelectQuery * select_query; const Settings & settings; - bool hasOptimizedDisjunctiveEqualityChains = false; /// Информация про OR-цепочки внутри запроса. DisjunctiveEqualityChainsMap disjunctive_equality_chains_map; /// Количество обработанных OR-цепочек. diff --git a/dbms/include/DB/Interpreters/Quota.h b/dbms/include/DB/Interpreters/Quota.h index ea8e86d8450..dfdfedc7691 100644 --- a/dbms/include/DB/Interpreters/Quota.h +++ b/dbms/include/DB/Interpreters/Quota.h @@ -11,7 +11,7 @@ #include -#include +#include #include #include diff --git a/dbms/include/DB/Interpreters/Set.h b/dbms/include/DB/Interpreters/Set.h index e2dc2405279..bde12d53287 100644 --- a/dbms/include/DB/Interpreters/Set.h +++ b/dbms/include/DB/Interpreters/Set.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/dbms/include/DB/Interpreters/Users.h b/dbms/include/DB/Interpreters/Users.h index 93c5927fcab..04359accba6 100644 --- a/dbms/include/DB/Interpreters/Users.h +++ b/dbms/include/DB/Interpreters/Users.h @@ -21,7 +21,7 @@ #include -#include +#include namespace DB diff --git a/dbms/include/DB/Parsers/IAST.h b/dbms/include/DB/Parsers/IAST.h index 8593676ffbd..161b5fcd2cd 100644 --- a/dbms/include/DB/Parsers/IAST.h +++ b/dbms/include/DB/Parsers/IAST.h @@ -8,7 +8,7 @@ #include -#include +#include #include #include diff --git a/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h index 7b58625bdd9..f09f824a685 100644 --- a/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h +++ b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h @@ -12,7 +12,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Storages/IStorage.h b/dbms/include/DB/Storages/IStorage.h index 1d8ecbbc919..05fddedf760 100644 --- a/dbms/include/DB/Storages/IStorage.h +++ b/dbms/include/DB/Storages/IStorage.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h b/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h index d80556903d2..f43e7148c6f 100644 --- a/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h +++ b/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h b/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h index 9513090ec7f..1ebe0aab3e7 100644 --- a/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h +++ b/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index c0240025939..b7ce477aba3 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index ff69a1eac78..0ac34f34c87 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/dbms/include/DB/Storages/StorageFactory.h b/dbms/include/DB/Storages/StorageFactory.h index 832fa0ead11..33d71fdb32f 100644 --- a/dbms/include/DB/Storages/StorageFactory.h +++ b/dbms/include/DB/Storages/StorageFactory.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 4c5a90b3796..efd8e8012e1 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -19,7 +19,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 4afed87c523..9ab713494a8 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/dbms/src/Client/ParallelReplicas.cpp b/dbms/src/Client/ParallelReplicas.cpp index b70160b889b..50a2c0cb746 100644 --- a/dbms/src/Client/ParallelReplicas.cpp +++ b/dbms/src/Client/ParallelReplicas.cpp @@ -205,8 +205,7 @@ void ParallelReplicas::registerReplica(Connection * connection) if (!res.second) throw Exception("Invalid set of connections.", ErrorCodes::LOGICAL_ERROR); - if (throttler) - connection->setThrottler(throttler); + connection->setThrottler(throttler); } Connection::Packet ParallelReplicas::receivePacketUnlocked() diff --git a/dbms/src/Common/FileChecker.cpp b/dbms/src/Common/FileChecker.cpp new file mode 100644 index 00000000000..48ef272f460 --- /dev/null +++ b/dbms/src/Common/FileChecker.cpp @@ -0,0 +1,153 @@ +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + + +FileChecker::FileChecker(const std::string & file_info_path_) : + files_info_path(file_info_path_) +{ + Poco::Path path(files_info_path); + tmp_files_info_path = path.parent().toString() + "tmp_" + path.getFileName(); +} + +void FileChecker::setPath(const std::string & file_info_path_) +{ + files_info_path = file_info_path_; +} + +void FileChecker::update(const Poco::File & file) +{ + initialize(); + updateImpl(file); + save(); +} + +void FileChecker::update(const Files::const_iterator & begin, const Files::const_iterator & end) +{ + initialize(); + for (auto it = begin; it != end; ++it) + updateImpl(*it); + save(); +} + +bool FileChecker::check() const +{ + /** Читаем файлы заново при каждом вызове check - чтобы не нарушать константность. + * Метод check вызывается редко. + */ + Map local_map; + load(local_map); + + if (local_map.empty()) + return true; + + for (const auto & name_size : local_map) + { + Poco::File file(Poco::Path(files_info_path).parent().toString() + "/" + name_size.first); + if (!file.exists()) + { + LOG_ERROR(log, "File " << file.path() << " doesn't exist"); + return false; + } + + size_t real_size = file.getSize(); + if (real_size != name_size.second) + { + LOG_ERROR(log, "Size of " << file.path() << " is wrong. Size is " << real_size << " but should be " << name_size.second); + return false; + } + } + + return true; +} + +void FileChecker::initialize() +{ + if (initialized) + return; + + load(map); + initialized = true; +} + +void FileChecker::updateImpl(const Poco::File & file) +{ + map[Poco::Path(file.path()).getFileName()] = file.getSize(); +} + +void FileChecker::save() const +{ + { + WriteBufferFromFile out(tmp_files_info_path); + + /// Столь сложная структура JSON-а - для совместимости со старым форматом. + writeCString("{\"yandex\":{", out); + + for (auto it = map.begin(); it != map.end(); ++it) + { + if (it != map.begin()) + writeString(",", out); + + /// escapeForFileName на самом деле не нужен. Но он оставлен для совместимости со старым кодом. + writeJSONString(escapeForFileName(it->first), out); + writeString(":{\"size\":\"", out); + writeIntText(it->second, out); + writeString("\"}", out); + } + + writeCString("}}", out); + out.next(); + } + + Poco::File current_file(files_info_path); + + if (current_file.exists()) + { + std::string old_file_name = files_info_path + ".old"; + current_file.renameTo(old_file_name); + Poco::File(tmp_files_info_path).renameTo(files_info_path); + Poco::File(old_file_name).remove(); + } + else + Poco::File(tmp_files_info_path).renameTo(files_info_path); +} + +void FileChecker::load(Map & map) const +{ + map.clear(); + + if (!Poco::File(files_info_path).exists()) + return; + + std::string content; + { + ReadBufferFromFile in(files_info_path); + WriteBufferFromString out(content); + + /// Библиотека JSON не поддерживает пробельные символы. Удаляем их. Неэффективно. + while (!in.eof()) + { + char c; + readChar(c, in); + if (!isspace(c)) + writeChar(c, out); + } + } + JSON json(content); + + JSON files = json["yandex"]; + for (const auto & name_value : files) + map[unescapeForFileName(name_value.getName())] = name_value.getValue()["size"].toUInt(); +} + +} diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index ac514b4517a..39f17506b62 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/dbms/src/Core/Exception.cpp b/dbms/src/Core/Exception.cpp index 6c2c52baa72..84624039b69 100644 --- a/dbms/src/Core/Exception.cpp +++ b/dbms/src/Core/Exception.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/DataStreams/tests/native_streams.cpp b/dbms/src/DataStreams/tests/native_streams.cpp index 0ca35cc5eed..576291bfb19 100644 --- a/dbms/src/DataStreams/tests/native_streams.cpp +++ b/dbms/src/DataStreams/tests/native_streams.cpp @@ -23,7 +23,7 @@ #include #include -#include +#include int main(int argc, char ** argv) diff --git a/dbms/src/IO/tests/parse_int_perf.cpp b/dbms/src/IO/tests/parse_int_perf.cpp index f3a539f4ac5..2fe16777710 100644 --- a/dbms/src/IO/tests/parse_int_perf.cpp +++ b/dbms/src/IO/tests/parse_int_perf.cpp @@ -3,7 +3,7 @@ #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index 14d404f0d9c..0ba17271c71 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index a4fcb2a262d..c91be012c8b 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index d4099540d9d..310f56c2aef 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -114,7 +114,7 @@ void ExpressionAnalyzer::init() InJoinSubqueriesPreprocessor<>(select_query, context, storage).perform(); /// Оптимизирует логические выражения. - LogicalExpressionsOptimizer(select_query, settings).optimizeDisjunctiveEqualityChains(); + LogicalExpressionsOptimizer(select_query, settings).perform(); /// Добавляет в множество известных алиасов те, которые объявлены в структуре таблицы (ALIAS-столбцы). addStorageAliases(); diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index 125310913cd..b53a25909ce 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -27,9 +27,11 @@ LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select { } -void LogicalExpressionsOptimizer::optimizeDisjunctiveEqualityChains() +void LogicalExpressionsOptimizer::perform() { - if ((select_query == nullptr) || hasOptimizedDisjunctiveEqualityChains) + if (select_query == nullptr) + return; + if (select_query->attributes & IAST::IsVisited) return; collectDisjunctiveEqualityChains(); @@ -50,8 +52,6 @@ void LogicalExpressionsOptimizer::optimizeDisjunctiveEqualityChains() cleanupOrExpressions(); fixBrokenOrExpressions(); } - - hasOptimizedDisjunctiveEqualityChains = true; } void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains() @@ -62,7 +62,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains() using Edge = std::pair; std::deque to_visit; - to_visit.push_back(Edge(nullptr, select_query)); + to_visit.emplace_back(nullptr, select_query); while (!to_visit.empty()) { auto edge = to_visit.back(); @@ -70,7 +70,6 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains() auto to_node = edge.second; to_visit.pop_back(); - to_node->attributes |= IAST::IsVisited; bool found_chain = false; @@ -104,6 +103,8 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains() } } + to_node->attributes |= IAST::IsVisited; + if (found_chain) { if (from_node != nullptr) @@ -196,6 +197,14 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain value_list->children.push_back(operands[1]); } + /// Отсортировать литералы. + std::sort(value_list->children.begin(), value_list->children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs) + { + const auto val_lhs = static_cast(&*lhs); + const auto val_rhs = static_cast(&*rhs); + return val_lhs->value < val_rhs->value; + }); + /// Получить выражение expr из цепочки expr = x1 OR ... OR expr = xN ASTPtr equals_expr_lhs; { @@ -241,7 +250,7 @@ void LogicalExpressionsOptimizer::cleanupOrExpressions() const auto & or_with_expression = chain.first; auto & operands = getFunctionOperands(or_with_expression.or_function); - garbage_map.insert(std::make_pair(or_with_expression.or_function, operands.end())); + garbage_map.emplace(or_with_expression.or_function, operands.end()); } /// Собрать мусор. diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp index 9a253ca96ee..8ede5a3b449 100644 --- a/dbms/src/Interpreters/Quota.cpp +++ b/dbms/src/Interpreters/Quota.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 0c73218551b..4ffaf637249 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -59,6 +59,58 @@ static void executeCreateQuery(const String & query, Context & context, const St } +struct Table +{ + String database_name; + String dir_name; + String file_name; +}; + + +static constexpr size_t MIN_TABLES_TO_PARALLEL_LOAD = 1; +static constexpr size_t PRINT_MESSAGE_EACH_N_TABLES = 256; +static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5; +static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; +static constexpr size_t TABLES_PARALLEL_LOAD_BUNCH_SIZE = 100; + + +static void loadTable(Context & context, const String & path, const Table & table) +{ + Logger * log = &Logger::get("loadTable"); + + const String path_to_metadata = path + "/" + table.dir_name + "/" + table.file_name; + + String s; + { + char in_buf[METADATA_FILE_BUFFER_SIZE]; + ReadBufferFromFile in(path_to_metadata, METADATA_FILE_BUFFER_SIZE, -1, in_buf); + WriteBufferFromString out(s); + copyData(in, out); + } + + /** Пустые файлы с метаданными образуются после грубого перезапуска сервера. + * Удаляем эти файлы, чтобы чуть-чуть уменьшить работу админов по запуску. + */ + if (s.empty()) + { + LOG_ERROR(log, "File " << path_to_metadata << " is empty. Removing."); + Poco::File(path_to_metadata).remove(); + return; + } + + try + { + executeCreateQuery(s, context, table.database_name, path_to_metadata); + } + catch (const Exception & e) + { + throw Exception("Cannot create table from metadata file " + path_to_metadata + ", error: " + e.displayText() + + ", stack trace:\n" + e.getStackTrace().toString(), + ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA); + } +} + + void loadMetadata(Context & context) { Logger * log = &Logger::get("loadMetadata"); @@ -66,16 +118,15 @@ void loadMetadata(Context & context) /// Здесь хранятся определения таблиц String path = context.getPath() + "metadata"; - struct Table - { - String database_name; - String dir_name; - String file_name; - }; - using Tables = std::vector; Tables tables; + /** Часть таблиц должны быть загружены раньше других, так как используются в конструкторе этих других. + * Это таблицы, имя которых начинается на .inner. + * NOTE Это довольно криво. Можно сделать лучше. + */ + Tables tables_to_load_first; + /// Цикл по базам данных Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator it(path); it != dir_end; ++it) @@ -116,26 +167,33 @@ void loadMetadata(Context & context) std::sort(file_names.begin(), file_names.end()); for (const auto & name : file_names) - tables.emplace_back(Table{ - .database_name = database, - .dir_name = it.name(), - .file_name = name}); + { + (0 == name.compare(0, strlen("%2Einner%2E"), "%2Einner%2E") + ? tables_to_load_first + : tables).emplace_back( + Table{ + .database_name = database, + .dir_name = it.name(), + .file_name = name}); + } LOG_INFO(log, "Found " << file_names.size() << " tables."); } + if (!tables_to_load_first.empty()) + { + LOG_INFO(log, "Loading inner tables for materialized views (total " << tables_to_load_first.size() << " tables)."); + + for (const auto & table : tables_to_load_first) + loadTable(context, path, table); + } + size_t total_tables = tables.size(); LOG_INFO(log, "Total " << total_tables << " tables."); StopwatchWithLock watch; size_t tables_processed = 0; - static constexpr size_t MIN_TABLES_TO_PARALLEL_LOAD = 1; - static constexpr size_t PRINT_MESSAGE_EACH_N_TABLES = 256; - static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5; - static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; - static constexpr size_t TABLES_PARALLEL_LOAD_BUNCH_SIZE = 100; - size_t num_threads = std::min(total_tables, SettingMaxThreads().getAutoValue()); std::unique_ptr thread_pool; @@ -151,7 +209,6 @@ void loadMetadata(Context & context) for (Tables::const_iterator it = begin; it != end; ++it) { const Table & table = *it; - const String path_to_metadata = path + "/" + table.dir_name + "/" + table.file_name; /// Сообщения, чтобы было не скучно ждать, когда сервер долго загружается. if (__sync_add_and_fetch(&tables_processed, 1) % PRINT_MESSAGE_EACH_N_TABLES == 0 @@ -161,34 +218,7 @@ void loadMetadata(Context & context) watch.restart(); } - String s; - { - char in_buf[METADATA_FILE_BUFFER_SIZE]; - ReadBufferFromFile in(path_to_metadata, METADATA_FILE_BUFFER_SIZE, -1, in_buf); - WriteBufferFromString out(s); - copyData(in, out); - } - - /** Пустые файлы с метаданными образуются после грубого перезапуска сервера. - * Удаляем эти файлы, чтобы чуть-чуть уменьшить работу админов по запуску. - */ - if (s.empty()) - { - LOG_ERROR(log, "File " << path_to_metadata << " is empty. Removing."); - Poco::File(path_to_metadata).remove(); - continue; - } - - try - { - executeCreateQuery(s, context, table.database_name, path_to_metadata); - } - catch (const Exception & e) - { - throw Exception("Cannot create table from metadata file " + path_to_metadata + ", error: " + e.displayText() + - ", stack trace:\n" + e.getStackTrace().toString(), - ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA); - } + loadTable(context, path, table); } }; diff --git a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp index 20685b96bb9..637732b1c7a 100644 --- a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp +++ b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp @@ -208,7 +208,7 @@ TestResult check(const TestEntry & entry) settings.optimize_min_equality_disjunction_chain_length = entry.limit; DB::LogicalExpressionsOptimizer optimizer(select_query, settings); - optimizer.optimizeDisjunctiveEqualityChains(); + optimizer.perform(); /// Парсить ожидаемый результат. DB::ASTPtr ast_expected; diff --git a/dbms/src/Server/OLAPAttributesMetadata.h b/dbms/src/Server/OLAPAttributesMetadata.h index edf4dc200e1..17463589dfa 100644 --- a/dbms/src/Server/OLAPAttributesMetadata.h +++ b/dbms/src/Server/OLAPAttributesMetadata.h @@ -9,7 +9,7 @@ #include -#include +#include #include #include #include diff --git a/dbms/src/Server/OLAPQueryParser.cpp b/dbms/src/Server/OLAPQueryParser.cpp index 4f356cf5703..b98f7d92e9b 100644 --- a/dbms/src/Server/OLAPQueryParser.cpp +++ b/dbms/src/Server/OLAPQueryParser.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/dbms/src/Server/OLAPQueryParser.h b/dbms/src/Server/OLAPQueryParser.h index 0efb9fd412c..ba2ef30c1f7 100644 --- a/dbms/src/Server/OLAPQueryParser.h +++ b/dbms/src/Server/OLAPQueryParser.h @@ -11,8 +11,8 @@ #include #include -#include -#include +#include +#include namespace DB { diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index ba3a9546903..4c04f24b030 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -5,9 +5,9 @@ #include #include -#include -#include -#include +#include +#include +#include #include #include diff --git a/dbms/src/Server/Server.h b/dbms/src/Server/Server.h index c2847e53464..e3f017d726c 100644 --- a/dbms/src/Server/Server.h +++ b/dbms/src/Server/Server.h @@ -17,7 +17,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index c38a441a377..2fdf88d93da 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -2,7 +2,7 @@ #include -#include +#include #include diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index d23df5eddac..8a9ea98949d 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include diff --git a/libs/libcommon/include/common/ApplicationExt.h b/libs/libcommon/include/common/ApplicationExt.h new file mode 100644 index 00000000000..5cd87b7390f --- /dev/null +++ b/libs/libcommon/include/common/ApplicationExt.h @@ -0,0 +1,40 @@ +/** + * @file + * @author Sergey N. Yatskevich + * @brief YANDEX_APP_MAIN macros + */ +/* + * $Id$ + */ +#ifndef __APPLICATION_EXT_H +#define __APPLICATION_EXT_H + +#include +#include + +#define YANDEX_APP_MAIN(AppClassName) \ + int \ + main (int _argc, char* _argv[]) \ + { \ + AppClassName app; \ + try \ + { \ + app.init (_argc, _argv); \ + return app.run (); \ + } \ + catch (const Poco::Exception& _ex) \ + { \ + app.logger ().log (_ex); \ + } \ + catch (const std::exception& _ex) \ + { \ + app.logger ().error (Poco::Logger::format ("Got exception: $0", _ex.what ())); \ + } \ + catch (...) \ + { \ + app.logger ().error ("Unknown exception"); \ + } \ + return Poco::Util::Application::EXIT_CONFIG; \ + } + +#endif diff --git a/libs/libcommon/include/common/ApplicationServerExt.h b/libs/libcommon/include/common/ApplicationServerExt.h new file mode 100644 index 00000000000..50a19a48f76 --- /dev/null +++ b/libs/libcommon/include/common/ApplicationServerExt.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include + +#define YANDEX_APP_SERVER_MAIN(AppServerClassName) \ + int \ + main (int _argc, char* _argv[]) \ + { \ + AppServerClassName app; \ + try \ + { \ + return app.run (_argc, _argv); \ + } \ + catch (const Poco::Exception& _ex) \ + { \ + std::cerr << "POCO ERROR: " << _ex.displayText() << std::endl; \ + app.logger().log (_ex); \ + } \ + catch (const std::exception& _ex) \ + { \ + std::cerr << "STD ERROR: " << _ex.what() << std::endl; \ + app.logger().error (Poco::Logger::format ("Got exception: $0", _ex.what ())); \ + } \ + catch (...) \ + { \ + std::cerr << "UNKNOWN ERROR" << std::endl; \ + app.logger().error ("Unknown exception"); \ + } \ + return Poco::Util::Application::EXIT_CONFIG; \ + } diff --git a/libs/libcommon/include/common/Common.h b/libs/libcommon/include/common/Common.h new file mode 100644 index 00000000000..7fc037ea653 --- /dev/null +++ b/libs/libcommon/include/common/Common.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include + +#include + +#include + + +typedef Poco::Int8 Int8; +typedef Poco::Int16 Int16; +typedef Poco::Int32 Int32; +typedef Poco::Int64 Int64; + +typedef Poco::UInt8 UInt8; +typedef Poco::UInt16 UInt16; +typedef Poco::UInt32 UInt32; +typedef Poco::UInt64 UInt64; + + +/// Обход проблемы с тем, что KDevelop не видит time_t и size_t (для подсветки синтаксиса). +#ifdef IN_KDEVELOP_PARSER + typedef Int64 time_t; + typedef UInt64 size_t; +#endif + + +/** Тип данных для хранения идентификатора пользователя. */ +typedef UInt64 UserID_t; + +/** Тип данных для хранения идентификатора счетчика. */ +typedef UInt32 CounterID_t; + +/** Идентификатор хита */ +typedef UInt64 WatchID_t; + +/** Идентификатор визита */ +STRONG_TYPEDEF(UInt64, VisitID_t); + +/** Идентификатор клика */ +typedef UInt64 ClickID_t; + +/** Идентификатор цели */ +typedef UInt32 GoalID_t; + + +namespace std +{ + template<> + struct hash : public unary_function + { + size_t operator()(VisitID_t x) const { return x; } + }; +} diff --git a/libs/libcommon/include/common/DateLUT.h b/libs/libcommon/include/common/DateLUT.h new file mode 100644 index 00000000000..24aa467706e --- /dev/null +++ b/libs/libcommon/include/common/DateLUT.h @@ -0,0 +1,77 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include +#include +#include + +/** Этот класс предоставляет метод для того, чтобы создать объект DateLUTImpl + * для заданного часового пояса, если он не существует. + */ +class DateLUT : public Singleton +{ + friend class Singleton; + +public: + DateLUT(const DateLUT &) = delete; + DateLUT & operator=(const DateLUT &) = delete; + + /// Вернуть единственный экземпляр объекта DateLUTImpl для часового пояса по-умолчанию. + static __attribute__((__always_inline__)) const DateLUTImpl & instance() + { + const auto & date_lut = Singleton::instance(); + return *date_lut.default_date_lut_impl; + } + + /// Вернуть единственный экземпляр объекта DateLUTImpl для заданного часового пояса. + static __attribute__((__always_inline__)) const DateLUTImpl & instance(const std::string & time_zone) + { + const auto & date_lut = Singleton::instance(); + return date_lut.get(time_zone); + } + +public: + /// Отображение часового пояса в группу эквивалентных часовый поясов. + /// Два часовых пояса эквивалентные, если они обладают одними и теми же свойствами. + using TimeZoneToGroup = std::unordered_map; + /// Хранилище для lookup таблиц DateLUTImpl. + using DateLUTImplList = std::vector >; + +protected: + DateLUT(); + +private: + __attribute__((__always_inline__)) const DateLUTImpl & get(const std::string & time_zone) const + { + if (time_zone.empty()) + return *default_date_lut_impl; + + auto it = time_zone_to_group.find(time_zone); + if (it == time_zone_to_group.end()) + throw Poco::Exception("Invalid time zone " + time_zone); + + const auto & group_id = it->second; + if (group_id == default_group_id) + return *default_date_lut_impl; + + return getImplementation(time_zone, group_id); + } + + const DateLUTImpl & getImplementation(const std::string & time_zone, size_t group_id) const; + +private: + /// Указатель на реализацию для часового пояса по-умолчанию. + DateLUTImpl * default_date_lut_impl; + /// Соответствующиая группа часовых поясов по-умолчанию. + size_t default_group_id; + /// + TimeZoneToGroup time_zone_to_group; + /// Lookup таблица для каждой группы часовых поясов. + mutable std::unique_ptr date_lut_impl_list; + mutable std::mutex mutex; +}; diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h new file mode 100644 index 00000000000..d4802203194 --- /dev/null +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -0,0 +1,501 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include +#include +#include + +#define DATE_LUT_MIN 0 +#define DATE_LUT_MAX (0x7FFFFFFF - 86400) +#define DATE_LUT_MAX_DAY_NUM (0x7FFFFFFF / 86400) +#define DATE_LUT_MIN_YEAR 1970 +#define DATE_LUT_MAX_YEAR 2037 /// Последний полный год +#define DATE_LUT_YEARS 68 /// Количество лет в lookup таблице + + +STRONG_TYPEDEF(UInt16, DayNum_t); + + +/** Lookup таблица для преобразования времени в дату, а также в месяц или в год или в день недели или в день месяца. + * Сейчас она используется для ускорения OLAPServer-а, который делает такие преобразования миллиардами. + */ +class DateLUTImpl +{ +public: + DateLUTImpl(const std::string & time_zone); + +public: + struct Values + { + /// 32 бита из time_t начала дня. + /// Знаковость важна, чтобы поддержать начало 1970-01-01 MSK, которое имело time_t == -10800. + /// Измените на time_t, если надо поддержать времена после 2038 года. + Int32 date; + + UInt16 year; + UInt8 month; + UInt8 day_of_month; + UInt8 day_of_week; + }; + +private: + /// Сравнительно много данных. То есть, лучше не класть объект на стек. + /// По сравнению с std::vector, на один indirection меньше. + Values lut[DATE_LUT_MAX_DAY_NUM + 1]; + + /// lookup таблица начал годов + DayNum_t years_lut[DATE_LUT_YEARS]; + + /// Смещение от UTC в начале Unix эпохи. + time_t offset_at_start_of_epoch; + + + inline size_t findIndex(time_t t) const + { + /// первое приближение + size_t precision = t / 86400; + if (precision >= DATE_LUT_MAX_DAY_NUM) + return 0; + if (t >= lut[precision].date && t < lut[precision + 1].date) + return precision; + + for (size_t i = 1;; ++i) + { + if (precision + i >= DATE_LUT_MAX_DAY_NUM) + return 0; + if (t >= lut[precision + i].date && t < lut[precision + i + 1].date) + return precision + i; + if (precision < i) + return 0; + if (t >= lut[precision - i].date && t < lut[precision - i + 1].date) + return precision - i; + } + } + + inline const Values & find(time_t t) const + { + return lut[findIndex(t)]; + } + + static inline DayNum_t fixDay(DayNum_t day) + { + return day > DATE_LUT_MAX_DAY_NUM ? static_cast(0) : day; + } + +public: + /// всё ниже thread-safe; корректность входных данных не проверяется + + inline time_t toDate(time_t t) const { return find(t).date; } + inline unsigned toMonth(time_t t) const { return find(t).month; } + inline unsigned toYear(time_t t) const { return find(t).year; } + inline unsigned toDayOfWeek(time_t t) const { return find(t).day_of_week; } + inline unsigned toDayOfMonth(time_t t) const { return find(t).day_of_month; } + + /// номер недели, начиная с какой-то недели в прошлом; неделя начинается с понедельника + /// (переводим к понедельнику и делим DayNum на 7; будем исходить из допущения, + /// что в области применения этой функции не было и не будет недель, состоящих не из семи дней) + inline unsigned toRelativeWeekNum(DayNum_t d) const + { + return (d - (lut[d].day_of_week - 1)) / 7; + } + + inline unsigned toRelativeWeekNum(time_t t) const + { + size_t index = findIndex(t); + return (index - (lut[index].day_of_week - 1)) / 7; + } + + /// номер месяца, начиная с какого-то месяца в прошлом (год * 12 + номер месяца в году) + inline unsigned toRelativeMonthNum(DayNum_t d) const + { + return lut[d].year * 12 + lut[d].month; + } + + inline unsigned toRelativeMonthNum(time_t t) const + { + size_t index = findIndex(t); + return lut[index].year * 12 + lut[index].month; + } + + /// делим unix timestamp на 3600; + /// (таким образом, учитываются прошедшие интервалы времени длительностью в час, не зависимо от перевода стрелок; + /// поддерживаются только часовые пояса, в которых перевод стрелок осуществлялся только на целое число часов) + inline time_t toRelativeHourNum(time_t t) const + { + return t / 3600; + } + + /// делим unix timestamp на 60 + inline time_t toRelativeMinuteNum(time_t t) const + { + return t / 60; + } + + /// округление вниз до понедельника + inline time_t toFirstDayOfWeek(time_t t) const + { + size_t index = findIndex(t); + return lut[index - (lut[index].day_of_week - 1)].date; + } + + inline DayNum_t toFirstDayNumOfWeek(DayNum_t d) const + { + return DayNum_t(d - (lut[d].day_of_week - 1)); + } + + inline DayNum_t toFirstDayNumOfWeek(time_t t) const + { + size_t index = findIndex(t); + return DayNum_t(index - (lut[index].day_of_week - 1)); + } + + /// округление вниз до первого числа месяца + inline time_t toFirstDayOfMonth(time_t t) const + { + size_t index = findIndex(t); + return lut[index - (lut[index].day_of_month - 1)].date; + } + + inline DayNum_t toFirstDayNumOfMonth(DayNum_t d) const + { + return DayNum_t(d - (lut[fixDay(d)].day_of_month - 1)); + } + + inline DayNum_t toFirstDayNumOfMonth(time_t t) const + { + size_t index = findIndex(t); + return DayNum_t(index - (lut[index].day_of_month - 1)); + } + + /// округление до первого числа квартала + inline time_t toFirstDayOfQuarter(time_t t) const + { + size_t index = findIndex(t); + switch (lut[index].month % 3) + { + case 0: + index = index - lut[index].day_of_month; + case 2: + index = index - lut[index].day_of_month; + case 1: + index = index - lut[index].day_of_month + 1; + } + return DayNum_t(index); + } + + inline DayNum_t toFirstDayNumOfQuarter(DayNum_t d) const + { + size_t index = fixDay(d); + switch (lut[index].month % 3) + { + case 0: + index = index - lut[index].day_of_month; + case 2: + index = index - lut[index].day_of_month; + case 1: + index = index - lut[index].day_of_month + 1; + } + return DayNum_t(index); + } + + inline DayNum_t toFirstDayNumOfQuarter(time_t t) const + { + size_t index = findIndex(t); + switch (lut[index].month % 3) + { + case 0: + index = index - lut[index].day_of_month; + case 2: + index = index - lut[index].day_of_month; + case 1: + index = index - lut[index].day_of_month + 1; + } + return DayNum_t(index); + } + + /// округление вниз до первого числа года + inline time_t toFirstDayOfYear(time_t t) const + { + return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date; + } + + inline DayNum_t toFirstDayNumOfYear(DayNum_t d) const + { + return years_lut[lut[fixDay(d)].year - DATE_LUT_MIN_YEAR]; + } + + inline time_t toFirstDayNumOfYear(time_t t) const + { + return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date; + } + + /// первое число следующего месяца + inline time_t toFirstDayOfNextMonth(time_t t) const + { + size_t index = findIndex(t); + index += 32 - lut[index].day_of_month; + return lut[index - (lut[index].day_of_month - 1)].date; + } + + /// первое число предыдущего месяца + inline time_t toFirstDayOfPrevMonth(time_t t) const + { + size_t index = findIndex(t); + index -= lut[index].day_of_month; + return lut[index - (lut[index].day_of_month - 1)].date; + } + + /// количество дней в месяце + inline size_t daysInMonth(time_t t) const + { + size_t today = findIndex(t); + size_t start_of_month = today - (lut[today].day_of_month - 1); + size_t next_month = start_of_month + 31; + size_t start_of_next_month = next_month - (lut[next_month].day_of_month - 1); + return start_of_next_month - start_of_month; + } + + /** Округление до даты; затем сдвиг на указанное количество дней. + * Замечание: результат сдвига должен находиться в пределах LUT. + */ + inline time_t toDateAndShift(time_t t, int days = 1) const + { + return lut[findIndex(t) + days].date; + } + + /** функции ниже исходят из допущения, что перевод стрелок вперёд, если осуществляется, то на час, в два часа ночи, + * а перевод стрелок назад, если осуществляется, то на час, в три часа ночи. + * (что, в общем, не верно, так как в Москве один раз перевод стрелок был осуществлён в другое время) + */ + + inline time_t toTimeInaccurate(time_t t) const + { + size_t index = findIndex(t); + time_t day_length = lut[index + 1].date - lut[index].date; + + time_t res = t - lut[index].date; + + if (unlikely(day_length == 90000 && res >= 10800)) /// был произведён перевод стрелок назад + res -= 3600; + else if (unlikely(day_length == 82800 && res >= 7200)) /// был произведён перевод стрелок вперёд + res += 3600; + + return res - offset_at_start_of_epoch; /// Отсчёт от 1970-01-01 00:00:00 по локальному времени + } + + inline unsigned toHourInaccurate(time_t t) const + { + size_t index = findIndex(t); + time_t day_length = lut[index + 1].date - lut[index].date; + unsigned res = (t - lut[index].date) / 3600; + + if (unlikely(day_length == 90000 && res >= 3)) /// был произведён перевод стрелок назад + --res; + else if (unlikely(day_length == 82800 && res >= 2)) /// был произведён перевод стрелок вперёд + ++res; + + return res; + } + + inline unsigned toMinute(time_t t) const { return ((t - find(t).date) % 3600) / 60; } + inline unsigned toSecond(time_t t) const { return (t - find(t).date) % 60; } + + inline unsigned toStartOfMinute(time_t t) const + { + time_t date = find(t).date; + return date + (t - date) / 60 * 60; + } + + inline unsigned toStartOfHour(time_t t) const + { + time_t date = find(t).date; + return date + (t - date) / 3600 * 3600; + } + + /** Только для часовых поясов, отличающихся от UTC на значение, кратное часу и без перевода стрелок не значение не кратное часу */ + + inline unsigned toMinuteInaccurate(time_t t) const { return (t / 60) % 60; } + inline unsigned toSecondInaccurate(time_t t) const { return t % 60; } + + inline unsigned toStartOfMinuteInaccurate(time_t t) const { return t / 60 * 60; } + inline unsigned toStartOfFiveMinuteInaccurate(time_t t) const { return t / 300 * 300; } + inline unsigned toStartOfHourInaccurate(time_t t) const { return t / 3600 * 3600; } + + /// Номер дня в пределах UNIX эпохи (и немного больше) - позволяет хранить дату в двух байтах + + inline DayNum_t toDayNum(time_t t) const { return static_cast(findIndex(t)); } + inline time_t fromDayNum(DayNum_t d) const { return lut[fixDay(d)].date; } + + inline time_t toDate(DayNum_t d) const { return lut[fixDay(d)].date; } + inline unsigned toMonth(DayNum_t d) const { return lut[fixDay(d)].month; } + inline unsigned toYear(DayNum_t d) const { return lut[fixDay(d)].year; } + inline unsigned toDayOfWeek(DayNum_t d) const { return lut[fixDay(d)].day_of_week; } + inline unsigned toDayOfMonth(DayNum_t d) const { return lut[fixDay(d)].day_of_month; } + + inline const Values & getValues(DayNum_t d) const { return lut[fixDay(d)]; } + inline const Values & getValues(time_t t) const { return lut[findIndex(t)]; } + + /// получает DayNum_t из года, месяца, дня + inline DayNum_t makeDayNum(short year, char month, char day_of_month) const + { + if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) + return DayNum_t(0); + DayNum_t any_day_of_month(years_lut[year - DATE_LUT_MIN_YEAR] + 31 * (month - 1)); + return DayNum_t(any_day_of_month - toDayOfMonth(any_day_of_month) + day_of_month); + } + + inline time_t makeDate(short year, char month, char day_of_month) const + { + return lut[makeDayNum(year, month, day_of_month)].date; + } + + /** Функция ниже исходит из допущения, что перевод стрелок вперёд, если осуществляется, то на час, в два часа ночи, + * а перевод стрелок назад, если осуществляется, то на час, в три часа ночи. + * (что, в общем, не верно, так как в Москве один раз перевод стрелок был осуществлён в другое время). + * Также, выдаётся лишь один из двух возможных вариантов при переводе стрелок назад. + */ + inline time_t makeDateTime(short year, char month, char day_of_month, char hour, char minute, char second) const + { + size_t index = makeDayNum(year, month, day_of_month); + time_t res = lut[index].date + hour * 3600 + minute * 60 + second; + time_t day_length = lut[index + 1].date - lut[index].date; + + if (unlikely(day_length == 90000 && hour >= 3)) /// был произведён перевод стрелок назад + res += 3600; + else if (unlikely(day_length == 82800 && hour >= 2)) /// был произведён перевод стрелок вперёд + res -= 3600; + + return res; + } + + + inline UInt32 toNumYYYYMMDD(time_t t) const + { + const Values & values = find(t); + return values.year * 10000 + values.month * 100 + values.day_of_month; + } + + inline UInt32 toNumYYYYMMDD(DayNum_t d) const + { + const Values & values = lut[fixDay(d)]; + return values.year * 10000 + values.month * 100 + values.day_of_month; + } + + inline time_t YYYYMMDDToDate(UInt32 num) const + { + return makeDate(num / 10000, num / 100 % 100, num % 100); + } + + inline DayNum_t YYYYMMDDToDayNum(UInt32 num) const + { + return makeDayNum(num / 10000, num / 100 % 100, num % 100); + } + + + inline UInt64 toNumYYYYMMDDhhmmss(time_t t) const + { + const Values & values = find(t); + return + toSecondInaccurate(t) + + toMinuteInaccurate(t) * 100 + + toHourInaccurate(t) * 10000 + + UInt64(values.day_of_month) * 1000000 + + UInt64(values.month) * 100000000 + + UInt64(values.year) * 10000000000; + } + + inline time_t YYYYMMDDhhmmssToTime(UInt64 num) const + { + return makeDateTime( + num / 10000000000, + num / 100000000 % 100, + num / 1000000 % 100, + num / 10000 % 100, + num / 100 % 100, + num % 100); + } + + + inline std::string timeToString(time_t t) const + { + const Values & values = find(t); + + std::string s {"0000-00-00 00:00:00"}; + + s[0] += values.year / 1000; + s[1] += (values.year / 100) % 10; + s[2] += (values.year / 10) % 10; + s[3] += values.year % 10; + s[5] += values.month / 10; + s[6] += values.month % 10; + s[8] += values.day_of_month / 10; + s[9] += values.day_of_month % 10; + + auto hour = toHourInaccurate(t); + auto minute = toMinuteInaccurate(t); + auto second = toSecondInaccurate(t); + + s[11] += hour / 10; + s[12] += hour % 10; + s[14] += minute / 10; + s[15] += minute % 10; + s[17] += second / 10; + s[18] += second % 10; + + return s; + } + + inline std::string dateToString(time_t t) const + { + const Values & values = find(t); + + std::string s {"0000-00-00"}; + + s[0] += values.year / 1000; + s[1] += (values.year / 100) % 10; + s[2] += (values.year / 10) % 10; + s[3] += values.year % 10; + s[5] += values.month / 10; + s[6] += values.month % 10; + s[8] += values.day_of_month / 10; + s[9] += values.day_of_month % 10; + + return s; + } + + inline std::string dateToString(DayNum_t d) const + { + const Values & values = lut[fixDay(d)]; + + std::string s {"0000-00-00"}; + + s[0] += values.year / 1000; + s[1] += (values.year / 100) % 10; + s[2] += (values.year / 10) % 10; + s[3] += values.year % 10; + s[5] += values.month / 10; + s[6] += values.month % 10; + s[8] += values.day_of_month / 10; + s[9] += values.day_of_month % 10; + + return s; + } +}; + + +namespace std +{ + template<> + struct hash + { + size_t operator() (DayNum_t x) const + { + return x; + } + }; +} diff --git a/libs/libcommon/include/common/ErrorHandlers.h b/libs/libcommon/include/common/ErrorHandlers.h new file mode 100644 index 00000000000..31271e75804 --- /dev/null +++ b/libs/libcommon/include/common/ErrorHandlers.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include + + +/** ErrorHandler для потоков, который в случае неперехваченного исключения, + * выводит ошибку в лог и завершает демон. + */ +class KillingErrorHandler : public Poco::ErrorHandler +{ +public: + void exception(const Poco::Exception & e) { std::terminate(); } + void exception(const std::exception & e) { std::terminate(); } + void exception() { std::terminate(); } +}; + + +/** То же самое, но не завершает работу в случае эксепшена типа Socket is not connected. + * Этот эксепшен возникает внутри реализаций Poco::Net::HTTPServer, Poco::Net::TCPServer, + * и иначе его не удаётся перехватить, и сервер завершает работу. + */ +class ServerErrorHandler : public KillingErrorHandler +{ +public: + void exception(const Poco::Exception & e) + { + if (e.code() == POCO_ENOTCONN) + LOG_WARNING(log, "Client has gone away."); + else + std::terminate(); + } + +private: + Logger * log = &Logger::get("ServerErrorHandler"); +}; diff --git a/libs/libcommon/include/common/JSON.h b/libs/libcommon/include/common/JSON.h new file mode 100644 index 00000000000..c282e4e264b --- /dev/null +++ b/libs/libcommon/include/common/JSON.h @@ -0,0 +1,197 @@ +#pragma once + +#include +#include +#include +#include + +#define PURE __attribute__((pure)) + + +/** Очень простой класс для чтения JSON (или его кусочков). + * Представляет собой ссылку на кусок памяти, в котором содержится JSON (или его кусочек). + * Не создаёт никаких структур данных в оперативке. Не выделяет память (кроме std::string). + * Не парсит JSON до конца (парсит только часть, необходимую для выполнения вызванного метода). + * Парсинг необходимой части запускается каждый раз при вызове методов. + * Может работать с обрезанным JSON-ом. + * При этом, (в отличие от SAX-подобных парсеров), предоставляет удобные методы для работы. + * + * Эта структура данных более оптимальна, если нужно доставать несколько элементов из большого количества маленьких JSON-ов. + * То есть, подходит для обработки "параметров визитов" и "параметров интернет магазинов" в Яндекс.Метрике. + * Если нужно много работать с одним большим JSON-ом, то этот класс может быть менее оптимальным. + * + * Имеются следующие соглашения: + * 1. Предполагается, что в JSON-е нет пробельных символов. + * 2. Предполагается, что строки в JSON в кодировке UTF-8; также могут использоваться \u-последовательности. + * Строки возвращаются в кодировке UTF-8, \u-последовательности переводятся в UTF-8. + * 3. Но суррогатная пара из двух \uXXXX\uYYYY переводится не в UTF-8, а в CESU-8. + * 4. Корректный JSON парсится корректно. + * При работе с некорректным JSON-ом, кидается исключение или возвращаются неверные результаты. + * (пример: считается, что если встретился символ 'n', то после него идёт 'ull' (null); + * если после него идёт ',1,', то исключение не кидается, и, таким образом, возвращается неверный результат) + * 5. Глубина вложенности JSON ограничена (см. MAX_JSON_DEPTH в cpp файле). + * При необходимости спуститься на большую глубину, кидается исключение. + * 6. В отличие от JSON, пользоволяет парсить значения вида 64-битное число, со знаком, или без. + * При этом, если число дробное - то дробная часть тихо отбрасывается. + * 7. Числа с плавающей запятой парсятся не с максимальной точностью. + * + * Подходит только для чтения JSON, модификация не предусмотрена. + * Все методы immutable, кроме operator++. + */ + + +POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception); + + +class JSON +{ +private: + typedef const char * Pos; + Pos ptr_begin; + Pos ptr_end; + unsigned level; + +public: + JSON(Pos ptr_begin_, Pos ptr_end_, unsigned level_ = 0) : ptr_begin(ptr_begin_), ptr_end(ptr_end_), level(level_) + { + checkInit(); + } + + JSON(const std::string & s) : ptr_begin(s.data()), ptr_end(s.data() + s.size()), level(0) + { + checkInit(); + } + + JSON(const JSON & rhs) : ptr_begin(rhs.ptr_begin), ptr_end(rhs.ptr_end), level(rhs.level) {} + + /// Для вставки в контейнеры (создаёт некорректный объект) + JSON() : ptr_begin(nullptr), ptr_end(ptr_begin + 1) {} + + const char * data() const PURE { return ptr_begin; } + const char * dataEnd() const PURE { return ptr_end; } + + enum ElementType + { + TYPE_OBJECT, + TYPE_ARRAY, + TYPE_NUMBER, + TYPE_STRING, + TYPE_BOOL, + TYPE_NULL, + TYPE_NAME_VALUE_PAIR, + TYPE_NOTYPE, + }; + + ElementType getType() const PURE; + + bool isObject() const PURE { return getType() == TYPE_OBJECT; }; + bool isArray() const PURE { return getType() == TYPE_ARRAY; }; + bool isNumber() const PURE { return getType() == TYPE_NUMBER; }; + bool isString() const PURE { return getType() == TYPE_STRING; }; + bool isBool() const PURE { return getType() == TYPE_BOOL; }; + bool isNull() const PURE { return getType() == TYPE_NULL; }; + bool isNameValuePair() const PURE { return getType() == TYPE_NAME_VALUE_PAIR; }; + + /// Количество элементов в массиве или объекте; если элемент - не массив или объект, то исключение. + size_t size() const PURE; + + /// Является ли массив или объект пустыми; если элемент - не массив или объект, то исключение. + bool empty() const PURE; + + /// Получить элемент массива по индексу; если элемент - не массив, то исключение. + JSON operator[] (size_t n) const PURE; + + /// Получить элемент объекта по имени; если элемент - не объект, то исключение. + JSON operator[] (const std::string & name) const PURE; + + /// Есть ли в объекте элемент с заданным именем; если элемент - не объект, то исключение. + bool has(const std::string & name) const PURE { return has(name.data(), name.size()); } + bool has(const char * data, size_t size) const PURE; + + /// Получить значение элемента; исключение, если элемент имеет неправильный тип. + template + T get() const PURE; + + template + T getWithDefault(const std::string & key, const T & default_ = T()) const PURE; + + double getDouble() const PURE; + Int64 getInt() const PURE; /// Отбросить дробную часть. + UInt64 getUInt() const PURE; /// Отбросить дробную часть. Если число отрицательное - исключение. + std::string getString() const PURE; + bool getBool() const PURE; + std::string getName() const PURE; /// Получить имя name-value пары. + JSON getValue() const PURE; /// Получить значение name-value пары. + + StringRef getRawString() const PURE; + StringRef getRawName() const PURE; + + /// Получить значение элемента; если элемент - строка, то распарсить значение из строки; если не строка или число - то исключение. + double toDouble() const PURE; + Int64 toInt() const PURE; + UInt64 toUInt() const PURE; + + /** Преобразовать любой элемент в строку. + * Для строки возвращается её значение, для всех остальных элементов - сериализованное представление. + */ + std::string toString() const PURE; + + /// Класс JSON одновременно является итератором по самому себе. + typedef JSON iterator; + typedef JSON const_iterator; + + iterator operator* () const PURE { return *this; } + const JSON * operator-> () const PURE { return this; } + bool operator== (const JSON & rhs) const PURE { return ptr_begin == rhs.ptr_begin; } + bool operator!= (const JSON & rhs) const PURE { return ptr_begin != rhs.ptr_begin; } + + /** Если элемент - массив или объект, то begin() возвращает iterator, + * который указывает на первый элемент массива или первую name-value пару объекта. + */ + iterator begin() const PURE; + + /** end() - значение, которое нельзя использовать; сигнализирует о том, что элементы закончились. + */ + iterator end() const PURE; + + /// Перейти к следующему элементу массива или следующей name-value паре объекта. + iterator & operator++(); + iterator operator++(int); + + /// Есть ли в строке escape-последовательности + bool hasEscapes() const PURE; + + /// Есть ли в строке спец-символы из набора \, ', \0, \b, \f, \r, \n, \t, возможно, заэскейпленные. + bool hasSpecialChars() const PURE; + +private: + /// Проверить глубину рекурсии, а также корректность диапазона памяти. + void checkInit() const PURE; + /// Проверить, что pos лежит внутри диапазона памяти. + void checkPos(Pos pos) const PURE; + + /// Вернуть позицию после заданного элемента. + Pos skipString() const PURE; + Pos skipNumber() const PURE; + Pos skipBool() const PURE; + Pos skipNull() const PURE; + Pos skipNameValuePair() const PURE; + Pos skipObject() const PURE; + Pos skipArray() const PURE; + + Pos skipElement() const PURE; + + /// Найти name-value пару с заданным именем в объекте. + Pos searchField(const std::string & name) const PURE { return searchField(name.data(), name.size()); } + Pos searchField(const char * data, size_t size) const PURE; +}; + +template +T JSON::getWithDefault(const std::string & key, const T & default_) const +{ + if (has(key)) + return (*this)[key].get(); + else + return default_; +} +#undef PURE diff --git a/libs/libcommon/include/common/MultiVersion.h b/libs/libcommon/include/common/MultiVersion.h new file mode 100644 index 00000000000..095eb1c18b5 --- /dev/null +++ b/libs/libcommon/include/common/MultiVersion.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include + + +/** Позволяет хранить некоторый объект, использовать его read-only в разных потоках, + * и заменять его на другой в других потоках. + * Замена производится атомарно, при этом, читающие потоки могут работать с разными версиями объекта. + * + * Использование: + * MultiVersion x; + * - при обновлении данных: + * x.set(new value); + * - при использовании данных для чтения в разных потоках: + * { + * MultiVersion::Version current_version = x.get(); + * // используем для чего-нибудь *current_version + * } // здесь перестаём владеть версией; если версия устарела, и её никто больше не использует - она будет уничтожена + * + * Все методы thread-safe. + */ +template > +class MultiVersion +{ +public: + /// Конкретная версия объекта для использования. SharedPtr определяет время жизни версии. + typedef Ptr Version; + + /// Инициализация по-умолчанию (NULL-ом). + MultiVersion() = default; + + /// Инициализация первой версией. + MultiVersion(const Version & value) + { + set(value); + } + + MultiVersion(Version && value) + { + set(std::move(value)); + } + + /// Получить текущую версию для использования. Возвращает SharedPtr, который определяет время жизни версии. + const Version get() const + { + /// TODO: можно ли заменять SharedPtr lock-free? (Можно, если сделать свою реализацию с использованием cmpxchg16b.) + Poco::ScopedLock lock(mutex); + return current_version; + } + + /// Обновить объект новой версией. + void set(Version value) + { + Poco::ScopedLock lock(mutex); + current_version = value; + } + +private: + Version current_version; + mutable Poco::FastMutex mutex; +}; diff --git a/libs/libcommon/include/common/Revision.h b/libs/libcommon/include/common/Revision.h new file mode 100644 index 00000000000..d20013912fb --- /dev/null +++ b/libs/libcommon/include/common/Revision.h @@ -0,0 +1,6 @@ +#pragma once + +namespace Revision +{ + unsigned get(); +} diff --git a/libs/libcommon/include/common/likely.h b/libs/libcommon/include/common/likely.h new file mode 100644 index 00000000000..7c3fa066fee --- /dev/null +++ b/libs/libcommon/include/common/likely.h @@ -0,0 +1,4 @@ +#pragma once + +#define likely(x) (__builtin_expect(!!(x), 1)) +#define unlikely(x) (__builtin_expect(!!(x), 0)) diff --git a/libs/libcommon/include/common/logger_useful.h b/libs/libcommon/include/common/logger_useful.h new file mode 100644 index 00000000000..79c0b8ac818 --- /dev/null +++ b/libs/libcommon/include/common/logger_useful.h @@ -0,0 +1,61 @@ +#pragma once +/// Вспомогательные определения облегчающие работу с PoCo logging. + +#include +#include + +#ifndef QUERY_PREVIEW_LENGTH +#define QUERY_PREVIEW_LENGTH 160 +#endif + +using Poco::Logger; + +/// Logs a message to a specified logger with that level. + +#define LOG_TRACE(logger, message) do { \ + if ((logger)->trace()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->trace(oss.str());}} while(0) + +#define LOG_DEBUG(logger, message) do { \ + if ((logger)->debug()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->debug(oss.str());}} while(0) + +#define LOG_INFO(logger, message) do { \ + if ((logger)->information()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->information(oss.str());}} while(0) + +#define LOG_NOTICE(logger, message) do { \ + if ((logger)->notice()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->notice(oss.str());}} while(0) + +#define LOG_WARNING(logger, message) do { \ + if ((logger)->warning()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->warning(oss.str());}} while(0) + +#define LOG_ERROR(logger, message) do { \ + if ((logger)->error()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->error(oss.str());}} while(0) + +#define LOG_CRITICAL(logger, message) do { \ + if ((logger)->critical()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->critical(oss.str());}} while(0) + +#define LOG_FATAL(logger, message) do { \ + if ((logger)->fatal()) {\ + std::stringstream oss; \ + oss << message; \ + (logger)->fatal(oss.str());}} while(0) diff --git a/libs/libcommon/include/common/singleton.h b/libs/libcommon/include/common/singleton.h new file mode 100644 index 00000000000..71e37aab692 --- /dev/null +++ b/libs/libcommon/include/common/singleton.h @@ -0,0 +1,38 @@ +#pragma once + +/** Пример: + * + * class Derived : public Singleton + * { + * friend class Singleton; + * ... + * protected: + * Derived() {}; + * }; + * + * Или так: + * + * class Some + * { + * ... + * }; + * + * class SomeSingleton : public Some, public Singleton {} + */ +template class Singleton +{ +public: + static Subject & instance() + { + /// Нормально при включенных thread safe statics в gcc (по-умолчанию). + static Subject instance; + return instance; + } + +protected: + Singleton(){}; + +private: + Singleton(const Singleton&); + Singleton& operator=(const Singleton&); +}; diff --git a/libs/libcommon/include/common/strong_typedef.h b/libs/libcommon/include/common/strong_typedef.h new file mode 100644 index 00000000000..e483bff6951 --- /dev/null +++ b/libs/libcommon/include/common/strong_typedef.h @@ -0,0 +1,23 @@ +#pragma once + +#include + +/** https://svn.boost.org/trac/boost/ticket/5182 + */ +#define STRONG_TYPEDEF(T, D) \ +struct D \ + : boost::totally_ordered1< D \ + , boost::totally_ordered2< D, T \ + > > \ +{ \ + T t; \ + explicit D(const T t_) : t(t_) {}; \ + D(): t() {}; \ + D(const D & t_) : t(t_.t){} \ + D & operator=(const D & rhs) { t = rhs.t; return *this;} \ + D & operator=(const T & rhs) { t = rhs; return *this;} \ + operator const T & () const {return t; } \ + operator T & () { return t; } \ + bool operator==(const D & rhs) const { return t == rhs.t; } \ + bool operator<(const D & rhs) const { return t < rhs.t; } \ +}; diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index c14834e92e6..759b8c35917 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index e118b241838..24e45c394fa 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/libs/libcommon/src/JSON.cpp b/libs/libcommon/src/JSON.cpp new file mode 100644 index 00000000000..4652bf3b414 --- /dev/null +++ b/libs/libcommon/src/JSON.cpp @@ -0,0 +1,788 @@ +#include + +#include +#include +#include +#include + +#include + +#define JSON_MAX_DEPTH 100 + + +POCO_IMPLEMENT_EXCEPTION(JSONException, Poco::Exception, "JSONException") + + +/// Прочитать беззнаковое целое в простом формате из не-0-terminated строки. +static UInt64 readUIntText(const char * buf, const char * end) +{ + UInt64 x = 0; + + if (buf == end) + throw JSONException("JSON: cannot parse unsigned integer: unexpected end of data."); + + while (buf != end) + { + switch (*buf) + { + case '+': + break; + case '0': + case '1': + case '2': + case '3': + case '4': + case '5': + case '6': + case '7': + case '8': + case '9': + x *= 10; + x += *buf - '0'; + break; + default: + return x; + } + ++buf; + } + + return x; +} + + +/// Прочитать знаковое целое в простом формате из не-0-terminated строки. +static Int64 readIntText(const char * buf, const char * end) +{ + bool negative = false; + Int64 x = 0; + + if (buf == end) + throw JSONException("JSON: cannot parse signed integer: unexpected end of data."); + + bool run = true; + while (buf != end && run) + { + switch (*buf) + { + case '+': + break; + case '-': + negative = true; + break; + case '0': + case '1': + case '2': + case '3': + case '4': + case '5': + case '6': + case '7': + case '8': + case '9': + x *= 10; + x += *buf - '0'; + break; + default: + run = false; + break; + } + ++buf; + } + if (negative) + x = -x; + + return x; +} + + +/// Прочитать число с плавающей запятой в простом формате, с грубым округлением, из не-0-terminated строки. +static double readFloatText(const char * buf, const char * end) +{ + bool negative = false; + double x = 0; + bool after_point = false; + double power_of_ten = 1; + + if (buf == end) + throw JSONException("JSON: cannot parse floating point number: unexpected end of data."); + + bool run = true; + while (buf != end && run) + { + switch (*buf) + { + case '+': + break; + case '-': + negative = true; + break; + case '.': + after_point = true; + break; + case '0': + case '1': + case '2': + case '3': + case '4': + case '5': + case '6': + case '7': + case '8': + case '9': + if (after_point) + { + power_of_ten /= 10; + x += (*buf - '0') * power_of_ten; + } + else + { + x *= 10; + x += *buf - '0'; + } + break; + case 'e': + case 'E': + { + ++buf; + Int32 exponent = readIntText(buf, end); + x *= exp10(exponent); + + run = false; + break; + } + default: + run = false; + break; + } + ++buf; + } + if (negative) + x = -x; + + return x; +} + + +void JSON::checkInit() const +{ + if (!(ptr_begin < ptr_end)) + throw JSONException("JSON: begin >= end."); + + if (level > JSON_MAX_DEPTH) + throw JSONException("JSON: too deep."); +} + + +JSON::ElementType JSON::getType() const +{ + switch (*ptr_begin) + { + case '{': + return TYPE_OBJECT; + case '[': + return TYPE_ARRAY; + case 't': + case 'f': + return TYPE_BOOL; + case 'n': + return TYPE_NULL; + case '-': + case '0': + case '1': + case '2': + case '3': + case '4': + case '5': + case '6': + case '7': + case '8': + case '9': + return TYPE_NUMBER; + case '"': + { + /// Проверим - это просто строка или name-value pair + Pos after_string = skipString(); + if (after_string < ptr_end && *after_string == ':') + return TYPE_NAME_VALUE_PAIR; + else + return TYPE_STRING; + } + default: + throw JSONException(std::string("JSON: unexpected char ") + *ptr_begin + ", expected one of '{[tfn-0123456789\"'"); + } +} + + +void JSON::checkPos(Pos pos) const +{ + if (pos >= ptr_end) + throw JSONException("JSON: unexpected end of data."); +} + + +JSON::Pos JSON::skipString() const +{ + //std::cerr << "skipString()\t" << data() << std::endl; + + Pos pos = ptr_begin; + if (*pos != '"') + throw JSONException(std::string("JSON: expected \", got ") + *pos); + ++pos; + + /// fast path: находим следующую двойную кавычку. Если перед ней нет бэкслеша - значит это конец строки (при допущении корректности JSON). + Pos closing_quote = reinterpret_cast(memchr(reinterpret_cast(pos), '\"', ptr_end - pos)); + if (nullptr != closing_quote && closing_quote[-1] != '\\') + return closing_quote + 1; + + /// slow path + while (pos < ptr_end && *pos != '"') + { + if (*pos == '\\') + { + ++pos; + checkPos(pos); + if (*pos == 'u') + { + pos += 4; + checkPos(pos); + } + } + ++pos; + } + + if (*pos != '"') + throw JSONException(std::string("JSON: expected \", got ") + *pos); + ++pos; + + return pos; +} + + +JSON::Pos JSON::skipNumber() const +{ + //std::cerr << "skipNumber()\t" << data() << std::endl; + + Pos pos = ptr_begin; + + if (*pos == '-') + ++pos; + checkPos(pos); + + while (pos < ptr_end && *pos >= '0' && *pos <= '9') + ++pos; + if (pos < ptr_end && *pos == '.') + ++pos; + while (pos < ptr_end && *pos >= '0' && *pos <= '9') + ++pos; + if (pos < ptr_end && (*pos == 'e' || *pos == 'E')) + ++pos; + if (pos < ptr_end && *pos == '-') + ++pos; + while (pos < ptr_end && *pos >= '0' && *pos <= '9') + ++pos; + + return pos; +} + + +JSON::Pos JSON::skipBool() const +{ + //std::cerr << "skipBool()\t" << data() << std::endl; + + Pos pos = ptr_begin; + + if (*ptr_begin == 't') + pos += 4; + else if (*ptr_begin == 'f') + pos += 5; + else + throw JSONException("JSON: expected true or false."); + + return pos; +} + + +JSON::Pos JSON::skipNull() const +{ + //std::cerr << "skipNull()\t" << data() << std::endl; + + return ptr_begin + 4; +} + + +JSON::Pos JSON::skipNameValuePair() const +{ + //std::cerr << "skipNameValuePair()\t" << data() << std::endl; + + Pos pos = skipString(); + checkPos(pos); + + if (*pos != ':') + throw JSONException("JSON: expected :."); + ++pos; + + return JSON(pos, ptr_end, level + 1).skipElement(); + +} + + +JSON::Pos JSON::skipArray() const +{ + //std::cerr << "skipArray()\t" << data() << std::endl; + + if (!isArray()) + throw JSONException("JSON: expected ["); + Pos pos = ptr_begin; + ++pos; + checkPos(pos); + if (*pos == ']') + return ++pos; + + while (1) + { + pos = JSON(pos, ptr_end, level + 1).skipElement(); + if (*pos != ',') + break; + ++pos; + } + return ++pos; +} + + +JSON::Pos JSON::skipObject() const +{ + //std::cerr << "skipObject()\t" << data() << std::endl; + + if (!isObject()) + throw JSONException("JSON: expected {"); + Pos pos = ptr_begin; + ++pos; + checkPos(pos); + if (*pos == '}') + return ++pos; + + while (1) + { + pos = JSON(pos, ptr_end, level + 1).skipNameValuePair(); + if (*pos != ',') + break; + ++pos; + } + return ++pos; +} + + +JSON::Pos JSON::skipElement() const +{ + //std::cerr << "skipElement()\t" << data() << std::endl; + + ElementType type = getType(); + + switch(type) + { + case TYPE_NULL: + return skipNull(); + case TYPE_BOOL: + return skipBool(); + case TYPE_NUMBER: + return skipNumber(); + case TYPE_STRING: + return skipString(); + case TYPE_NAME_VALUE_PAIR: + return skipNameValuePair(); + case TYPE_ARRAY: + return skipArray(); + case TYPE_OBJECT: + return skipObject(); + default: + throw JSONException("Logical error in JSON: unknown element type: " + Poco::NumberFormatter::format(type)); + } +} + +size_t JSON::size() const +{ + size_t i = 0; + + for (const_iterator it = begin(); it != end(); ++it) + ++i; + + return i; +} + + +bool JSON::empty() const +{ + return size() == 0; +} + + +JSON JSON::operator[] (size_t n) const +{ + ElementType type = getType(); + + if (type != TYPE_ARRAY) + throw JSONException("JSON: not array when calling operator[](size_t) method."); + + Pos pos = ptr_begin; + ++pos; + checkPos(pos); + + size_t i = 0; + const_iterator it = begin(); + while (i < n && it != end()) + ++it, ++i; + + if (i != n) + throw JSONException("JSON: array index " + Poco::NumberFormatter::format(n) + " out of bounds."); + + return *it; +} + + +JSON::Pos JSON::searchField(const char * data, size_t size) const +{ + ElementType type = getType(); + + if (type != TYPE_OBJECT) + throw JSONException("JSON: not object when calling operator[](const char *) or has(const char *) method."); + + const_iterator it = begin(); + for (; it != end(); ++it) + { + if (!it->hasEscapes()) + { + if (static_cast(size) + 2 > it->dataEnd() - it->data()) + continue; + if (!strncmp(data, it->data() + 1, size)) + break; + } + else + { + std::string current_name = it->getName(); + if (current_name.size() == size && 0 == memcmp(current_name.data(), data, size)) + break; + } + } + + if (it == end()) + return nullptr; + else + return it->data(); +} + + +bool JSON::hasEscapes() const +{ + Pos pos = ptr_begin + 1; + while (pos < ptr_end && *pos != '"' && *pos != '\\') + ++pos; + + if (*pos == '"') + return false; + else if (*pos == '\\') + return true; + throw JSONException("JSON: unexpected end of data."); +} + + +bool JSON::hasSpecialChars() const +{ + Pos pos = ptr_begin + 1; + while (pos < ptr_end && *pos != '"' + && *pos != '\\' && *pos != '\r' && *pos != '\n' && *pos != '\t' + && *pos != '\f' && *pos != '\b' && *pos != '\0' && *pos != '\'') + ++pos; + + if (*pos == '"') + return false; + else if (pos < ptr_end) + return true; + throw JSONException("JSON: unexpected end of data."); +} + + +JSON JSON::operator[] (const std::string & name) const +{ + Pos pos = searchField(name); + if (!pos) + throw JSONException("JSON: there is no element '" + std::string(name) + "' in object."); + + return JSON(pos, ptr_end, level + 1).getValue(); +} + + +bool JSON::has(const char * data, size_t size) const +{ + return nullptr != searchField(data, size); +} + + +double JSON::getDouble() const +{ + return readFloatText(ptr_begin, ptr_end); +} + +Int64 JSON::getInt() const +{ + return readIntText(ptr_begin, ptr_end); +} + +UInt64 JSON::getUInt() const +{ + return readUIntText(ptr_begin, ptr_end); +} + +bool JSON::getBool() const +{ + if (*ptr_begin == 't') + return true; + if (*ptr_begin == 'f') + return false; + throw JSONException("JSON: cannot parse boolean."); +} + +std::string JSON::getString() const +{ + Pos s = ptr_begin; + if (*s != '"') + throw JSONException(std::string("JSON: expected \", got ") + *s); + ++s; + checkPos(s); + + std::string buf; + while (s < ptr_end) + { + switch (*s) + { + case '\\': + ++s; + checkPos(s); + + switch(*s) + { + case '"': + buf += '"'; + break; + case '\\': + buf += '\\'; + break; + case '/': + buf += '/'; + break; + case 'b': + buf += '\b'; + break; + case 'f': + buf += '\f'; + break; + case 'n': + buf += '\n'; + break; + case 'r': + buf += '\r'; + break; + case 't': + buf += '\t'; + break; + case 'u': + { + Poco::UTF8Encoding utf8; + + ++s; + checkPos(s + 4); + std::string hex(s, 4); + s += 3; + int unicode; + try + { + unicode = Poco::NumberParser::parseHex(hex); + } + catch (const Poco::SyntaxException & e) + { + throw JSONException("JSON: incorrect syntax: incorrect HEX code."); + } + buf.resize(buf.size() + 6); /// максимальный размер UTF8 многобайтовой последовательности + int res = utf8.convert(unicode, + reinterpret_cast(const_cast(buf.data())) + buf.size() - 6, 6); + if (!res) + throw JSONException("JSON: cannot convert unicode " + Poco::NumberFormatter::format(unicode) + + " to UTF8."); + buf.resize(buf.size() - 6 + res); + break; + } + default: + buf += *s; + break; + } + ++s; + break; + case '"': + return buf; + default: + buf += *s; + ++s; + break; + } + } + throw JSONException("JSON: incorrect syntax (expected end of string, found end of JSON)."); +} + +std::string JSON::getName() const +{ + return getString(); +} + +StringRef JSON::getRawString() const +{ + Pos s = ptr_begin; + if (*s != '"') + throw JSONException(std::string("JSON: expected \", got ") + *s); + while (++s != ptr_end && *s != '"'); + if (s != ptr_end ) + return StringRef(ptr_begin + 1, s - ptr_begin - 1); + throw JSONException("JSON: incorrect syntax (expected end of string, found end of JSON)."); +} + +StringRef JSON::getRawName() const +{ + return getRawString(); +} + +JSON JSON::getValue() const +{ + Pos pos = skipString(); + checkPos(pos); + if (*pos != ':') + throw JSONException("JSON: expected :."); + ++pos; + checkPos(pos); + return JSON(pos, ptr_end, level + 1); +} + + +double JSON::toDouble() const +{ + ElementType type = getType(); + + if (type == TYPE_NUMBER) + return getDouble(); + else if (type == TYPE_STRING) + return JSON(ptr_begin + 1, ptr_end, level + 1).getDouble(); + else + throw JSONException("JSON: cannot convert value to double."); +} + +Int64 JSON::toInt() const +{ + ElementType type = getType(); + + if (type == TYPE_NUMBER) + return getInt(); + else if (type == TYPE_STRING) + return JSON(ptr_begin + 1, ptr_end, level + 1).getInt(); + else + throw JSONException("JSON: cannot convert value to signed integer."); +} + +UInt64 JSON::toUInt() const +{ + ElementType type = getType(); + + if (type == TYPE_NUMBER) + return getUInt(); + else if (type == TYPE_STRING) + return JSON(ptr_begin + 1, ptr_end, level + 1).getUInt(); + else + throw JSONException("JSON: cannot convert value to unsigned integer."); +} + +std::string JSON::toString() const +{ + ElementType type = getType(); + + if (type == TYPE_STRING) + return getString(); + else + { + Pos pos = skipElement(); + return std::string(ptr_begin, pos - ptr_begin); + } +} + + +JSON::iterator JSON::iterator::begin() const +{ + ElementType type = getType(); + + if (type != TYPE_ARRAY && type != TYPE_OBJECT) + throw JSONException("JSON: not array or object when calling begin() method."); + + //std::cerr << "begin()\t" << data() << std::endl; + + Pos pos = ptr_begin + 1; + checkPos(pos); + if (*pos == '}' || *pos == ']') + return end(); + + return JSON(pos, ptr_end, level + 1); +} + +JSON::iterator JSON::iterator::end() const +{ + return JSON(nullptr, ptr_end, level + 1); +} + +JSON::iterator & JSON::iterator::operator++() +{ + Pos pos = skipElement(); + checkPos(pos); + + if (*pos != ',') + ptr_begin = nullptr; + else + { + ++pos; + checkPos(pos); + ptr_begin = pos; + } + + return *this; +} + +JSON::iterator JSON::iterator::operator++(int) +{ + iterator copy(*this); + ++*this; + return copy; +} + +template <> +double JSON::get() const +{ + return getDouble(); +} + +template <> +std::string JSON::get() const +{ + return getString(); +} + +template <> +Int64 JSON::get() const +{ + return getInt(); +} + +template <> +UInt64 JSON::get() const +{ + return getUInt(); +} + +template <> +bool JSON::get() const +{ + return getBool(); +} + diff --git a/libs/libcommon/src/Revision.cpp b/libs/libcommon/src/Revision.cpp index 31d3f3e8271..1bb66c43281 100644 --- a/libs/libcommon/src/Revision.cpp +++ b/libs/libcommon/src/Revision.cpp @@ -1,4 +1,4 @@ -#include +#include #include "revision.h" namespace Revision diff --git a/libs/libcommon/src/tests/date_lut2.cpp b/libs/libcommon/src/tests/date_lut2.cpp index 6c889bc84b9..b20efd98a7e 100644 --- a/libs/libcommon/src/tests/date_lut2.cpp +++ b/libs/libcommon/src/tests/date_lut2.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include static std::string toString(time_t Value) diff --git a/libs/libcommon/src/tests/date_lut3.cpp b/libs/libcommon/src/tests/date_lut3.cpp index df4ea9dc6b1..a55c8803d68 100644 --- a/libs/libcommon/src/tests/date_lut3.cpp +++ b/libs/libcommon/src/tests/date_lut3.cpp @@ -3,7 +3,7 @@ #include -#include +#include static std::string toString(time_t Value) diff --git a/libs/libcommon/src/tests/date_lut4.cpp b/libs/libcommon/src/tests/date_lut4.cpp index 69a15c1f25e..5a03f9c396b 100644 --- a/libs/libcommon/src/tests/date_lut4.cpp +++ b/libs/libcommon/src/tests/date_lut4.cpp @@ -1,5 +1,5 @@ #include -#include +#include int main(int argc, char ** argv) diff --git a/libs/libcommon/src/tests/date_lut_init.cpp b/libs/libcommon/src/tests/date_lut_init.cpp index 763336734fb..97377190807 100644 --- a/libs/libcommon/src/tests/date_lut_init.cpp +++ b/libs/libcommon/src/tests/date_lut_init.cpp @@ -1,4 +1,4 @@ -#include +#include /// Позволяет проверить время инициализации DateLUT. int main(int argc, char ** argv) diff --git a/libs/libcommon/src/tests/multi_version.cpp b/libs/libcommon/src/tests/multi_version.cpp index c26bb8198cc..5f9928c9765 100644 --- a/libs/libcommon/src/tests/multi_version.cpp +++ b/libs/libcommon/src/tests/multi_version.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include typedef std::string T; diff --git a/libs/libmysqlxx/include/mysqlxx/Connection.h b/libs/libmysqlxx/include/mysqlxx/Connection.h index c80e38a1e34..54363ad43d5 100644 --- a/libs/libmysqlxx/include/mysqlxx/Connection.h +++ b/libs/libmysqlxx/include/mysqlxx/Connection.h @@ -4,7 +4,7 @@ #include -#include +#include #include diff --git a/libs/libmysqlxx/include/mysqlxx/Date.h b/libs/libmysqlxx/include/mysqlxx/Date.h index 03ecec0f619..b1fa5b6b362 100644 --- a/libs/libmysqlxx/include/mysqlxx/Date.h +++ b/libs/libmysqlxx/include/mysqlxx/Date.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/libs/libmysqlxx/include/mysqlxx/DateTime.h b/libs/libmysqlxx/include/mysqlxx/DateTime.h index c4bebb95e17..83b843a543f 100644 --- a/libs/libmysqlxx/include/mysqlxx/DateTime.h +++ b/libs/libmysqlxx/include/mysqlxx/DateTime.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/libs/libmysqlxx/include/mysqlxx/Pool.h b/libs/libmysqlxx/include/mysqlxx/Pool.h index d2c13d31013..073aa48accc 100644 --- a/libs/libmysqlxx/include/mysqlxx/Pool.h +++ b/libs/libmysqlxx/include/mysqlxx/Pool.h @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include diff --git a/libs/libmysqlxx/include/mysqlxx/Value.h b/libs/libmysqlxx/include/mysqlxx/Value.h index e6e4988d9ef..e89867c8490 100644 --- a/libs/libmysqlxx/include/mysqlxx/Value.h +++ b/libs/libmysqlxx/include/mysqlxx/Value.h @@ -8,8 +8,8 @@ #include #include -#include -#include +#include +#include #include diff --git a/libs/libpocoext/src/ThreadNumber.cpp b/libs/libpocoext/src/ThreadNumber.cpp index 0637013296b..c1305e0ac58 100644 --- a/libs/libpocoext/src/ThreadNumber.cpp +++ b/libs/libpocoext/src/ThreadNumber.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include diff --git a/libs/libzkutil/include/zkutil/LeaderElection.h b/libs/libzkutil/include/zkutil/LeaderElection.h index 8a667171294..c1218a2efea 100644 --- a/libs/libzkutil/include/zkutil/LeaderElection.h +++ b/libs/libzkutil/include/zkutil/LeaderElection.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace zkutil diff --git a/libs/libzkutil/include/zkutil/Lock.h b/libs/libzkutil/include/zkutil/Lock.h index d008972c3cd..e92482f049e 100644 --- a/libs/libzkutil/include/zkutil/Lock.h +++ b/libs/libzkutil/include/zkutil/Lock.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include namespace zkutil diff --git a/libs/libzkutil/include/zkutil/Types.h b/libs/libzkutil/include/zkutil/Types.h index b6d50dca5cf..0b8faaac8f2 100644 --- a/libs/libzkutil/include/zkutil/Types.h +++ b/libs/libzkutil/include/zkutil/Types.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include diff --git a/libs/libzkutil/include/zkutil/ZooKeeper.h b/libs/libzkutil/include/zkutil/ZooKeeper.h index 99fe588e03e..e2727265ca2 100644 --- a/libs/libzkutil/include/zkutil/ZooKeeper.h +++ b/libs/libzkutil/include/zkutil/ZooKeeper.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace zkutil diff --git a/libs/libzkutil/src/ZooKeeper.cpp b/libs/libzkutil/src/ZooKeeper.cpp index bde84d57a28..26ea0aae896 100644 --- a/libs/libzkutil/src/ZooKeeper.cpp +++ b/libs/libzkutil/src/ZooKeeper.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include diff --git a/libs/libzkutil/src/tests/zkutil_test.cpp b/libs/libzkutil/src/tests/zkutil_test.cpp index 0693ff93c94..9ea3ddbac12 100644 --- a/libs/libzkutil/src/tests/zkutil_test.cpp +++ b/libs/libzkutil/src/tests/zkutil_test.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include