From 58d7d1e45195cf01fb26d8771f1f56c2c872e92f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Apr 2019 19:07:09 +0300 Subject: [PATCH 1/2] Resolved some of PVS-Studio warnings --- dbms/programs/copier/ClusterCopier.cpp | 2 +- .../performance-test/ConfigPreprocessor.cpp | 2 +- dbms/src/Common/HashTable/HashTable.h | 4 +- .../src/Common/ZooKeeper/ZooKeeperNodeCache.h | 2 +- dbms/src/Core/SortCursor.h | 2 +- .../CollapsingFinalBlockInputStream.h | 2 +- .../MergingSortedBlockInputStream.h | 2 +- .../src/DataStreams/ParallelInputsProcessor.h | 2 +- .../src/Dictionaries/ExternalQueryBuilder.cpp | 2 +- dbms/src/IO/HexWriteBuffer.h | 2 +- dbms/src/IO/readFloatText.h | 2 +- dbms/src/Interpreters/Aggregator.h | 4 +- dbms/src/Interpreters/Cluster.h | 4 +- dbms/src/Interpreters/ExpressionJIT.cpp | 2 +- dbms/src/Storages/MergeTree/BoolMask.h | 4 +- libs/libmysqlxx/include/mysqlxx/Row.h | 6 +- libs/libmysqlxx/src/Query.cpp | 3 + utils/test-data-generator/CMakeLists.txt | 6 - utils/test-data-generator/main.cpp | 378 ------------------ utils/test-data-generator/markov-model.cpp | 114 ------ 20 files changed, 25 insertions(+), 520 deletions(-) delete mode 100644 utils/test-data-generator/main.cpp delete mode 100644 utils/test-data-generator/markov-model.cpp diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 67351d9696d..75096df74ed 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -2054,7 +2054,7 @@ private: ConfigurationPtr task_cluster_initial_config; ConfigurationPtr task_cluster_current_config; - Coordination::Stat task_descprtion_current_stat; + Coordination::Stat task_descprtion_current_stat{}; std::unique_ptr task_cluster; diff --git a/dbms/programs/performance-test/ConfigPreprocessor.cpp b/dbms/programs/performance-test/ConfigPreprocessor.cpp index c448d84bc88..3ea095a5175 100644 --- a/dbms/programs/performance-test/ConfigPreprocessor.cpp +++ b/dbms/programs/performance-test/ConfigPreprocessor.cpp @@ -16,7 +16,7 @@ std::vector ConfigPreprocessor::processConfig( std::vector result; for (const auto & path : paths) { - result.emplace_back(new XMLConfiguration(path)); + result.emplace_back(XMLConfigurationPtr(new XMLConfiguration(path))); result.back()->setString("path", Poco::Path(path).absolute().toString()); } diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index 795c05a2a3e..f15fe22f8fd 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -526,7 +526,7 @@ public: { public: Reader(DB::ReadBuffer & in_) - : in(in_) + : in(in_) { } @@ -566,7 +566,7 @@ public: DB::ReadBuffer & in; Cell cell; size_t read_count = 0; - size_t size; + size_t size = 0; bool is_eof = false; bool is_initialized = false; }; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h index 73e7753cdcf..8bdb55e6ebd 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h @@ -41,7 +41,7 @@ public: { bool exists = false; std::string contents; - Coordination::Stat stat; + Coordination::Stat stat{}; }; ZNode get(const std::string & path, EventPtr watch_event); diff --git a/dbms/src/Core/SortCursor.h b/dbms/src/Core/SortCursor.h index cd427fe8ab1..5a49209cb71 100644 --- a/dbms/src/Core/SortCursor.h +++ b/dbms/src/Core/SortCursor.h @@ -29,7 +29,7 @@ struct SortCursorImpl * Cursor number (always?) equals to number of merging part. * Therefore this field can be used to determine part number of current row (see ColumnGathererStream). */ - size_t order; + size_t order = 0; using NeedCollationFlags = std::vector; diff --git a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h index 769c8de7e2b..af049cce7c0 100644 --- a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h +++ b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h @@ -163,7 +163,7 @@ private: struct Cursor { MergingBlockPtr block; - size_t pos; + size_t pos = 0; Cursor() {} explicit Cursor(const MergingBlockPtr & block_, size_t pos_ = 0) : block(block_), pos(pos_) {} diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.h b/dbms/src/DataStreams/MergingSortedBlockInputStream.h index be05783c3ed..c519914f174 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.h @@ -81,7 +81,7 @@ protected: struct RowRef { ColumnRawPtrs * columns = nullptr; - size_t row_num; + size_t row_num = 0; SharedBlockPtr shared_block; void swap(RowRef & other) diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 9c7a1fc6928..9088323af45 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -163,7 +163,7 @@ private: struct InputData { BlockInputStreamPtr in; - size_t i; /// The source number (for debugging). + size_t i = 0; /// The source number (for debugging). InputData() {} InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {} diff --git a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp index 029229cf9a4..8f3135f23b7 100644 --- a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp @@ -298,7 +298,7 @@ ExternalQueryBuilder::composeLoadKeysQuery(const Columns & key_columns, const st composeKeyCondition(key_columns, row, out); } } - else if (method == IN_WITH_TUPLES) + else /* if (method == IN_WITH_TUPLES) */ { writeString(composeKeyTupleDefinition(), out); writeString(" IN (", out); diff --git a/dbms/src/IO/HexWriteBuffer.h b/dbms/src/IO/HexWriteBuffer.h index 80be88692e3..a68dd29065b 100644 --- a/dbms/src/IO/HexWriteBuffer.h +++ b/dbms/src/IO/HexWriteBuffer.h @@ -15,7 +15,7 @@ namespace DB class HexWriteBuffer final : public WriteBuffer { protected: - char buf[DBMS_HEX_WRITE_BUFFER_SIZE]; + char buf[DBMS_HEX_WRITE_BUFFER_SIZE]; //-V730 WriteBuffer & out; void nextImpl() override; diff --git a/dbms/src/IO/readFloatText.h b/dbms/src/IO/readFloatText.h index 7dea367d924..01303886b7e 100644 --- a/dbms/src/IO/readFloatText.h +++ b/dbms/src/IO/readFloatText.h @@ -312,7 +312,7 @@ template ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) { static_assert(std::is_same_v || std::is_same_v, "Argument for readFloatTextImpl must be float or double"); - static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); + static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); //-V501 static constexpr bool throw_exception = std::is_same_v; diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 5a13599bf89..c3f17f37b03 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -407,12 +407,12 @@ struct AggregatedDataVariants : private boost::noncopyable */ Aggregator * aggregator = nullptr; - size_t keys_size; /// Number of keys. NOTE do we need this field? + size_t keys_size{}; /// Number of keys. NOTE do we need this field? Sizes key_sizes; /// Dimensions of keys, if keys of fixed length /// Pools for states of aggregate functions. Ownership will be later transferred to ColumnAggregateFunction. Arenas aggregates_pools; - Arena * aggregates_pool; /// The pool that is currently used for allocation. + Arena * aggregates_pool{}; /// The pool that is currently used for allocation. /** Specialization for the case when there are no keys, and for keys not fitted into max_rows_to_group_by. */ diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 8179caf10a0..06714da5cef 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -112,8 +112,8 @@ public: /// Name of directory for asynchronous write to StorageDistributed if has_internal_replication std::string dir_name_for_internal_replication; /// Number of the shard, the indexation begins with 1 - UInt32 shard_num; - UInt32 weight; + UInt32 shard_num = 0; + UInt32 weight = 1; Addresses local_addresses; /// nullptr if there are no remote addresses ConnectionPoolWithFailoverPtr pool; diff --git a/dbms/src/Interpreters/ExpressionJIT.cpp b/dbms/src/Interpreters/ExpressionJIT.cpp index 8cb9f2003e1..567341c2490 100644 --- a/dbms/src/Interpreters/ExpressionJIT.cpp +++ b/dbms/src/Interpreters/ExpressionJIT.cpp @@ -77,7 +77,7 @@ namespace { const char * data = nullptr; const char * null = nullptr; - size_t stride; + size_t stride = 0; }; struct ColumnDataPlaceholder diff --git a/dbms/src/Storages/MergeTree/BoolMask.h b/dbms/src/Storages/MergeTree/BoolMask.h index a7e66a917b0..75f9a6ff363 100644 --- a/dbms/src/Storages/MergeTree/BoolMask.h +++ b/dbms/src/Storages/MergeTree/BoolMask.h @@ -3,8 +3,8 @@ /// Multiple Boolean values. That is, two Boolean values: can it be true, can it be false. struct BoolMask { - bool can_be_true; - bool can_be_false; + bool can_be_true = false; + bool can_be_false = false; BoolMask() {} BoolMask(bool can_be_true_, bool can_be_false_) : can_be_true(can_be_true_), can_be_false(can_be_false_) {} diff --git a/libs/libmysqlxx/include/mysqlxx/Row.h b/libs/libmysqlxx/include/mysqlxx/Row.h index 6e22695bd19..a0b88638546 100644 --- a/libs/libmysqlxx/include/mysqlxx/Row.h +++ b/libs/libmysqlxx/include/mysqlxx/Row.h @@ -80,9 +80,9 @@ public: operator private_bool_type() const { return row == nullptr ? nullptr : &Row::row; } private: - MYSQL_ROW row = nullptr; - ResultBase * res = nullptr; - MYSQL_LENGTHS lengths; + MYSQL_ROW row{}; + ResultBase * res{}; + MYSQL_LENGTHS lengths{}; }; } diff --git a/libs/libmysqlxx/src/Query.cpp b/libs/libmysqlxx/src/Query.cpp index 57609e8365a..08474b8d864 100644 --- a/libs/libmysqlxx/src/Query.cpp +++ b/libs/libmysqlxx/src/Query.cpp @@ -40,6 +40,9 @@ Query::Query(const Query & other) : std::ostream(0), conn(other.conn) Query & Query::operator= (const Query & other) { + if (this == &other) + return *this; + conn = other.conn; seekp(0); diff --git a/utils/test-data-generator/CMakeLists.txt b/utils/test-data-generator/CMakeLists.txt index 787d29e4087..2e11b537873 100644 --- a/utils/test-data-generator/CMakeLists.txt +++ b/utils/test-data-generator/CMakeLists.txt @@ -1,9 +1,3 @@ -add_executable (test-data-generator main.cpp) -target_link_libraries(test-data-generator PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) - -add_executable (markov-model markov-model.cpp) -target_link_libraries(markov-model PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) - if(USE_PROTOBUF) protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs ProtobufDelimitedMessagesSerializer_Hdrs ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format.proto) protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs2 ProtobufDelimitedMessagesSerializer_Hdrs2 ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto) diff --git a/utils/test-data-generator/main.cpp b/utils/test-data-generator/main.cpp deleted file mode 100644 index 2a0db1e4b3d..00000000000 --- a/utils/test-data-generator/main.cpp +++ /dev/null @@ -1,378 +0,0 @@ -#include -#include -#include -#include - -#include -#include -#include -#include - -#include "MarkovModel.h" - - -/** Generate infinite stream of pseudo-random data - * like 'hits' table in Yandex.Metrica (with structure as it was in 2013) - * and output it in TabSeparated format. - */ - -using namespace DB; - - -struct Models -{ - MarkovModel Title; - MarkovModel URL; - MarkovModel SearchPhrase; - - static void read(MarkovModel & model, const String & path) - { - ReadBufferFromFile in(path); - model.read(in); - } - - Models() - { - read(Title, "Title.model"); - read(URL, "URL.model"); - read(SearchPhrase, "SearchPhrase.model"); - } -}; - - -struct Generator -{ - WriteBufferFromFileDescriptor out; - pcg64 random; - pcg64 random_with_seed; - Models models; - -// UInt64 WatchID = random(); - String Title; - String URL; -/* String Referer; - String FlashMinor2; - String UserAgentMinor; - String MobilePhoneModel; - String Params;*/ - String SearchPhrase; -/* String PageCharset; - String OriginalURL; - String BrowserLanguage; - String BrowserCountry; - String SocialNetwork; - String SocialAction; - String SocialSourcePage; - String ParamCurrency; - String OpenstatServiceName; - String OpenstatCampaignID; - String OpenstatAdID; - String OpenstatSourceID; - String UTMSource; - String UTMMedium; - String UTMCampaign; - String UTMContent; - String UTMTerm; - String FromTag;*/ - - Generator() : out(STDOUT_FILENO) {} - - /** Choosing of distributions parameters sometimes resembles real data, but quite arbitary. - */ - - void generateRow() - { -// auto gen_random64 = [&]{ return random(); }; - - /// Unique identifier of event. -/* WatchID += std::uniform_int_distribution(0, 10000000000)(random); - writeText(WatchID, out); - writeChar('\t', out); - - bool JavaEnable = std::bernoulli_distribution(0.6)(random); - writeText(JavaEnable, out); - writeChar('\t', out);*/ - - LocalDateTime EventTime; - EventTime.year(2013); - EventTime.month(7); - EventTime.day(std::discrete_distribution<>({ - 0, 0, 13, 30, 0, 14, 42, 5, 6, 31, 17, 0, 0, 0, 0, 23, 10, 0, 0, 0, 19, 24, 8, 7, 0, 0, 8, 2, 15, 12, 7, 29})(random)); - EventTime.hour(std::discrete_distribution<>({ - 13, 7, 4, 3, 2, 3, 4, 6, 10, 16, 20, 23, 24, 23, 18, 19, 19, 19, 14, 15, 14, 13, 17, 17})(random)); - EventTime.minute(std::uniform_int_distribution(0, 59)(random)); - EventTime.second(std::uniform_int_distribution(0, 59)(random)); - - UInt64 UserID = hash(4, powerLaw(5000, 1.1)); - UserID = UserID / 10000000000ULL * 10000000000ULL + static_cast(EventTime) + UserID % 1000000; - - random_with_seed.seed(powerLaw(5000, 1.1)); - auto get_random_with_seed = [&]{ return random_with_seed(); }; - - Title.resize(10000); - Title.resize(models.Title.generate(&Title[0], Title.size(), get_random_with_seed)); - writeText(Title, out); - writeChar('\t', out); - -/* bool GoodEvent = 1; - writeText(GoodEvent, out); - writeChar('\t', out);*/ - - writeText(EventTime, out); - writeChar('\t', out); - - LocalDate EventDate = EventTime.toDate(); - writeText(EventDate, out); - writeChar('\t', out); - - UInt32 CounterID = hash(1, powerLaw(20, 1.1)) % 10000000; - writeText(CounterID, out); - writeChar('\t', out); - -/* UInt32 ClientIP = hash(2, powerLaw(5000, 1.1)); - writeText(ClientIP, out); - writeChar('\t', out); - - UInt32 RegionID = hash(3, powerLaw(15, 1.1)) % 5000; - writeText(RegionID, out); - writeChar('\t', out); -*/ - writeText(UserID, out); - writeChar('\t', out); - -/* bool CounterClass = (hash(5, CounterID) % 100) < 25; - writeText(CounterClass, out); - writeChar('\t', out); - - UInt8 OS = hash(6, powerLaw(10, 4)) % 100; - writeText(OS, out); - writeChar('\t', out); - - UInt8 UserAgent = hash(7, powerLaw(10, 4)) % 100; - writeText(UserAgent, out); - writeChar('\t', out); -*/ - URL.resize(10000); - URL.resize(models.URL.generate(&URL[0], URL.size(), get_random_with_seed)); - writeText(URL, out); - writeChar('\t', out); - - /// Referer - -/* bool Refresh = std::bernoulli_distribution(0.1)(random); - writeText(Refresh, out); - writeChar('\t', out); - - UInt16 RefererCategoryID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(8, powerLaw(10, 4)) % 10000); - writeText(RefererCategoryID, out); - writeChar('\t', out); - - UInt32 RefererRegionID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(9, powerLaw(15, 1.1)) % 5000); - writeText(RefererRegionID, out); - writeChar('\t', out); - - UInt16 URLCategoryID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(10, powerLaw(10, 4)) % 10000); - writeText(URLCategoryID, out); - writeChar('\t', out); - - UInt32 URLRegionID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(11, powerLaw(15, 1.1)) % 5000); - writeText(URLRegionID, out); - writeChar('\t', out); - - UInt16 ResolutionWidth; - UInt16 ResolutionHeight; - - std::tie(ResolutionWidth, ResolutionHeight) = powerLawSampleFrom>(15, 1.1, - { - {1366, 768}, {1280, 1024}, {1920, 1080}, {0, 0}, {1024, 768}, - {1280, 800}, {1440, 900}, {1600, 900}, {1600, 900}, {1680, 1050}, - {768, 1024}, {1024, 600}, {1360, 768}, {1280, 720}, {1152, 864}, - {1280, 768}, {320, 480}, {1920, 1200}, {320, 568}, {1093, 614}, - }); - - if (std::bernoulli_distribution(0.1)(random)) - { - ResolutionWidth = std::bernoulli_distribution(0.1)(random) - ? std::uniform_int_distribution(160, 3000)(random) - : (std::uniform_int_distribution(160, 3000)(random) / 16 * 16); - - ResolutionHeight = std::bernoulli_distribution(0.1)(random) - ? std::uniform_int_distribution(160, 3000)(random) - : (ResolutionWidth / 16 * 10); - } - - writeText(ResolutionWidth, out); - writeChar('\t', out); - - writeText(ResolutionHeight, out); - writeChar('\t', out); - - UInt8 ResolutionDepth = weightedSelect({32, 24, 0, 16, 8}, {2000000, 700000, 300000, 50000, 100}); - writeText(ResolutionDepth, out); - writeChar('\t', out); - - UInt8 FlashMajor = weightedSelect({11, 0, 10, 6, 9, 8, 7, 5, 12}, {2000000, 600000, 200000, 100000, 8000, 800, 600, 20, 10}); - UInt8 FlashMinor = weightedSelect({7, 0, 8, 1, 6, 3, 2, 5, 4}, {1500000, 700000, 500000, 150000, 100000, 80000, 60000, 50000, 50000}); - - writeText(FlashMajor, out); - writeChar('\t', out); - - writeText(FlashMinor, out); - writeChar('\t', out); - - FlashMinor2.clear(); - if (FlashMajor && FlashMinor) - { - FlashMinor2 = toString(hash(12, powerLaw(10, 4)) % 1000); - if (std::bernoulli_distribution(0.2)(random)) - { - FlashMinor2 += '.'; - FlashMinor2 += toString(hash(13, powerLaw(10, 4)) % 1000); - } - } - - writeText(FlashMinor2, out); - writeChar('\t', out); - - UInt8 NetMajor = weightedSelect({0, 3, 2, 1, 4}, {3000000, 100000, 10000, 5000, 2}); - UInt8 NetMinor = weightedSelect({0, 5, 1}, {3000000, 200000, 5000}); - - writeText(NetMajor, out); - writeChar('\t', out); - - writeText(NetMinor, out); - writeChar('\t', out); - - UInt16 UserAgentMajor = UserAgent ? hash(14, powerLaw(10, 4)) % 100 : 0; - writeText(UserAgentMajor, out); - writeChar('\t', out); - - UserAgentMinor.clear(); - if (UserAgentMajor) - { - UserAgentMinor = toString(hash(15, powerLaw(10, 4)) % 100); - if (UserAgentMinor.size() == 1 && std::bernoulli_distribution(0.1)(random)) - UserAgentMinor += 'a' + std::uniform_int_distribution(0, 25)(random); - } - writeText(UserAgentMinor, out); - writeChar('\t', out); - - bool CookieEnable = std::bernoulli_distribution(0.999)(random); - writeText(CookieEnable, out); - writeChar('\t', out); - - bool JavascriptEnable = std::bernoulli_distribution(0.95)(random); - writeText(JavascriptEnable, out); - writeChar('\t', out); - - bool IsMobile = std::bernoulli_distribution(0.15)(random); - writeText(IsMobile, out); - writeChar('\t', out); - - UInt8 MobilePhone = IsMobile ? hash(16, powerLaw(10, 4)) % 100 : 0; - writeText(MobilePhone, out); - writeChar('\t', out); -*/ -/* MobilePhoneModel.resize(100); - MobilePhoneModel.resize(models.MobilePhoneModel.generate(&MobilePhoneModel[0], MobilePhoneModel.size(), gen_random64)); - writeText(MobilePhoneModel, out); - writeChar('\t', out); - - Params.resize(10000); - Params.resize(models.Params.generate(&Params[0], Params.size(), gen_random64)); - writeText(Params, out); - writeChar('\t', out); - - UInt32 IPNetworkID = hash(17, powerLaw(15, 1.1)) % 5000; - writeText(IPNetworkID, out); - writeChar('\t', out); - - Int8 TraficSourceID = weightedSelect( - {-1, 0, 1, 2, 3, 4, 5, 6, 7, 8}, {2000000, 300000, 200000, 600000, 50000, 700, 30000, 40000, 500, 2500}); - writeText(TraficSourceID, out); - writeChar('\t', out); - - UInt16 SearchEngineID = TraficSourceID == 2 - ? hash(18, powerLaw(10, 4)) % 100 - : (TraficSourceID == 3 - ? (std::bernoulli_distribution(0.5)(random) - ? hash(19, powerLaw(10, 4)) % 10 - : 0) - : 0); - - if (!SearchEngineID) - SearchPhrase.clear(); - else - {*/ - SearchPhrase.resize(1000); - SearchPhrase.resize(models.SearchPhrase.generate(&SearchPhrase[0], SearchPhrase.size(), get_random_with_seed)); -// } - writeText(SearchPhrase, out); - /* writeChar('\t', out); - - UInt8 AdvEngineID = weightedSelect( - {0, 2, 12, 17, 18, 27, 34, 36}, {3000000, 30000, 3000, 30000, 1, 100, 40, 30}); - writeText(AdvEngineID, out); - writeChar('\t', out); - - bool IsArtificial = std::bernoulli_distribution(0.07)(random); - writeText(IsArtificial, out); - writeChar('\t', out); - - UInt16 WindowClientWidth = std::max(3000, ResolutionWidth - hash(20, UserID) % 100); - UInt16 WindowClientHeight = std::max(3000, ResolutionHeight - hash(21, UserID) % 100); - - writeText(WindowClientWidth, out); - writeChar('\t', out); - - writeText(WindowClientHeight, out);*/ - writeChar('\n', out); - } - - UInt64 powerLawImpl(double scale, double alpha, double unit_random_value) - { - return scale * std::pow(unit_random_value, -1.0 / alpha) - scale; - } - - UInt64 powerLaw(double scale, double alpha) - { - return powerLawImpl(scale, alpha, std::uniform_real_distribution(0, 1)(random)); - } - - template - T powerLawSampleFrom(double scale, double alpha, std::initializer_list set) - { - return set.begin()[powerLaw(scale, alpha) % set.size()]; - } - - template - T weightedSelect(std::initializer_list items, std::initializer_list weights) - { - return items.begin()[std::discrete_distribution<>(weights)(random)]; - } - - static UInt64 hash(unsigned seed, UInt64 x) - { - return intHash64(x + seed * 0xDEADBEEF); - } -}; - - -int main(int argc, char ** argv) -try -{ - Generator generator; - while (true) - generator.generateRow(); - - return 0; -} -catch (...) -{ - /// Broken pipe, when piped to 'head', by example. - if (errno != EPIPE) - { - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; - } -} diff --git a/utils/test-data-generator/markov-model.cpp b/utils/test-data-generator/markov-model.cpp deleted file mode 100644 index 87849f8f398..00000000000 --- a/utils/test-data-generator/markov-model.cpp +++ /dev/null @@ -1,114 +0,0 @@ -#include -#include -#include - -#include - -#include -#include -#include - -#include "MarkovModel.h" - -using namespace DB; - - -int main(int argc, char ** argv) -try -{ - boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "produce help message") - ("create", "create model") - ("order", boost::program_options::value(), "order of model to create") - ("noise", boost::program_options::value(), "relative random noise to apply to created model") - ("generate", "generate random strings with model") - ("max-string-size", boost::program_options::value()->default_value(10000), "maximum size of generated string") - ("limit", boost::program_options::value(), "stop after specified count of generated strings") - ("seed", boost::program_options::value(), "seed passed to random number generator") - ; - - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - - auto show_usage = [&] - { - std::cout << "Usage: \n" - << argv[0] << " --create --order=N < strings.tsv > model\n" - << argv[0] << " --generate < model > strings.tsv\n\n"; - std::cout << desc << std::endl; - }; - - if (options.count("help")) - { - show_usage(); - return 1; - } - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - pcg64 random; - - if (options.count("seed")) - random.seed(options["seed"].as()); - - if (options.count("create")) - { - MarkovModel model(options["order"].as()); - - String s; - while (!in.eof()) - { - readText(s, in); - assertChar('\n', in); - - model.consume(s.data(), s.size()); - } - - if (options.count("noise")) - { - double noise = options["noise"].as(); - model.modifyCounts([&](UInt32 count) - { - double modified = std::normal_distribution(count, count * noise)(random); - if (modified < 1) - modified = 1; - - return std::round(modified); - }); - } - - model.write(out); - } - else if (options.count("generate")) - { - MarkovModel model; - model.read(in); - String s; - - UInt64 limit = options.count("limit") ? options["limit"].as() : 0; - UInt64 max_string_size = options["max-string-size"].as(); - - for (size_t i = 0; limit == 0 || i < limit; ++i) - { - s.resize(max_string_size); - s.resize(model.generate(&s[0], s.size(), [&]{ return random(); })); - - writeText(s, out); - writeChar('\n', out); - } - } - else - { - show_usage(); - return 1; - } - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} From 720f9113799b9436bd048eb8a5edb2df9b1f4b8c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Apr 2019 22:28:10 +0300 Subject: [PATCH 2/2] Removed obsolete files --- utils/travis/normal.sh | 45 ---------------------------------------- utils/travis/pbuilder.sh | 35 ------------------------------- 2 files changed, 80 deletions(-) delete mode 100755 utils/travis/normal.sh delete mode 100755 utils/travis/pbuilder.sh diff --git a/utils/travis/normal.sh b/utils/travis/normal.sh deleted file mode 100755 index 7f45641d45f..00000000000 --- a/utils/travis/normal.sh +++ /dev/null @@ -1,45 +0,0 @@ -#!/bin/bash - -# Manual run: -# env CXX=g++-7 CC=gcc-7 utils/travis/normal.sh -# env CXX=clang++-5.0 CC=clang-5.0 utils/travis/normal.sh - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -CXX=${CXX=g++} -CC=${CC=gcc} - -set -e -set -x - -date - -# clean not used ~600mb -[ -n "$TRAVIS" ] && rm -rf .git contrib/poco/openssl - -ccache -s -ccache -M ${CCACHE_SIZE:=4G} -df -h - -date - -mkdir -p build -cd build -cmake $CUR_DIR/../.. -DCMAKE_CXX_COMPILER=`which $DEB_CXX $CXX` -DCMAKE_C_COMPILER=`which $DEB_CC $CC` \ - `# Does not optimize to speedup build, skip debug info to use less disk` \ - -DCMAKE_C_FLAGS_ADD="-O0 -g0" -DCMAKE_CXX_FLAGS_ADD="-O0 -g0" \ - `# ignore ccache disabler on trusty` \ - -DCMAKE_C_COMPILER_LAUNCHER=`which ccache` -DCMAKE_CXX_COMPILER_LAUNCHER=`which ccache` \ - `# Use all possible contrib libs from system` \ - -DUNBUNDLED=1 \ - `# Disable all features` \ - -DENABLE_CAPNP=0 -DENABLE_RDKAFKA=0 -DENABLE_EMBEDDED_COMPILER=0 -DUSE_INTERNAL_LLVM_LIBRARY=0 -DENABLE_JEMALLOC=0 -DENABLE_UNWIND=0 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 $CMAKE_FLAGS - -ninja clickhouse-bundle - -# Skip tests: -# 00281 requires internal compiler -# 00428 requires sudo (not all vms allow this) -# 00385 runs infinitly (TODO: fix it) -[ ! ${TEST_RUN=1} ] || ( ( cd $CUR_DIR/../.. && env TEST_OPT="--skip long compile 00428 00385 $TEST_OPT" TEST_PORT_RANDOM= TEST_PERF= TEST_SERVER_STARTUP_WAIT=10 bash -x dbms/tests/clickhouse-test-server ) || ${TEST_TRUE=false} ) - -date diff --git a/utils/travis/pbuilder.sh b/utils/travis/pbuilder.sh deleted file mode 100755 index d993e8715b8..00000000000 --- a/utils/travis/pbuilder.sh +++ /dev/null @@ -1,35 +0,0 @@ -#!/bin/bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) - -# env CXX=clang++-5.0 CC=clang-5.0 DH_VERBOSE=1 utils/travis/pbuilder.sh - -set -e -set -x - -df -h - -date - -env TEST_RUN=${TEST_RUN=1} \ - TEST_PORT_RANDOM= \ - `# Skip tests:` \ - `# 00416 requires patched poco from contrib/` \ - TEST_OPT="--skip long pocopatch $TEST_OPT" \ - TEST_SSL="" `# Application: SSL context exception: Error loading certificate from file /etc/clickhouse-server/server.crt: No error -- when using system poco on artful` \ - TEST_TRUE=${TEST_TRUE=false} \ - `# travisci will not upload ccache cache after timeout (48min), use our less timeout` \ - PBUILDER_OPT="--timeout ${PBUILDER_TIMEOUT:=35m} $PBUILDER_OPT" \ - `# clang is faster than gcc` \ - DEB_CC=${DEB_CC=$CC} DEB_CXX=${DEB_CXX=$CXX} \ - CCACHE_SIZE=${CCACHE_SIZE:=4G} \ - `# Disable all features` \ - CMAKE_FLAGS="-DCMAKE_BUILD_TYPE=Debug -DUNBUNDLED=1 -DENABLE_CAPNP=0 -DENABLE_RDKAFKA=0 -DENABLE_JEMALLOC=0 -DENABLE_UNWIND=0 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 -DUSE_INTERNAL_LLVM_LIBRARY=0 -DCMAKE_C_FLAGS_ADD='-O0 -g0' -DCMAKE_CXX_FLAGS_ADD='-O0 -g0' $CMAKE_FLAGS" \ - `# Use all possible contrib libs from system` \ - `# psmisc - killall` \ - EXTRAPACKAGES="psmisc clang-5.0 lld-5.0 liblld-5.0-dev libclang-5.0-dev liblld-5.0 libc++abi-dev libc++-dev libboost-program-options-dev libboost-system-dev libboost-filesystem-dev libboost-thread-dev zlib1g-dev liblz4-dev libdouble-conversion-dev libsparsehash-dev librdkafka-dev libpoco-dev libsparsehash-dev libgoogle-perftools-dev libzstd-dev libre2-dev libjemalloc-dev $EXTRAPACKAGES" \ - `# Travis trusty cant unpack bionic: E: debootstrap failed, TODO: check again, can be fixed` \ - DIST=${DIST=artful} \ - $CUR_DIR/../../release $RELEASE_OPT - -date