Resolved some of PVS-Studio warnings

This commit is contained in:
Alexey Milovidov 2019-04-22 19:07:09 +03:00
parent e5ca222129
commit 58d7d1e451
20 changed files with 25 additions and 520 deletions

View File

@ -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<TaskCluster> task_cluster;

View File

@ -16,7 +16,7 @@ std::vector<XMLConfigurationPtr> ConfigPreprocessor::processConfig(
std::vector<XMLConfigurationPtr> 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());
}

View File

@ -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;
};

View File

@ -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);

View File

@ -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<UInt8>;

View File

@ -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_) {}

View File

@ -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)

View File

@ -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_) {}

View File

@ -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);

View File

@ -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;

View File

@ -312,7 +312,7 @@ template <typename T, typename ReturnType>
ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
{
static_assert(std::is_same_v<T, double> || std::is_same_v<T, float>, "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<ReturnType, void>;

View File

@ -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.
*/

View File

@ -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;

View File

@ -77,7 +77,7 @@ namespace
{
const char * data = nullptr;
const char * null = nullptr;
size_t stride;
size_t stride = 0;
};
struct ColumnDataPlaceholder

View File

@ -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_) {}

View File

@ -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{};
};
}

View File

@ -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);

View File

@ -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)

View File

@ -1,378 +0,0 @@
#include <iostream>
#include <random>
#include <pcg_random.hpp>
#include <cmath>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromFile.h>
#include <Common/HashTable/Hash.h>
#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<UInt64>(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<UInt8>(0, 59)(random));
EventTime.second(std::uniform_int_distribution<UInt8>(0, 59)(random));
UInt64 UserID = hash(4, powerLaw(5000, 1.1));
UserID = UserID / 10000000000ULL * 10000000000ULL + static_cast<time_t>(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<std::pair<UInt16, UInt16>>(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<UInt16>(160, 3000)(random)
: (std::uniform_int_distribution<UInt16>(160, 3000)(random) / 16 * 16);
ResolutionHeight = std::bernoulli_distribution(0.1)(random)
? std::uniform_int_distribution<UInt16>(160, 3000)(random)
: (ResolutionWidth / 16 * 10);
}
writeText(ResolutionWidth, out);
writeChar('\t', out);
writeText(ResolutionHeight, out);
writeChar('\t', out);
UInt8 ResolutionDepth = weightedSelect<UInt8>({32, 24, 0, 16, 8}, {2000000, 700000, 300000, 50000, 100});
writeText(ResolutionDepth, out);
writeChar('\t', out);
UInt8 FlashMajor = weightedSelect<UInt8>({11, 0, 10, 6, 9, 8, 7, 5, 12}, {2000000, 600000, 200000, 100000, 8000, 800, 600, 20, 10});
UInt8 FlashMinor = weightedSelect<UInt8>({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<UInt8>({0, 3, 2, 1, 4}, {3000000, 100000, 10000, 5000, 2});
UInt8 NetMinor = weightedSelect<UInt8>({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<UInt8>(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<Int8>(
{-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<UInt8>(
{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<double>(0, 1)(random));
}
template <typename T>
T powerLawSampleFrom(double scale, double alpha, std::initializer_list<T> set)
{
return set.begin()[powerLaw(scale, alpha) % set.size()];
}
template <typename T>
T weightedSelect(std::initializer_list<T> items, std::initializer_list<double> 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;
}
}

View File

@ -1,114 +0,0 @@
#include <iostream>
#include <random>
#include <pcg_random.hpp>
#include <boost/program_options.hpp>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFile.h>
#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<unsigned>(), "order of model to create")
("noise", boost::program_options::value<double>(), "relative random noise to apply to created model")
("generate", "generate random strings with model")
("max-string-size", boost::program_options::value<UInt64>()->default_value(10000), "maximum size of generated string")
("limit", boost::program_options::value<UInt64>(), "stop after specified count of generated strings")
("seed", boost::program_options::value<UInt64>(), "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<UInt64>());
if (options.count("create"))
{
MarkovModel model(options["order"].as<unsigned>());
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<double>();
model.modifyCounts([&](UInt32 count)
{
double modified = std::normal_distribution<double>(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<UInt64>() : 0;
UInt64 max_string_size = options["max-string-size"].as<UInt64>();
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;
}