mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fixed warnings found by cppcheck [#CLICKHOUSE-3300].
This commit is contained in:
parent
0dcee75180
commit
8b313ab99e
@ -102,7 +102,7 @@ template <bool positive>
|
||||
struct ColumnFixedString::less
|
||||
{
|
||||
const ColumnFixedString & parent;
|
||||
less(const ColumnFixedString & parent_) : parent(parent_) {}
|
||||
explicit less(const ColumnFixedString & parent_) : parent(parent_) {}
|
||||
bool operator()(size_t lhs, size_t rhs) const
|
||||
{
|
||||
/// TODO: memcmp slows down.
|
||||
|
@ -163,7 +163,7 @@ template <bool positive>
|
||||
struct ColumnString::less
|
||||
{
|
||||
const ColumnString & parent;
|
||||
less(const ColumnString & parent_) : parent(parent_) {}
|
||||
explicit less(const ColumnString & parent_) : parent(parent_) {}
|
||||
bool operator()(size_t lhs, size_t rhs) const
|
||||
{
|
||||
size_t left_len = parent.sizeAt(lhs);
|
||||
|
@ -103,7 +103,7 @@ namespace
|
||||
IColumn::Offsets_t & res_offsets;
|
||||
IColumn::Offset_t current_src_offset = 0;
|
||||
|
||||
ResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) : res_offsets(*res_offsets_) {}
|
||||
explicit ResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) : res_offsets(*res_offsets_) {}
|
||||
|
||||
void reserve(ssize_t result_size_hint, size_t src_size)
|
||||
{
|
||||
@ -147,7 +147,7 @@ namespace
|
||||
|
||||
struct NoResultOffsetsBuilder
|
||||
{
|
||||
NoResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) {}
|
||||
explicit NoResultOffsetsBuilder(IColumn::Offsets_t * res_offsets_) {}
|
||||
void reserve(ssize_t result_size_hint, size_t src_size) {}
|
||||
void insertOne(size_t array_size) {}
|
||||
|
||||
|
@ -30,7 +30,6 @@ void throwFromErrno(const std::string & s, int code, int e)
|
||||
const size_t buf_size = 128;
|
||||
char buf[buf_size];
|
||||
#ifndef _GNU_SOURCE
|
||||
const char * unknown_message = "Unknown error ";
|
||||
int rc = strerror_r(e, buf, buf_size);
|
||||
#ifdef __APPLE__
|
||||
if (rc != 0 && rc != EINVAL)
|
||||
@ -40,6 +39,7 @@ void throwFromErrno(const std::string & s, int code, int e)
|
||||
{
|
||||
std::string tmp = std::to_string(code);
|
||||
const char * code = tmp.c_str();
|
||||
const char * unknown_message = "Unknown error ";
|
||||
strcpy(buf, unknown_message);
|
||||
strcpy(buf + strlen(unknown_message), code);
|
||||
}
|
||||
|
@ -17,19 +17,12 @@ namespace DB
|
||||
class Exception : public Poco::Exception
|
||||
{
|
||||
public:
|
||||
Exception(int code = 0) : Poco::Exception(code) {}
|
||||
Exception() {} /// For deferred initialization.
|
||||
Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {}
|
||||
Exception(const std::string & msg, const std::string & arg, int code = 0) : Poco::Exception(msg, arg, code) {}
|
||||
Exception(const std::string & msg, const Exception & exc, int code = 0) : Poco::Exception(msg, exc, code), trace(exc.trace) {}
|
||||
Exception(const Exception & exc) : Poco::Exception(exc), trace(exc.trace) {}
|
||||
explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText()) {}
|
||||
~Exception() throw() override {}
|
||||
Exception & operator = (const Exception & exc)
|
||||
{
|
||||
Poco::Exception::operator=(exc);
|
||||
trace = exc.trace;
|
||||
return *this;
|
||||
}
|
||||
|
||||
const char * name() const throw() override { return "DB::Exception"; }
|
||||
const char * className() const throw() override { return "DB::Exception"; }
|
||||
DB::Exception * clone() const override { return new DB::Exception(*this); }
|
||||
@ -49,16 +42,12 @@ private:
|
||||
class ErrnoException : public Exception
|
||||
{
|
||||
public:
|
||||
ErrnoException(int code = 0, int saved_errno_ = 0)
|
||||
: Exception(code), saved_errno(saved_errno_) {}
|
||||
ErrnoException(const std::string & msg, int code = 0, int saved_errno_ = 0)
|
||||
: Exception(msg, code), saved_errno(saved_errno_) {}
|
||||
ErrnoException(const std::string & msg, const std::string & arg, int code = 0, int saved_errno_ = 0)
|
||||
: Exception(msg, arg, code), saved_errno(saved_errno_) {}
|
||||
ErrnoException(const std::string & msg, const Exception & exc, int code = 0, int saved_errno_ = 0)
|
||||
: Exception(msg, exc, code), saved_errno(saved_errno_) {}
|
||||
ErrnoException(const ErrnoException & exc)
|
||||
: Exception(exc), saved_errno(exc.saved_errno) {}
|
||||
|
||||
int getErrno() const { return saved_errno; }
|
||||
|
||||
|
@ -5,23 +5,21 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class NetException : public DB::Exception
|
||||
{
|
||||
public:
|
||||
explicit NetException(int code = 0) : DB::Exception(code) {}
|
||||
NetException(const std::string & msg, int code = 0) : DB::Exception(msg, code) {}
|
||||
NetException(const std::string & msg, const std::string & arg, int code = 0) : DB::Exception(msg, arg, code) {}
|
||||
NetException(const std::string & msg, const DB::Exception & exc, int code = 0) : DB::Exception(msg, exc, code) {}
|
||||
|
||||
explicit NetException(const DB::Exception & exc) : DB::Exception(exc) {}
|
||||
explicit NetException(const Poco::Exception & exc) : DB::Exception(exc.displayText()) {}
|
||||
NetException(const DB::NetException & exc) = default;
|
||||
|
||||
~NetException() throw() override {}
|
||||
|
||||
const char * name() const throw() override { return "DB::NetException"; }
|
||||
const char * className() const throw() override { return "DB::NetException"; }
|
||||
DB::NetException * clone() const override { return new DB::NetException(*this); }
|
||||
void rethrow() const override { throw *this; }
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,16 +2,18 @@
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <boost/core/noncopyable.hpp>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Allows you to open a dynamic library and get a pointer to a function from it.
|
||||
|
||||
/** Allows you to open a dynamic library and get a pointer to a function from it.
|
||||
*/
|
||||
class SharedLibrary : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
SharedLibrary(const std::string & path);
|
||||
explicit SharedLibrary(const std::string & path);
|
||||
|
||||
~SharedLibrary();
|
||||
|
||||
@ -33,4 +35,5 @@ private:
|
||||
};
|
||||
|
||||
using SharedLibraryPtr = std::shared_ptr<SharedLibrary>;
|
||||
|
||||
}
|
||||
|
@ -67,8 +67,6 @@ void SingleBarrier::enter(UInt64 timeout)
|
||||
{
|
||||
__sync_synchronize();
|
||||
|
||||
bool is_first_crossing = true;
|
||||
|
||||
RWLock lock{get_zookeeper, path + "/lock"};
|
||||
|
||||
try
|
||||
@ -92,6 +90,7 @@ void SingleBarrier::enter(UInt64 timeout)
|
||||
throw KeeperException{code};
|
||||
|
||||
Stopwatch watch;
|
||||
bool is_first_crossing = true;
|
||||
|
||||
while (true)
|
||||
{
|
||||
|
@ -33,17 +33,14 @@ const int CreateMode::Ephemeral = ZOO_EPHEMERAL;
|
||||
const int CreateMode::EphemeralSequential = ZOO_EPHEMERAL | ZOO_SEQUENCE;
|
||||
const int CreateMode::PersistentSequential = ZOO_SEQUENCE;
|
||||
|
||||
void check(int32_t code, const std::string path = "")
|
||||
|
||||
static void check(int32_t code, const std::string & path)
|
||||
{
|
||||
if (code != ZOK)
|
||||
{
|
||||
if (path.size())
|
||||
throw KeeperException(code, path);
|
||||
else
|
||||
throw KeeperException(code);
|
||||
}
|
||||
throw KeeperException(code, path);
|
||||
}
|
||||
|
||||
|
||||
struct WatchContext
|
||||
{
|
||||
/// ZooKeeper instance exists for the entire WatchContext lifetime.
|
||||
|
@ -39,8 +39,7 @@ int main(int argc, char ** argv)
|
||||
int code;
|
||||
try
|
||||
{
|
||||
code = zk.tryMulti(ops);std::string unused;
|
||||
//code = zk.tryCreate("/test", "", zkutil::CreateMode::Persistent, unused);
|
||||
code = zk.tryMulti(ops);
|
||||
}
|
||||
catch (zkutil::KeeperException & e)
|
||||
{
|
||||
|
@ -54,7 +54,7 @@ struct Test
|
||||
wb.write(reinterpret_cast<const char *>(&store), sizeof(store));
|
||||
const unsigned char * p = reinterpret_cast<const unsigned char *>(&store);
|
||||
for (size_t i = 0; i < sizeof(store); ++i)
|
||||
++p;
|
||||
++*p;
|
||||
}
|
||||
|
||||
{
|
||||
|
@ -306,16 +306,16 @@ int main(int argc, char ** argv)
|
||||
|
||||
setAffinity();
|
||||
|
||||
if (!method || method == 0) test<identity> (n, &data[0], "0: identity");
|
||||
if (!method || method == 1) test<intHash32> (n, &data[0], "1: intHash32");
|
||||
if (!method || method == 2) test<intHash64> (n, &data[0], "2: intHash64");
|
||||
if (!method || method == 3) test<hash3> (n, &data[0], "3: two rounds");
|
||||
if (!method || method == 4) test<hash4> (n, &data[0], "4: two rounds and two variables");
|
||||
if (!method || method == 5) test<hash5> (n, &data[0], "5: two rounds with less ops");
|
||||
if (!method || method == 6) test<murmurMix> (n, &data[0], "6: murmur64 mixer");
|
||||
if (!method || method == 7) test<mulShift> (n, &data[0], "7: mulShift");
|
||||
if (!method || method == 8) test<tabulation>(n, &data[0], "8: tabulation");
|
||||
if (!method || method == 9) test<crc32Hash> (n, &data[0], "9: crc32");
|
||||
if (!method || method == 1) test<identity> (n, &data[0], "0: identity");
|
||||
if (!method || method == 2) test<intHash32> (n, &data[0], "1: intHash32");
|
||||
if (!method || method == 3) test<intHash64> (n, &data[0], "2: intHash64");
|
||||
if (!method || method == 4) test<hash3> (n, &data[0], "3: two rounds");
|
||||
if (!method || method == 5) test<hash4> (n, &data[0], "4: two rounds and two variables");
|
||||
if (!method || method == 6) test<hash5> (n, &data[0], "5: two rounds with less ops");
|
||||
if (!method || method == 7) test<murmurMix> (n, &data[0], "6: murmur64 mixer");
|
||||
if (!method || method == 8) test<mulShift> (n, &data[0], "7: mulShift");
|
||||
if (!method || method == 9) test<tabulation>(n, &data[0], "8: tabulation");
|
||||
if (!method || method == 10) test<crc32Hash> (n, &data[0], "9: crc32");
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
@ -46,7 +46,7 @@ void run()
|
||||
}
|
||||
}
|
||||
|
||||
void runTest(unsigned int num, const std::function<bool()> func)
|
||||
void runTest(unsigned int num, const std::function<bool()> & func)
|
||||
{
|
||||
bool ok;
|
||||
|
||||
|
@ -8,7 +8,7 @@ struct C
|
||||
{
|
||||
volatile int data;
|
||||
|
||||
C(int n = 0) : data(n) {}
|
||||
explicit C(int n = 0) : data(n) {}
|
||||
|
||||
C(const C & x)
|
||||
{
|
||||
@ -140,7 +140,7 @@ C f10()
|
||||
|
||||
C f11()
|
||||
{
|
||||
return rand() % 2 ? f1() : f2();
|
||||
return (rand() % 2) ? f1() : f2();
|
||||
}
|
||||
|
||||
C f12()
|
||||
|
@ -146,7 +146,7 @@ private:
|
||||
/// An event by which the main thread is telling merging threads that it is possible to process the next group of blocks.
|
||||
std::condition_variable have_space;
|
||||
|
||||
ParallelMergeData(size_t max_threads) : pool(max_threads) {}
|
||||
explicit ParallelMergeData(size_t max_threads) : pool(max_threads) {}
|
||||
};
|
||||
|
||||
std::unique_ptr<ParallelMergeData> parallel_merge_data;
|
||||
|
@ -241,7 +241,7 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
|
||||
/// If it is zero, and without it the output stream will be empty, we will write it anyway.
|
||||
if (!current_row_is_zero || !output_is_non_empty)
|
||||
{
|
||||
++merged_rows;
|
||||
++merged_rows; /// Dead store (result is unused). Left for clarity.
|
||||
insertCurrentRow(merged_columns);
|
||||
}
|
||||
|
||||
@ -257,15 +257,15 @@ class FieldVisitorSum : public StaticVisitor<bool>
|
||||
private:
|
||||
const Field & rhs;
|
||||
public:
|
||||
FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
|
||||
explicit FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
|
||||
|
||||
bool operator() (UInt64 & x) const { x += get<UInt64>(rhs); return x != 0; }
|
||||
bool operator() (Int64 & x) const { x += get<Int64>(rhs); return x != 0; }
|
||||
bool operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
|
||||
bool operator() (UInt64 & x) const { x += get<UInt64>(rhs); return x != 0; }
|
||||
bool operator() (Int64 & x) const { x += get<Int64>(rhs); return x != 0; }
|
||||
bool operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
|
||||
|
||||
bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
|
||||
};
|
||||
|
||||
|
||||
|
@ -230,11 +230,11 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r
|
||||
IColumn & nested_column = column_array.getData();
|
||||
|
||||
size_t size = 0;
|
||||
bool first = true;
|
||||
assertChar('[', istr);
|
||||
|
||||
try
|
||||
{
|
||||
bool first = true;
|
||||
while (!istr.eof() && *istr.position() != ']')
|
||||
{
|
||||
if (!first)
|
||||
|
@ -113,9 +113,6 @@ void DataTypeNumberBase<T>::serializeTextJSON(const IColumn & column, size_t row
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
|
||||
{
|
||||
static constexpr bool is_uint8 = std::is_same<T, UInt8>::value;
|
||||
static constexpr bool is_int8 = std::is_same<T, Int8>::value;
|
||||
|
||||
bool has_quote = false;
|
||||
if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without.
|
||||
{
|
||||
@ -135,6 +132,9 @@ void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & i
|
||||
}
|
||||
else
|
||||
{
|
||||
static constexpr bool is_uint8 = std::is_same<T, UInt8>::value;
|
||||
static constexpr bool is_int8 = std::is_same<T, Int8>::value;
|
||||
|
||||
if (is_uint8 || is_int8)
|
||||
{
|
||||
// extra conditions to parse true/false strings into 1/0
|
||||
|
@ -5,8 +5,8 @@
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
@ -35,8 +35,7 @@ try
|
||||
offsets[i] = (i + 1) * size;
|
||||
}
|
||||
|
||||
std::ofstream ostr("test");
|
||||
WriteBufferFromFileDescriptor out_buf(STDOUT_FILENO);
|
||||
WriteBufferFromFile out_buf("test");
|
||||
|
||||
stopwatch.restart();
|
||||
data_type.serializeBinaryBulk(*column, out_buf, 0, 0);
|
||||
@ -48,8 +47,7 @@ try
|
||||
{
|
||||
std::shared_ptr<ColumnString> column = std::make_shared<ColumnString>();
|
||||
|
||||
std::ifstream istr("test");
|
||||
ReadBufferFromFileDescriptor in_buf(STDIN_FILENO);
|
||||
ReadBufferFromFile in_buf("test");
|
||||
|
||||
stopwatch.restart();
|
||||
data_type.deserializeBinaryBulk(*column, in_buf, n, 0);
|
||||
|
@ -176,7 +176,7 @@ struct TableSet
|
||||
using Container = std::map<String, TableDescription>;
|
||||
Container map;
|
||||
|
||||
TableSet(const String & data)
|
||||
explicit TableSet(const String & data)
|
||||
{
|
||||
ReadBufferFromString in(data);
|
||||
read(in);
|
||||
@ -466,7 +466,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
DatabaseCloudIterator(DatabasePtr database)
|
||||
explicit DatabaseCloudIterator(DatabasePtr database)
|
||||
: owned_database(database),
|
||||
zookeeper(parent().context.getZooKeeper()),
|
||||
zookeeper_path(parent().zookeeper_path + "/tables/" + parent().name),
|
||||
|
@ -14,13 +14,12 @@ namespace ErrorCodes
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
}
|
||||
|
||||
const std::string lib_config_settings = ".settings";
|
||||
|
||||
class CStringsHolder
|
||||
{
|
||||
public:
|
||||
using strings_type = std::vector<std::string>;
|
||||
CStringsHolder(strings_type strings_pass)
|
||||
using Container = std::vector<std::string>;
|
||||
explicit CStringsHolder(const Container & strings_pass)
|
||||
{
|
||||
strings_holder = strings_pass;
|
||||
strings.size = strings_holder.size();
|
||||
@ -38,15 +37,22 @@ public:
|
||||
|
||||
private:
|
||||
std::unique_ptr<ClickHouseLibrary::CString[]> ptr_holder = nullptr;
|
||||
strings_type strings_holder;
|
||||
Container strings_holder;
|
||||
|
||||
};
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
const std::string lib_config_settings = ".settings";
|
||||
|
||||
|
||||
CStringsHolder getLibSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_root)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(config_root, config_keys);
|
||||
CStringsHolder::strings_type strings;
|
||||
CStringsHolder::Container strings;
|
||||
for (const auto & key : config_keys)
|
||||
{
|
||||
std::string key_name = key;
|
||||
@ -59,10 +65,12 @@ CStringsHolder getLibSettings(const Poco::Util::AbstractConfiguration & config,
|
||||
return CStringsHolder(strings);
|
||||
}
|
||||
|
||||
|
||||
bool dataToBlock(const void * data, Block & block)
|
||||
{
|
||||
if (!data)
|
||||
return true;
|
||||
|
||||
auto columns_received = static_cast<const ClickHouseLibrary::ColumnsUInt64 *>(data);
|
||||
std::vector<IColumn *> columns(block.columns());
|
||||
for (const auto i : ext::range(0, columns.size()))
|
||||
@ -82,6 +90,9 @@ bool dataToBlock(const void * data, Block & block)
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
LibraryDictionarySource::LibraryDictionarySource(const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
|
@ -7,6 +7,7 @@
|
||||
|
||||
#include <iostream>
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
@ -15,6 +16,7 @@ class Logger;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class CStringsHolder;
|
||||
|
||||
/// Allows loading dictionaries from dynamic libraries (.so)
|
||||
|
@ -44,7 +44,7 @@ using VarCallback = std::function<size_t(ColumnString::Chars_t & to_data,
|
||||
/// the code slightly simpler to read.
|
||||
struct Chunk
|
||||
{
|
||||
Chunk(const VarCallback & var_callback_)
|
||||
explicit Chunk(const VarCallback & var_callback_)
|
||||
: var_callback{var_callback_}
|
||||
{
|
||||
}
|
||||
@ -90,7 +90,7 @@ public:
|
||||
|
||||
Chunk get() const override
|
||||
{
|
||||
return {var_callback};
|
||||
return Chunk{var_callback};
|
||||
}
|
||||
|
||||
void next() override
|
||||
@ -174,7 +174,7 @@ public:
|
||||
|
||||
Chunk get() const override
|
||||
{
|
||||
return {var_callback};
|
||||
return Chunk{var_callback};
|
||||
}
|
||||
|
||||
void next() override
|
||||
|
@ -2304,7 +2304,7 @@ DataTypePtr FunctionRange::getReturnTypeImpl(const DataTypes & arguments) const
|
||||
|
||||
if (!checkDataType<DataTypeUInt8>(arg) &&
|
||||
!checkDataType<DataTypeUInt16>(arg) &&
|
||||
!checkDataType<DataTypeUInt32>(arg) &
|
||||
!checkDataType<DataTypeUInt32>(arg) &&
|
||||
!checkDataType<DataTypeUInt64>(arg))
|
||||
{
|
||||
throw Exception{
|
||||
|
@ -44,7 +44,7 @@ private:
|
||||
{
|
||||
UConverter * impl;
|
||||
|
||||
Converter(const String & charset)
|
||||
explicit Converter(const String & charset)
|
||||
{
|
||||
UErrorCode status = U_ZERO_ERROR;
|
||||
impl = ucnv_open(charset.data(), &status);
|
||||
|
@ -1280,7 +1280,7 @@ public:
|
||||
return std::make_shared<FunctionUptime>(context.getUptimeSeconds());
|
||||
}
|
||||
|
||||
FunctionUptime(time_t uptime_) : uptime(uptime_)
|
||||
explicit FunctionUptime(time_t uptime_) : uptime(uptime_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -1598,7 +1598,7 @@ public:
|
||||
return std::make_shared<FunctionHasColumnInTable>(context.getGlobalContext());
|
||||
}
|
||||
|
||||
FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_)
|
||||
explicit FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -49,8 +49,6 @@ try
|
||||
arg_nums.push_back(0);
|
||||
arg_nums.push_back(1);
|
||||
|
||||
size_t res_num = 2;
|
||||
|
||||
DataTypePtr res_type = f.getReturnType(arg_types);
|
||||
|
||||
ColumnWithTypeAndName descr_res;
|
||||
@ -61,6 +59,7 @@ try
|
||||
Stopwatch stopwatch;
|
||||
stopwatch.start();
|
||||
|
||||
size_t res_num = 2;
|
||||
f.execute(block, arg_nums, res_num);
|
||||
|
||||
stopwatch.stop();
|
||||
|
@ -75,7 +75,7 @@ struct AssociativeOperationImpl
|
||||
AssociativeOperationImpl<Op, N - 1> continuation;
|
||||
|
||||
/// Remembers the last N columns from in.
|
||||
AssociativeOperationImpl(UInt8ColumnPtrs & in)
|
||||
explicit AssociativeOperationImpl(UInt8ColumnPtrs & in)
|
||||
: vec(in[in.size() - N]->getData()), continuation(in) {}
|
||||
|
||||
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
|
||||
@ -97,7 +97,7 @@ struct AssociativeOperationImpl<Op, 1>
|
||||
|
||||
const UInt8 * vec;
|
||||
|
||||
AssociativeOperationImpl(UInt8ColumnPtrs & in)
|
||||
explicit AssociativeOperationImpl(UInt8ColumnPtrs & in)
|
||||
: vec(&in[in.size() - 1]->getData()[0]) {}
|
||||
|
||||
inline UInt8 apply(size_t i) const
|
||||
|
@ -86,6 +86,7 @@ struct Memory : boost::noncopyable, Allocator<false>
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
static size_t align(const size_t value, const size_t alignment)
|
||||
{
|
||||
if (!alignment)
|
||||
@ -94,7 +95,6 @@ struct Memory : boost::noncopyable, Allocator<false>
|
||||
return (value + alignment - 1) / alignment * alignment;
|
||||
}
|
||||
|
||||
private:
|
||||
void alloc()
|
||||
{
|
||||
if (!m_capacity)
|
||||
|
@ -15,12 +15,10 @@ namespace ErrorCodes
|
||||
class ReadBufferFromMemoryWriteBuffer : public ReadBuffer, boost::noncopyable, private Allocator<false>
|
||||
{
|
||||
public:
|
||||
|
||||
ReadBufferFromMemoryWriteBuffer(MemoryWriteBuffer && origin)
|
||||
:
|
||||
ReadBuffer(nullptr, 0),
|
||||
chunk_list(std::move(origin.chunk_list)),
|
||||
end_pos(origin.position())
|
||||
explicit ReadBufferFromMemoryWriteBuffer(MemoryWriteBuffer && origin)
|
||||
: ReadBuffer(nullptr, 0),
|
||||
chunk_list(std::move(origin.chunk_list)),
|
||||
end_pos(origin.position())
|
||||
{
|
||||
chunk_head = chunk_list.begin();
|
||||
setChunk();
|
||||
@ -76,11 +74,16 @@ private:
|
||||
|
||||
|
||||
MemoryWriteBuffer::MemoryWriteBuffer(size_t max_total_size_, size_t initial_chunk_size_, double growth_rate_, size_t max_chunk_size_)
|
||||
: WriteBuffer(nullptr, 0), max_total_size(max_total_size_), initial_chunk_size(initial_chunk_size_), max_chunk_size(max_chunk_size_), growth_rate(growth_rate_)
|
||||
: WriteBuffer(nullptr, 0),
|
||||
max_total_size(max_total_size_),
|
||||
initial_chunk_size(initial_chunk_size_),
|
||||
max_chunk_size(max_chunk_size_),
|
||||
growth_rate(growth_rate_)
|
||||
{
|
||||
addChunk();
|
||||
}
|
||||
|
||||
|
||||
void MemoryWriteBuffer::nextImpl()
|
||||
{
|
||||
if (unlikely(hasPendingData()))
|
||||
@ -93,6 +96,7 @@ void MemoryWriteBuffer::nextImpl()
|
||||
addChunk();
|
||||
}
|
||||
|
||||
|
||||
void MemoryWriteBuffer::addChunk()
|
||||
{
|
||||
size_t next_chunk_size;
|
||||
@ -126,6 +130,7 @@ void MemoryWriteBuffer::addChunk()
|
||||
set(chunk_tail->begin(), chunk_tail->size());
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<ReadBuffer> MemoryWriteBuffer::getReadBufferImpl()
|
||||
{
|
||||
auto res = std::make_shared<ReadBufferFromMemoryWriteBuffer>(std::move(*this));
|
||||
@ -137,6 +142,7 @@ std::shared_ptr<ReadBuffer> MemoryWriteBuffer::getReadBufferImpl()
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
MemoryWriteBuffer::~MemoryWriteBuffer()
|
||||
{
|
||||
for (const auto & range : chunk_list)
|
||||
|
@ -49,6 +49,7 @@ void parseUUID(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16)
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
/// FIXME This code looks like trash.
|
||||
parseHex(&src36[0], dst16 + 8, 4);
|
||||
parseHex(&src36[9], dst16 + 12, 2);
|
||||
parseHex(&src36[14], dst16 + 14, 2);
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <inttypes.h>
|
||||
#include <stdio.h>
|
||||
#include <Common/hex.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -8,30 +9,11 @@ namespace DB
|
||||
template <typename IteratorSrc, typename IteratorDst>
|
||||
void formatHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes)
|
||||
{
|
||||
/// More optimal than lookup table by nibbles.
|
||||
constexpr auto hex =
|
||||
"000102030405060708090a0b0c0d0e0f"
|
||||
"101112131415161718191a1b1c1d1e1f"
|
||||
"202122232425262728292a2b2c2d2e2f"
|
||||
"303132333435363738393a3b3c3d3e3f"
|
||||
"404142434445464748494a4b4c4d4e4f"
|
||||
"505152535455565758595a5b5c5d5e5f"
|
||||
"606162636465666768696a6b6c6d6e6f"
|
||||
"707172737475767778797a7b7c7d7e7f"
|
||||
"808182838485868788898a8b8c8d8e8f"
|
||||
"909192939495969798999a9b9c9d9e9f"
|
||||
"a0a1a2a3a4a5a6a7a8a9aaabacadaeaf"
|
||||
"b0b1b2b3b4b5b6b7b8b9babbbcbdbebf"
|
||||
"c0c1c2c3c4c5c6c7c8c9cacbcccdcecf"
|
||||
"d0d1d2d3d4d5d6d7d8d9dadbdcdddedf"
|
||||
"e0e1e2e3e4e5e6e7e8e9eaebecedeeef"
|
||||
"f0f1f2f3f4f5f6f7f8f9fafbfcfdfeff";
|
||||
|
||||
size_t src_pos = 0;
|
||||
size_t dst_pos = 0;
|
||||
for (; src_pos < num_bytes; ++src_pos)
|
||||
{
|
||||
memcpy(&dst[dst_pos], &hex[src[src_pos] * 2], 2);
|
||||
writeHexByteLowercase(src[src_pos], &dst[dst_pos]);
|
||||
dst_pos += 2;
|
||||
}
|
||||
}
|
||||
|
@ -43,7 +43,6 @@ int main(int argc, char ** argv)
|
||||
{
|
||||
DB::ReadBufferFromFile buf("test1");
|
||||
DB::CompressedReadBuffer compressed_buf(buf);
|
||||
std::string s;
|
||||
|
||||
stopwatch.restart();
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
|
@ -20,7 +20,7 @@ void prepare3(std::string & filename, std::string & buf);
|
||||
void prepare4(std::string & filename, std::string & buf);
|
||||
std::string createTmpFile();
|
||||
void die(const std::string & msg);
|
||||
void runTest(unsigned int num, const std::function<bool()> func);
|
||||
void runTest(unsigned int num, const std::function<bool()> & func);
|
||||
|
||||
bool test1(const std::string & filename);
|
||||
bool test2(const std::string & filename, const std::string & buf);
|
||||
@ -184,7 +184,7 @@ void die(const std::string & msg)
|
||||
::exit(EXIT_FAILURE);
|
||||
}
|
||||
|
||||
void runTest(unsigned int num, const std::function<bool()> func)
|
||||
void runTest(unsigned int num, const std::function<bool()> & func)
|
||||
{
|
||||
bool ok;
|
||||
|
||||
|
@ -15,7 +15,7 @@ namespace fs = boost::filesystem;
|
||||
|
||||
void run();
|
||||
void die(const std::string & msg);
|
||||
void runTest(unsigned int num, const std::function<bool()> func);
|
||||
void runTest(unsigned int num, const std::function<bool()> & func);
|
||||
std::string createTmpFile();
|
||||
std::string generateString(size_t n);
|
||||
|
||||
@ -60,7 +60,7 @@ void die(const std::string & msg)
|
||||
::exit(EXIT_FAILURE);
|
||||
}
|
||||
|
||||
void runTest(unsigned int num, const std::function<bool()> func)
|
||||
void runTest(unsigned int num, const std::function<bool()> & func)
|
||||
{
|
||||
bool ok;
|
||||
|
||||
|
@ -1734,7 +1734,7 @@ private:
|
||||
std::mutex mutex;
|
||||
std::condition_variable condvar;
|
||||
|
||||
ParallelMergeData(size_t threads) : pool(threads) {}
|
||||
explicit ParallelMergeData(size_t threads) : pool(threads) {}
|
||||
|
||||
~ParallelMergeData()
|
||||
{
|
||||
|
@ -100,7 +100,7 @@ struct ContextShared
|
||||
mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper.
|
||||
|
||||
String interserver_io_host; /// The host name by which this server is available for other servers.
|
||||
int interserver_io_port; /// and port,
|
||||
UInt16 interserver_io_port = 0; /// and port.
|
||||
|
||||
String path; /// Path to the data directory, with a slash at the end.
|
||||
String tmp_path; /// The path to the temporary files that occur when processing the request.
|
||||
|
@ -62,7 +62,7 @@ struct HostID
|
||||
|
||||
HostID() = default;
|
||||
|
||||
HostID(const Cluster::Address & address)
|
||||
explicit HostID(const Cluster::Address & address)
|
||||
: host_name(address.host_name), port(address.port) {}
|
||||
|
||||
static HostID fromString(const String & host_port_str)
|
||||
|
@ -320,19 +320,16 @@ void ExternalDictionaries::reloadFromConfigFile(const std::string & config_path,
|
||||
{
|
||||
failed_dict_it->second = FailedDictionaryInfo{
|
||||
std::move(dict_ptr),
|
||||
std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}
|
||||
};
|
||||
std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}};
|
||||
}
|
||||
else
|
||||
failed_dictionaries.emplace(name, FailedDictionaryInfo{
|
||||
std::move(dict_ptr),
|
||||
std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}
|
||||
});
|
||||
std::chrono::system_clock::now() + std::chrono::seconds{backoff_initial_sec}});
|
||||
|
||||
std::rethrow_exception(exception_ptr);
|
||||
}
|
||||
|
||||
if (!dict_ptr->isCached())
|
||||
else if (!dict_ptr->isCached())
|
||||
{
|
||||
const auto & lifetime = dict_ptr->getLifetime();
|
||||
if (lifetime.min_sec != 0 && lifetime.max_sec != 0)
|
||||
|
@ -245,10 +245,10 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr
|
||||
|
||||
BlockInputStreamPtr input_stream;
|
||||
BlockOutputStreamPtr output_stream;
|
||||
IProfilingBlockInputStream * input_stream_casted;
|
||||
|
||||
if (elem->tryGetQueryStreams(input_stream, output_stream))
|
||||
{
|
||||
IProfilingBlockInputStream * input_stream_casted;
|
||||
if (input_stream && (input_stream_casted = dynamic_cast<IProfilingBlockInputStream *>(input_stream.get())))
|
||||
{
|
||||
input_stream_casted->cancel();
|
||||
|
@ -162,7 +162,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
HostExactPattern(const String & host_) : host(host_) {}
|
||||
explicit HostExactPattern(const String & host_) : host(host_) {}
|
||||
|
||||
bool contains(const Poco::Net::IPAddress & addr) const override
|
||||
{
|
||||
@ -192,7 +192,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {}
|
||||
explicit HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {}
|
||||
|
||||
bool contains(const Poco::Net::IPAddress & addr) const override
|
||||
{
|
||||
|
@ -50,7 +50,7 @@ struct PartialSortingLess
|
||||
{
|
||||
const ColumnsWithSortDescriptions & columns;
|
||||
|
||||
PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {}
|
||||
explicit PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {}
|
||||
|
||||
bool operator() (size_t a, size_t b) const
|
||||
{
|
||||
@ -70,7 +70,7 @@ struct PartialSortingLessWithCollation
|
||||
{
|
||||
const ColumnsWithSortDescriptions & columns;
|
||||
|
||||
PartialSortingLessWithCollation(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {}
|
||||
explicit PartialSortingLessWithCollation(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {}
|
||||
|
||||
bool operator() (size_t a, size_t b) const
|
||||
{
|
||||
|
@ -40,7 +40,7 @@ struct CompactStringRef
|
||||
}
|
||||
|
||||
CompactStringRef(const unsigned char * data_, size_t size_) : CompactStringRef(reinterpret_cast<const char *>(data_), size_) {}
|
||||
CompactStringRef(const std::string & s) : CompactStringRef(s.data(), s.size()) {}
|
||||
explicit CompactStringRef(const std::string & s) : CompactStringRef(s.data(), s.size()) {}
|
||||
CompactStringRef() {}
|
||||
|
||||
const char * data() const { return reinterpret_cast<const char *>(reinterpret_cast<intptr_t>(data_mixed) & 0x0000FFFFFFFFFFFFULL); }
|
||||
|
@ -49,7 +49,7 @@ struct SmallStringRef
|
||||
}
|
||||
|
||||
SmallStringRef(const unsigned char * data_, size_t size_) : SmallStringRef(reinterpret_cast<const char *>(data_), size_) {}
|
||||
SmallStringRef(const std::string & s) : SmallStringRef(s.data(), s.size()) {}
|
||||
explicit SmallStringRef(const std::string & s) : SmallStringRef(s.data(), s.size()) {}
|
||||
SmallStringRef() {}
|
||||
|
||||
std::string toString() const { return std::string(data(), size); }
|
||||
|
@ -82,7 +82,7 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n
|
||||
struct Arrow
|
||||
{
|
||||
Arrow() = default;
|
||||
Arrow(size_t to_position_) :
|
||||
explicit Arrow(size_t to_position_) :
|
||||
to_position(to_position_), is_selected(true)
|
||||
{
|
||||
}
|
||||
|
@ -26,10 +26,8 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return ParserFunction{}.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
bool has_case_expr = false;
|
||||
|
||||
auto old_pos = pos;
|
||||
has_case_expr = !s_when.parse(pos, node, expected);
|
||||
bool has_case_expr = !s_when.parse(pos, node, expected);
|
||||
pos = old_pos;
|
||||
|
||||
ASTs args;
|
||||
|
@ -19,7 +19,7 @@ namespace DB
|
||||
class InterserverIOHTTPHandler : public Poco::Net::HTTPRequestHandler
|
||||
{
|
||||
public:
|
||||
InterserverIOHTTPHandler(IServer & server_)
|
||||
explicit InterserverIOHTTPHandler(IServer & server_)
|
||||
: server(server_)
|
||||
, log(&Poco::Logger::get("InterserverIOHTTPHandler"))
|
||||
{
|
||||
|
@ -58,6 +58,8 @@ static String pad(size_t padding)
|
||||
return String(padding * 4, ' ');
|
||||
}
|
||||
|
||||
|
||||
/// NOTE The code is totally wrong.
|
||||
class JSONString
|
||||
{
|
||||
private:
|
||||
@ -65,7 +67,7 @@ private:
|
||||
size_t padding;
|
||||
|
||||
public:
|
||||
JSONString(size_t padding_ = 1) : padding(padding_){};
|
||||
explicit JSONString(size_t padding_ = 1) : padding(padding_){};
|
||||
|
||||
void set(const String key, String value, bool wrap = true)
|
||||
{
|
||||
@ -1041,16 +1043,13 @@ private:
|
||||
TestStopConditions & stop_conditions = stop_conditions_by_run[run_index];
|
||||
Stats & statistics = statistics_by_run[run_index];
|
||||
|
||||
size_t iteration = 0;
|
||||
|
||||
statistics.clear();
|
||||
execute(query, statistics, stop_conditions);
|
||||
|
||||
if (exec_type == ExecutionType::Loop)
|
||||
{
|
||||
while (!gotSIGINT)
|
||||
for (size_t iteration = 1; !gotSIGINT; ++iteration)
|
||||
{
|
||||
++iteration;
|
||||
stop_conditions.reportIterations(iteration);
|
||||
if (stop_conditions.areFulfilled())
|
||||
break;
|
||||
|
@ -15,7 +15,7 @@ private:
|
||||
Context & context;
|
||||
|
||||
public:
|
||||
ReplicasStatusHandler(Context & context_);
|
||||
explicit ReplicasStatusHandler(Context & context_);
|
||||
|
||||
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override;
|
||||
};
|
||||
|
@ -15,7 +15,7 @@ private:
|
||||
IServer & server;
|
||||
|
||||
public:
|
||||
RootRequestHandler(IServer & server_) : server(server_)
|
||||
explicit RootRequestHandler(IServer & server_) : server(server_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
class StatusFile : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
StatusFile(const std::string & path_);
|
||||
explicit StatusFile(const std::string & path_);
|
||||
~StatusFile();
|
||||
|
||||
private:
|
||||
|
@ -17,7 +17,7 @@ private:
|
||||
Poco::Logger * log;
|
||||
|
||||
public:
|
||||
TCPHandlerFactory(IServer & server_)
|
||||
explicit TCPHandlerFactory(IServer & server_)
|
||||
: server(server_)
|
||||
, log(&Logger::get("TCPHandlerFactory"))
|
||||
{
|
||||
|
@ -72,7 +72,6 @@ protected:
|
||||
MergeTreeReaderPtr reader;
|
||||
MergeTreeReaderPtr pre_reader;
|
||||
|
||||
Logger * log;
|
||||
size_t max_block_size_marks;
|
||||
};
|
||||
|
||||
|
@ -45,8 +45,6 @@ MergeTreeBlockInputStream::MergeTreeBlockInputStream(
|
||||
check_columns(check_columns),
|
||||
path(data_part->getFullPath())
|
||||
{
|
||||
log = &Logger::get("MergeTreeBlockInputStream");
|
||||
|
||||
/// Let's estimate total number of rows for progress bar.
|
||||
size_t total_rows = 0;
|
||||
for (const auto & range : all_mark_ranges)
|
||||
|
@ -68,6 +68,8 @@ private:
|
||||
bool check_columns;
|
||||
String path;
|
||||
bool is_first_task = true;
|
||||
|
||||
Logger * log = &Logger::get("MergeTreeBlockInputStream");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -36,8 +36,6 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream(
|
||||
}
|
||||
else
|
||||
min_marks_to_read = min_marks_to_read_;
|
||||
|
||||
log = &Logger::get("MergeTreeThreadBlockInputStream");
|
||||
}
|
||||
|
||||
|
||||
|
@ -837,9 +837,9 @@ bool PKCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, co
|
||||
|
||||
/// The case when the column is wrapped in a chain of possibly monotonic functions.
|
||||
Range key_range_transformed;
|
||||
bool evaluation_is_not_possible = false;
|
||||
if (!element.monotonic_functions_chain.empty())
|
||||
{
|
||||
bool evaluation_is_not_possible = false;
|
||||
key_range_transformed = *key_range;
|
||||
DataTypePtr current_type = data_types[element.key_column];
|
||||
for (auto & func : element.monotonic_functions_chain)
|
||||
|
@ -115,7 +115,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
Status(const std::string & str)
|
||||
explicit Status(const std::string & str)
|
||||
{
|
||||
size_t pos = str.find(',');
|
||||
code = static_cast<ReshardingWorker::StatusCode>(std::stoull(str.substr(0, pos)));
|
||||
@ -2094,8 +2094,6 @@ ReshardingWorker::StatusCode ReshardingWorker::getStatusCommon(const std::string
|
||||
|
||||
std::string ReshardingWorker::dumpCoordinatorState(const std::string & coordinator_id)
|
||||
{
|
||||
std::string out;
|
||||
|
||||
auto current_host = getFQDNOrHostName();
|
||||
|
||||
WriteBufferFromOwnString buf;
|
||||
|
@ -101,7 +101,7 @@ bool allow(
|
||||
// std::cerr << "sum_size: " << sum_size << "\n";
|
||||
|
||||
/// Map size to 0..1 using logarithmic scale
|
||||
double size_normalized = mapPiecewiseLinearToUnit(log(1 + sum_size), log(1 + settings.min_size_to_lower_base), log(1 + settings.max_size_to_lower_base));
|
||||
double size_normalized = mapPiecewiseLinearToUnit(log1p(sum_size), log1p(settings.min_size_to_lower_base), log1p(settings.max_size_to_lower_base));
|
||||
|
||||
// std::cerr << "size_normalized: " << size_normalized << "\n";
|
||||
|
||||
|
@ -210,7 +210,7 @@ static void appendBlock(const Block & from, Block & to)
|
||||
class BufferBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
BufferBlockOutputStream(StorageBuffer & storage_) : storage(storage_) {}
|
||||
explicit BufferBlockOutputStream(StorageBuffer & storage_) : storage(storage_) {}
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
@ -440,67 +440,65 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds)
|
||||
size_t bytes = 0;
|
||||
time_t time_passed = 0;
|
||||
|
||||
std::lock_guard<std::mutex> lock(buffer.mutex);
|
||||
|
||||
block_to_write = buffer.data.cloneEmpty();
|
||||
|
||||
rows = buffer.data.rows();
|
||||
bytes = buffer.data.bytes();
|
||||
if (buffer.first_write_time)
|
||||
time_passed = current_time - buffer.first_write_time;
|
||||
|
||||
if (check_thresholds)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(buffer.mutex);
|
||||
if (!checkThresholdsImpl(rows, bytes, time_passed))
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (rows == 0)
|
||||
return;
|
||||
}
|
||||
|
||||
block_to_write = buffer.data.cloneEmpty();
|
||||
buffer.data.swap(block_to_write);
|
||||
buffer.first_write_time = 0;
|
||||
|
||||
rows = buffer.data.rows();
|
||||
bytes = buffer.data.bytes();
|
||||
if (buffer.first_write_time)
|
||||
time_passed = current_time - buffer.first_write_time;
|
||||
CurrentMetrics::sub(CurrentMetrics::StorageBufferRows, block_to_write.rows());
|
||||
CurrentMetrics::sub(CurrentMetrics::StorageBufferBytes, block_to_write.bytes());
|
||||
|
||||
if (check_thresholds)
|
||||
{
|
||||
if (!checkThresholdsImpl(rows, bytes, time_passed))
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (rows == 0)
|
||||
return;
|
||||
}
|
||||
ProfileEvents::increment(ProfileEvents::StorageBufferFlush);
|
||||
|
||||
LOG_TRACE(log, "Flushing buffer with " << rows << " rows, " << bytes << " bytes, age " << time_passed << " seconds.");
|
||||
|
||||
if (no_destination)
|
||||
return;
|
||||
|
||||
/** For simplicity, buffer is locked during write.
|
||||
* We could unlock buffer temporary, but it would lead to too much difficulties:
|
||||
* - data, that is written, will not be visible for SELECTs;
|
||||
* - new data could be appended to buffer, and in case of exception, we must merge it with old data, that has not been written;
|
||||
* - this could lead to infinite memory growth.
|
||||
*/
|
||||
try
|
||||
{
|
||||
writeBlockToDestination(block_to_write, context.tryGetTable(destination_database, destination_table));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::StorageBufferErrorOnFlush);
|
||||
|
||||
/// Return the block to its place in the buffer.
|
||||
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferRows, block_to_write.rows());
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, block_to_write.bytes());
|
||||
|
||||
buffer.data.swap(block_to_write);
|
||||
buffer.first_write_time = 0;
|
||||
|
||||
CurrentMetrics::sub(CurrentMetrics::StorageBufferRows, block_to_write.rows());
|
||||
CurrentMetrics::sub(CurrentMetrics::StorageBufferBytes, block_to_write.bytes());
|
||||
if (!buffer.first_write_time)
|
||||
buffer.first_write_time = current_time;
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::StorageBufferFlush);
|
||||
|
||||
LOG_TRACE(log, "Flushing buffer with " << rows << " rows, " << bytes << " bytes, age " << time_passed << " seconds.");
|
||||
|
||||
if (no_destination)
|
||||
return;
|
||||
|
||||
/** For simplicity, buffer is locked during write.
|
||||
* We could unlock buffer temporary, but it would lead to too much difficulties:
|
||||
* - data, that is written, will not be visible for SELECTs;
|
||||
* - new data could be appended to buffer, and in case of exception, we must merge it with old data, that has not been written;
|
||||
* - this could lead to infinite memory growth.
|
||||
*/
|
||||
try
|
||||
{
|
||||
writeBlockToDestination(block_to_write, context.tryGetTable(destination_database, destination_table));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::StorageBufferErrorOnFlush);
|
||||
|
||||
/// Return the block to its place in the buffer.
|
||||
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferRows, block_to_write.rows());
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, block_to_write.bytes());
|
||||
|
||||
buffer.data.swap(block_to_write);
|
||||
|
||||
if (!buffer.first_write_time)
|
||||
buffer.first_write_time = current_time;
|
||||
|
||||
/// After a while, the next write attempt will happen.
|
||||
throw;
|
||||
}
|
||||
/// After a while, the next write attempt will happen.
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -26,6 +26,8 @@ StorageCloud::StorageCloud(
|
||||
throw Exception("DatabaseCloud is detached", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
DatabaseCloud & db = static_cast<DatabaseCloud &>(*owned_db);
|
||||
|
||||
/// This code is unfinished.
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -180,7 +180,7 @@ class StorageFileBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
|
||||
StorageFileBlockOutputStream(StorageFile & storage_)
|
||||
explicit StorageFileBlockOutputStream(StorageFile & storage_)
|
||||
: storage(storage_), lock(storage.rwlock)
|
||||
{
|
||||
if (storage.use_table_fd)
|
||||
|
@ -133,7 +133,7 @@ private:
|
||||
class LogBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
LogBlockOutputStream(StorageLog & storage_)
|
||||
explicit LogBlockOutputStream(StorageLog & storage_)
|
||||
: storage(storage_),
|
||||
lock(storage.rwlock),
|
||||
marks_stream(storage.marks_file.path(), 4096, O_APPEND | O_CREAT | O_WRONLY),
|
||||
|
@ -61,7 +61,7 @@ private:
|
||||
class MemoryBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {}
|
||||
explicit MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {}
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
|
@ -380,7 +380,7 @@ namespace
|
||||
{
|
||||
const MergeTreeData & data;
|
||||
|
||||
TableMetadata(const MergeTreeData & data_)
|
||||
explicit TableMetadata(const MergeTreeData & data_)
|
||||
: data(data_) {}
|
||||
|
||||
void write(WriteBuffer & out) const
|
||||
|
@ -114,7 +114,7 @@ private:
|
||||
class StripeLogBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
StripeLogBlockOutputStream(StorageStripeLog & storage_)
|
||||
explicit StripeLogBlockOutputStream(StorageStripeLog & storage_)
|
||||
: storage(storage_), lock(storage.rwlock),
|
||||
data_out_compressed(storage.full_path() + "data.bin", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT),
|
||||
data_out(data_out_compressed, CompressionMethod::LZ4, storage.max_compress_block_size),
|
||||
|
@ -96,7 +96,7 @@ private:
|
||||
class TinyLogBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
TinyLogBlockOutputStream(StorageTinyLog & storage_)
|
||||
explicit TinyLogBlockOutputStream(StorageTinyLog & storage_)
|
||||
: storage(storage_)
|
||||
{
|
||||
for (const auto & col : storage.getColumnsList())
|
||||
|
@ -279,7 +279,7 @@ void StorageTrivialBuffer::flush(bool check_thresholds, bool is_called_from_back
|
||||
class TrivialBufferBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
TrivialBufferBlockOutputStream(StorageTrivialBuffer & buffer_) : buffer(buffer_) {}
|
||||
explicit TrivialBufferBlockOutputStream(StorageTrivialBuffer & buffer_) : buffer(buffer_) {}
|
||||
void write(const Block & block) override
|
||||
{
|
||||
if (!block)
|
||||
|
@ -23,7 +23,10 @@ void * ClickHouseDictionary_v1_loadIds(
|
||||
void * data_ptr, ClickHouseLibrary::CStrings * settings, ClickHouseLibrary::CStrings * columns, const struct ClickHouseLibrary::VectorUInt64 * ids)
|
||||
{
|
||||
auto ptr = static_cast<DataHolder *>(data_ptr);
|
||||
std::cerr << "loadIds lib call ptr=" << data_ptr << " => " << ptr << " size=" << ids->size << "\n";
|
||||
|
||||
if (ids)
|
||||
std::cerr << "loadIds lib call ptr=" << data_ptr << " => " << ptr << " size=" << ids->size << "\n";
|
||||
|
||||
if (settings)
|
||||
{
|
||||
std::cerr << "settings passed: " << settings->size << "\n";
|
||||
|
@ -223,9 +223,9 @@ public:
|
||||
StopThread = -2
|
||||
};
|
||||
|
||||
SignalListener(BaseDaemon & daemon_)
|
||||
: log(&Logger::get("BaseDaemon"))
|
||||
, daemon(daemon_)
|
||||
explicit SignalListener(BaseDaemon & daemon_)
|
||||
: log(&Logger::get("BaseDaemon"))
|
||||
, daemon(daemon_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -400,7 +400,7 @@ static void terminate_handler()
|
||||
if (terminating)
|
||||
{
|
||||
abort();
|
||||
return;
|
||||
return; /// Just for convenience.
|
||||
}
|
||||
|
||||
terminating = true;
|
||||
|
@ -85,8 +85,8 @@ Pool::~Pool()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
for (Connections::iterator it = connections.begin(); it != connections.end(); it++)
|
||||
delete static_cast<Connection *>(*it);
|
||||
for (auto & connection : connections)
|
||||
delete static_cast<Connection *>(connection);
|
||||
}
|
||||
|
||||
|
||||
@ -97,10 +97,10 @@ Pool::Entry Pool::Get()
|
||||
initialize();
|
||||
for (;;)
|
||||
{
|
||||
for (Connections::iterator it = connections.begin(); it != connections.end(); it++)
|
||||
for (auto & connection : connections)
|
||||
{
|
||||
if ((*it)->ref_count == 0)
|
||||
return Entry(*it, this);
|
||||
if (connection->ref_count == 0)
|
||||
return Entry(connection, this);
|
||||
}
|
||||
|
||||
if (connections.size() < static_cast<size_t>(max_connections))
|
||||
@ -124,11 +124,11 @@ Pool::Entry Pool::tryGet()
|
||||
initialize();
|
||||
|
||||
/// Searching for connection which was established but wasn't used.
|
||||
for (Connections::iterator it = connections.begin(); it != connections.end(); ++it)
|
||||
for (auto & connection : connections)
|
||||
{
|
||||
if ((*it)->ref_count == 0)
|
||||
if (connection->ref_count == 0)
|
||||
{
|
||||
Entry res(*it, this);
|
||||
Entry res(connection, this);
|
||||
return res.tryForceConnected() ? res : Entry();
|
||||
}
|
||||
}
|
||||
|
@ -17,6 +17,8 @@
|
||||
#include <common/ThreadPool.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
|
||||
#include <cstdlib>
|
||||
|
||||
#ifdef __APPLE__
|
||||
@ -37,29 +39,9 @@ enum Mode
|
||||
};
|
||||
|
||||
|
||||
struct AlignedBuffer
|
||||
{
|
||||
int size;
|
||||
char * data;
|
||||
|
||||
AlignedBuffer(int size_)
|
||||
{
|
||||
size_t page = sysconf(_SC_PAGESIZE);
|
||||
size = size_;
|
||||
int rc = posix_memalign(reinterpret_cast<void **>(&data), page, (size + page - 1) / page * page);
|
||||
if (data == nullptr || rc != 0)
|
||||
throwFromErrno("memalign failed");
|
||||
}
|
||||
|
||||
~AlignedBuffer()
|
||||
{
|
||||
free(data);
|
||||
}
|
||||
};
|
||||
|
||||
void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t count)
|
||||
{
|
||||
AlignedBuffer direct_buf(block_size);
|
||||
DB::Memory direct_buf(block_size, sysconf(_SC_PAGESIZE));
|
||||
std::vector<char> simple_buf(block_size);
|
||||
|
||||
char * buf;
|
||||
|
@ -16,6 +16,8 @@
|
||||
#include <common/ThreadPool.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
|
||||
#include <stdlib.h>
|
||||
#if !defined(__APPLE__) && !defined(__FreeBSD__)
|
||||
#include <malloc.h>
|
||||
@ -61,42 +63,6 @@ enum Mode
|
||||
};
|
||||
|
||||
|
||||
struct AlignedBuffer
|
||||
{
|
||||
int size = 0;
|
||||
char * data = nullptr;
|
||||
|
||||
AlignedBuffer() {}
|
||||
|
||||
void init(int size_)
|
||||
{
|
||||
uninit();
|
||||
size_t page = sysconf(_SC_PAGESIZE);
|
||||
size = size_;
|
||||
data = static_cast<char*>(memalign(page, (size + page - 1) / page * page));
|
||||
if (!data)
|
||||
throwFromErrno("memalign failed");
|
||||
}
|
||||
|
||||
void uninit()
|
||||
{
|
||||
if (data)
|
||||
free(data);
|
||||
data = nullptr;
|
||||
size = 0;
|
||||
}
|
||||
|
||||
AlignedBuffer(int size_) : size(0), data(NULL)
|
||||
{
|
||||
init(size_);
|
||||
}
|
||||
|
||||
~AlignedBuffer()
|
||||
{
|
||||
uninit();
|
||||
}
|
||||
};
|
||||
|
||||
struct AioContext
|
||||
{
|
||||
aio_context_t ctx;
|
||||
@ -119,11 +85,9 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
|
||||
{
|
||||
AioContext ctx;
|
||||
|
||||
std::vector<AlignedBuffer> buffers(buffers_count);
|
||||
std::vector<DB::Memory> buffers(buffers_count);
|
||||
for (size_t i = 0; i < buffers_count; ++i)
|
||||
{
|
||||
buffers[i].init(block_size);
|
||||
}
|
||||
buffers[i] = DB::Memory(block_size, sysconf(_SC_PAGESIZE));
|
||||
|
||||
drand48_data rand_data;
|
||||
timespec times;
|
||||
|
@ -74,7 +74,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
MarkovModel(size_t n_) : n(n_) {}
|
||||
explicit MarkovModel(size_t n_) : n(n_) {}
|
||||
MarkovModel() {}
|
||||
|
||||
void consume(const char * data, size_t size)
|
||||
|
Loading…
Reference in New Issue
Block a user