mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Fix clang-tidy warnings in Interpreters, IO folders
This commit is contained in:
parent
538f8cbaad
commit
e30117a3d6
@ -26,17 +26,17 @@ int io_setup(unsigned nr, aio_context_t * ctxp);
|
||||
int io_destroy(aio_context_t ctx);
|
||||
|
||||
/// last argument is an array of pointers technically speaking
|
||||
int io_submit(aio_context_t ctx, long nr, struct iocb * iocbpp[]);
|
||||
int io_submit(aio_context_t ctx, long nr, struct iocb * iocbpp[]); /// NOLINT
|
||||
|
||||
int io_getevents(aio_context_t ctx, long min_nr, long max_nr, io_event * events, struct timespec * timeout);
|
||||
int io_getevents(aio_context_t ctx, long min_nr, long max_nr, io_event * events, struct timespec * timeout); /// NOLINT
|
||||
|
||||
|
||||
struct AIOContext : private boost::noncopyable
|
||||
{
|
||||
aio_context_t ctx = 0;
|
||||
|
||||
AIOContext() {}
|
||||
AIOContext(unsigned int nr_events);
|
||||
AIOContext() = default;
|
||||
explicit AIOContext(unsigned int nr_events);
|
||||
~AIOContext();
|
||||
AIOContext(AIOContext && rhs) noexcept;
|
||||
AIOContext & operator=(AIOContext && rhs) noexcept;
|
||||
|
@ -29,7 +29,7 @@ public:
|
||||
|
||||
/// Sets compression method and level.
|
||||
/// Changing them will affect next file in the archive.
|
||||
virtual void setCompression(int /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) {}
|
||||
virtual void setCompression(int /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) {} /// NOLINT
|
||||
|
||||
/// Sets password. If the password is not empty it will enable encryption in the archive.
|
||||
virtual void setPassword(const String & /* password */) {}
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
|
||||
struct LocalFileDescriptor : public IFileDescriptor
|
||||
{
|
||||
LocalFileDescriptor(int fd_) : fd(fd_) {}
|
||||
explicit LocalFileDescriptor(int fd_) : fd(fd_) {}
|
||||
int fd;
|
||||
};
|
||||
|
||||
|
@ -52,8 +52,7 @@ public:
|
||||
bits_count(0)
|
||||
{}
|
||||
|
||||
~BitReader()
|
||||
{}
|
||||
~BitReader() = default;
|
||||
|
||||
// reads bits_to_read high-bits from bits_buffer
|
||||
inline UInt64 readBits(UInt8 bits_to_read)
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
class BrotliReadBuffer : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
BrotliReadBuffer(
|
||||
explicit BrotliReadBuffer(
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
class Bzip2ReadBuffer : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
Bzip2ReadBuffer(
|
||||
explicit Bzip2ReadBuffer(
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
using WriteBufferConstructor = std::function<WriteBufferPtr (const WriteBufferPtr & prev_buf)>;
|
||||
using WriteBufferConstructors = std::vector<WriteBufferConstructor>;
|
||||
|
||||
CascadeWriteBuffer(WriteBufferPtrs && prepared_sources_, WriteBufferConstructors && lazy_sources_ = {});
|
||||
explicit CascadeWriteBuffer(WriteBufferPtrs && prepared_sources_, WriteBufferConstructors && lazy_sources_ = {});
|
||||
|
||||
void nextImpl() override;
|
||||
|
||||
|
@ -29,9 +29,6 @@ template <> struct DoubleToStringConverterFlags<true>
|
||||
template <bool emit_decimal_point>
|
||||
class DoubleConverter : private boost::noncopyable
|
||||
{
|
||||
DoubleConverter(const DoubleConverter &) = delete;
|
||||
DoubleConverter & operator=(const DoubleConverter &) = delete;
|
||||
|
||||
DoubleConverter() = default;
|
||||
|
||||
public:
|
||||
|
@ -56,7 +56,7 @@ public:
|
||||
|
||||
/// Adds a specified offset to the counter.
|
||||
InitVector & operator++() { ++counter; return *this; }
|
||||
InitVector operator++(int) { InitVector res = *this; ++counter; return res; }
|
||||
InitVector operator++(int) { InitVector res = *this; ++counter; return res; } /// NOLINT
|
||||
InitVector & operator+=(size_t offset) { counter += offset; return *this; }
|
||||
InitVector operator+(size_t offset) const { InitVector res = *this; return res += offset; }
|
||||
|
||||
|
@ -17,7 +17,7 @@ class IHashingBuffer : public BufferWithOwnMemory<Buffer>
|
||||
public:
|
||||
using uint128 = CityHash_v1_0_2::uint128;
|
||||
|
||||
IHashingBuffer(size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE)
|
||||
explicit IHashingBuffer(size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE)
|
||||
: BufferWithOwnMemory<Buffer>(block_size_), block_pos(0), block_size(block_size_), state(0, 0)
|
||||
{
|
||||
}
|
||||
@ -66,7 +66,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
HashingWriteBuffer(
|
||||
explicit HashingWriteBuffer(
|
||||
WriteBuffer & out_,
|
||||
size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE)
|
||||
: IHashingBuffer<DB::WriteBuffer>(block_size_), out(out_)
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
class LZMAInflatingReadBuffer : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
LZMAInflatingReadBuffer(
|
||||
explicit LZMAInflatingReadBuffer(
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
|
@ -29,7 +29,7 @@ private:
|
||||
void finalizeBefore() override;
|
||||
void finalizeAfter() override;
|
||||
|
||||
LZ4F_preferences_t kPrefs;
|
||||
LZ4F_preferences_t kPrefs; /// NOLINT
|
||||
LZ4F_compressionContext_t ctx;
|
||||
|
||||
void * in_data;
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
class Lz4InflatingReadBuffer : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
Lz4InflatingReadBuffer(
|
||||
explicit Lz4InflatingReadBuffer(
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
||||
protected:
|
||||
MMapReadBufferFromFileDescriptor() {}
|
||||
MMapReadBufferFromFileDescriptor() = default;
|
||||
void init();
|
||||
|
||||
MMappedFileDescriptor mapped;
|
||||
|
@ -27,7 +27,7 @@ private:
|
||||
using Base = LRUCache<UInt128, MMappedFile, UInt128TrivialHash>;
|
||||
|
||||
public:
|
||||
MMappedFileCache(size_t max_size_in_bytes)
|
||||
explicit MMappedFileCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
|
||||
/// Calculate key from path to file and offset.
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
MMappedFileDescriptor(int fd_, size_t offset_);
|
||||
|
||||
/// Makes empty object that can be initialized with `set`.
|
||||
MMappedFileDescriptor() {}
|
||||
MMappedFileDescriptor() = default;
|
||||
|
||||
virtual ~MMappedFileDescriptor();
|
||||
|
||||
@ -40,10 +40,11 @@ public:
|
||||
void set(int fd_, size_t offset_, size_t length_);
|
||||
void set(int fd_, size_t offset_);
|
||||
|
||||
protected:
|
||||
MMappedFileDescriptor(const MMappedFileDescriptor &) = delete;
|
||||
MMappedFileDescriptor(MMappedFileDescriptor &&) = delete;
|
||||
|
||||
protected:
|
||||
|
||||
void init();
|
||||
|
||||
int fd = -1;
|
||||
|
@ -18,7 +18,7 @@ class MemoryWriteBuffer : public WriteBuffer, public IReadableWriteBuffer, boost
|
||||
public:
|
||||
|
||||
/// Use max_total_size_ = 0 for unlimited storage
|
||||
MemoryWriteBuffer(
|
||||
explicit MemoryWriteBuffer(
|
||||
size_t max_total_size_ = 0,
|
||||
size_t initial_chunk_size_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
double growth_rate_ = 2.0,
|
||||
|
@ -13,7 +13,7 @@ class MySQLPacketPayloadWriteBuffer : public WriteBuffer
|
||||
public:
|
||||
MySQLPacketPayloadWriteBuffer(WriteBuffer & out_, size_t payload_length_, uint8_t & sequence_id_);
|
||||
|
||||
bool remainingPayloadSize() { return total_left; }
|
||||
bool remainingPayloadSize() const { return total_left; }
|
||||
|
||||
protected:
|
||||
void nextImpl() override;
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
class NullWriteBuffer : public BufferWithOwnMemory<WriteBuffer>, boost::noncopyable
|
||||
{
|
||||
public:
|
||||
NullWriteBuffer(size_t buf_size = 16<<10, char * existing_memory = nullptr, size_t alignment = false);
|
||||
explicit NullWriteBuffer(size_t buf_size = 16<<10, char * existing_memory = nullptr, size_t alignment = false);
|
||||
void nextImpl() override;
|
||||
};
|
||||
|
||||
|
@ -56,7 +56,7 @@ struct FileProgress
|
||||
size_t read_bytes;
|
||||
size_t total_bytes_to_read;
|
||||
|
||||
FileProgress(size_t read_bytes_, size_t total_bytes_to_read_ = 0) : read_bytes(read_bytes_), total_bytes_to_read(total_bytes_to_read_) {}
|
||||
explicit FileProgress(size_t read_bytes_, size_t total_bytes_to_read_ = 0) : read_bytes(read_bytes_), total_bytes_to_read(total_bytes_to_read_) {}
|
||||
};
|
||||
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -61,7 +61,7 @@ public:
|
||||
class ReadBufferFromFilePRead : public ReadBufferFromFile
|
||||
{
|
||||
public:
|
||||
ReadBufferFromFilePRead(
|
||||
explicit ReadBufferFromFilePRead(
|
||||
const std::string & file_name_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
int flags = -1,
|
||||
@ -84,7 +84,7 @@ private:
|
||||
OpenedFileCache::OpenedFilePtr file;
|
||||
|
||||
public:
|
||||
ReadBufferFromFilePReadWithDescriptorsCache(
|
||||
explicit ReadBufferFromFilePReadWithDescriptorsCache(
|
||||
const std::string & file_name_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
int flags = -1,
|
||||
|
@ -27,7 +27,7 @@ protected:
|
||||
std::string getFileName() const override;
|
||||
|
||||
public:
|
||||
ReadBufferFromFileDescriptor(
|
||||
explicit ReadBufferFromFileDescriptor(
|
||||
int fd_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
@ -70,7 +70,7 @@ private:
|
||||
class ReadBufferFromFileDescriptorPRead : public ReadBufferFromFileDescriptor
|
||||
{
|
||||
public:
|
||||
ReadBufferFromFileDescriptorPRead(
|
||||
explicit ReadBufferFromFileDescriptorPRead(
|
||||
int fd_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
|
@ -106,7 +106,7 @@ inline void readChar(char & x, ReadBuffer & buf)
|
||||
template <typename T>
|
||||
inline void readPODBinary(T & x, ReadBuffer & buf)
|
||||
{
|
||||
buf.readStrict(reinterpret_cast<char *>(&x), sizeof(x));
|
||||
buf.readStrict(reinterpret_cast<char *>(&x), sizeof(x)); /// NOLINT
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -611,7 +611,7 @@ void readStringUntilNewlineInto(Vector & s, ReadBuffer & buf);
|
||||
struct NullOutput
|
||||
{
|
||||
void append(const char *, size_t) {}
|
||||
void push_back(char) {}
|
||||
void push_back(char) {} /// NOLINT
|
||||
};
|
||||
|
||||
void parseUUID(const UInt8 * src36, UInt8 * dst16);
|
||||
@ -1278,7 +1278,6 @@ inline void readTextWithSizeSuffix(T & x, ReadBuffer & buf)
|
||||
default:
|
||||
return;
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
/// Read something from text format and trying to parse the suffix.
|
||||
|
@ -49,13 +49,13 @@ class PocoHTTPResponse : public Aws::Http::Standard::StandardHttpResponse
|
||||
public:
|
||||
using SessionPtr = HTTPSessionPtr;
|
||||
|
||||
PocoHTTPResponse(const std::shared_ptr<const Aws::Http::HttpRequest> request)
|
||||
explicit PocoHTTPResponse(const std::shared_ptr<const Aws::Http::HttpRequest> request)
|
||||
: Aws::Http::Standard::StandardHttpResponse(request)
|
||||
, body_stream(request->GetResponseStreamFactory())
|
||||
{
|
||||
}
|
||||
|
||||
void SetResponseBody(Aws::IStream & incoming_stream, SessionPtr & session_)
|
||||
void SetResponseBody(Aws::IStream & incoming_stream, SessionPtr & session_) /// NOLINT
|
||||
{
|
||||
body_stream = Aws::Utils::Stream::ResponseStream(
|
||||
Aws::New<SessionAwareIOStream<SessionPtr>>("http result streambuf", session_, incoming_stream.rdbuf())
|
||||
|
@ -49,7 +49,6 @@ public:
|
||||
private:
|
||||
ClientFactory();
|
||||
|
||||
private:
|
||||
Aws::SDKOptions aws_options;
|
||||
};
|
||||
|
||||
|
@ -42,7 +42,7 @@ private:
|
||||
using Base = LRUCache<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction>;
|
||||
|
||||
public:
|
||||
UncompressedCache(size_t max_size_in_bytes)
|
||||
explicit UncompressedCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
|
||||
/// Calculate key from path to file and offset.
|
||||
|
@ -132,7 +132,7 @@ inline void readVarUIntImpl(UInt64 & x, ReadBuffer & istr)
|
||||
if (istr.eof())
|
||||
throwReadAfterEOF();
|
||||
|
||||
UInt64 byte = *istr.position();
|
||||
UInt64 byte = *istr.position(); /// NOLINT
|
||||
++istr.position();
|
||||
x |= (byte & 0x7F) << (7 * i);
|
||||
|
||||
@ -172,7 +172,7 @@ inline const char * readVarUInt(UInt64 & x, const char * istr, size_t size)
|
||||
if (istr == end)
|
||||
throwReadAfterEOF();
|
||||
|
||||
UInt64 byte = *istr;
|
||||
UInt64 byte = *istr; /// NOLINT
|
||||
++istr;
|
||||
x |= (byte & 0x7F) << (7 * i);
|
||||
|
||||
|
@ -28,7 +28,7 @@ protected:
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::OpenFileForWrite};
|
||||
|
||||
public:
|
||||
WriteBufferFromFile(
|
||||
explicit WriteBufferFromFile(
|
||||
const std::string & file_name_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
int flags = -1,
|
||||
@ -37,7 +37,7 @@ public:
|
||||
size_t alignment = 0);
|
||||
|
||||
/// Use pre-opened file descriptor.
|
||||
WriteBufferFromFile(
|
||||
explicit WriteBufferFromFile(
|
||||
int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object.
|
||||
const std::string & original_file_name = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
class WriteBufferFromFileDescriptor : public WriteBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
WriteBufferFromFileDescriptor(
|
||||
explicit WriteBufferFromFileDescriptor(
|
||||
int fd_ = -1,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
|
@ -17,7 +17,7 @@ protected:
|
||||
|
||||
public:
|
||||
using WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor;
|
||||
~WriteBufferFromFileDescriptorDiscardOnFailure() override {}
|
||||
~WriteBufferFromFileDescriptorDiscardOnFailure() override = default;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
class WriteBufferFromOStream : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
public:
|
||||
WriteBufferFromOStream(
|
||||
explicit WriteBufferFromOStream(
|
||||
std::ostream & ostr_,
|
||||
size_t size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
@ -21,7 +21,7 @@ public:
|
||||
~WriteBufferFromOStream() override;
|
||||
|
||||
protected:
|
||||
WriteBufferFromOStream(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0);
|
||||
explicit WriteBufferFromOStream(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0);
|
||||
|
||||
void nextImpl() override;
|
||||
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
class WriteBufferFromPocoSocket : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
public:
|
||||
WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
explicit WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
|
||||
~WriteBufferFromPocoSocket() override;
|
||||
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
~WriteBufferFromTemporaryFile() override;
|
||||
|
||||
private:
|
||||
WriteBufferFromTemporaryFile(std::unique_ptr<TemporaryFile> && tmp_file);
|
||||
explicit WriteBufferFromTemporaryFile(std::unique_ptr<TemporaryFile> && tmp_file);
|
||||
|
||||
std::shared_ptr<ReadBuffer> getReadBufferImpl() override;
|
||||
|
||||
|
@ -67,7 +67,7 @@ private:
|
||||
void finalizeImpl() override final
|
||||
{
|
||||
vector.resize(
|
||||
((position() - reinterpret_cast<Position>(vector.data()))
|
||||
((position() - reinterpret_cast<Position>(vector.data())) /// NOLINT
|
||||
+ sizeof(typename VectorType::value_type) - 1) /// Align up.
|
||||
/ sizeof(typename VectorType::value_type));
|
||||
|
||||
|
@ -16,7 +16,7 @@ class WriteBufferValidUTF8 final : public BufferWithOwnMemory<WriteBuffer>
|
||||
public:
|
||||
static const size_t DEFAULT_SIZE;
|
||||
|
||||
WriteBufferValidUTF8(
|
||||
explicit WriteBufferValidUTF8(
|
||||
WriteBuffer & output_buffer_,
|
||||
bool group_replacements_ = true,
|
||||
const char * replacement_ = "\xEF\xBF\xBD",
|
||||
|
@ -7,7 +7,7 @@ namespace DB
|
||||
{
|
||||
|
||||
template <typename IteratorSrc, typename IteratorDst>
|
||||
void formatHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes)
|
||||
void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes)
|
||||
{
|
||||
size_t src_pos = 0;
|
||||
size_t dst_pos = 0;
|
||||
|
@ -80,7 +80,7 @@ inline void writeChar(char c, size_t n, WriteBuffer & buf)
|
||||
template <typename T>
|
||||
inline void writePODBinary(const T & x, WriteBuffer & buf)
|
||||
{
|
||||
buf.write(reinterpret_cast<const char *>(&x), sizeof(x));
|
||||
buf.write(reinterpret_cast<const char *>(&x), sizeof(x)); /// NOLINT
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -663,7 +663,7 @@ inline void writeXMLStringForTextElement(const StringRef & s, WriteBuffer & buf)
|
||||
}
|
||||
|
||||
template <typename IteratorSrc, typename IteratorDst>
|
||||
void formatHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes);
|
||||
void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes);
|
||||
void formatUUID(const UInt8 * src16, UInt8 * dst36);
|
||||
void formatUUID(std::reverse_iterator<const UInt8 *> src16, UInt8 * dst36);
|
||||
|
||||
|
@ -5,22 +5,19 @@
|
||||
#include <base/itoa.h>
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename T> constexpr size_t max_int_width = 20;
|
||||
template <> inline constexpr size_t max_int_width<UInt8> = 3; /// 255
|
||||
template <> inline constexpr size_t max_int_width<Int8> = 4; /// -128
|
||||
template <> inline constexpr size_t max_int_width<UInt16> = 5; /// 65535
|
||||
template <> inline constexpr size_t max_int_width<Int16> = 6; /// -32768
|
||||
template <> inline constexpr size_t max_int_width<UInt32> = 10; /// 4294967295
|
||||
template <> inline constexpr size_t max_int_width<Int32> = 11; /// -2147483648
|
||||
template <> inline constexpr size_t max_int_width<UInt64> = 20; /// 18446744073709551615
|
||||
template <> inline constexpr size_t max_int_width<Int64> = 20; /// -9223372036854775808
|
||||
template <> inline constexpr size_t max_int_width<UInt128> = 39; /// 340282366920938463463374607431768211455
|
||||
template <> inline constexpr size_t max_int_width<Int128> = 40; /// -170141183460469231731687303715884105728
|
||||
template <> inline constexpr size_t max_int_width<UInt256> = 78; /// 115792089237316195423570985008687907853269984665640564039457584007913129639935
|
||||
template <> inline constexpr size_t max_int_width<Int256> = 78; /// -57896044618658097711785492504343953926634992332820282019728792003956564819968
|
||||
}
|
||||
template <typename T> constexpr size_t max_int_width = 20;
|
||||
template <> inline constexpr size_t max_int_width<UInt8> = 3; /// 255
|
||||
template <> inline constexpr size_t max_int_width<Int8> = 4; /// -128
|
||||
template <> inline constexpr size_t max_int_width<UInt16> = 5; /// 65535
|
||||
template <> inline constexpr size_t max_int_width<Int16> = 6; /// -32768
|
||||
template <> inline constexpr size_t max_int_width<UInt32> = 10; /// 4294967295
|
||||
template <> inline constexpr size_t max_int_width<Int32> = 11; /// -2147483648
|
||||
template <> inline constexpr size_t max_int_width<UInt64> = 20; /// 18446744073709551615
|
||||
template <> inline constexpr size_t max_int_width<Int64> = 20; /// -9223372036854775808
|
||||
template <> inline constexpr size_t max_int_width<UInt128> = 39; /// 340282366920938463463374607431768211455
|
||||
template <> inline constexpr size_t max_int_width<Int128> = 40; /// -170141183460469231731687303715884105728
|
||||
template <> inline constexpr size_t max_int_width<UInt256> = 78; /// 115792089237316195423570985008687907853269984665640564039457584007913129639935
|
||||
template <> inline constexpr size_t max_int_width<Int256> = 78; /// -57896044618658097711785492504343953926634992332820282019728792003956564819968
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -16,7 +16,7 @@ namespace ErrorCodes
|
||||
class ZstdInflatingReadBuffer : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
ZstdInflatingReadBuffer(
|
||||
explicit ZstdInflatingReadBuffer(
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
|
@ -154,7 +154,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf)
|
||||
|
||||
if (likely(!buf.eof() && buf.position() + MAX_LENGTH <= buf.buffer().end()))
|
||||
{
|
||||
auto initial_position = buf.position();
|
||||
auto * initial_position = buf.position();
|
||||
auto res = fast_float::from_chars(initial_position, buf.buffer().end(), x);
|
||||
|
||||
if (unlikely(res.ec != std::errc()))
|
||||
|
@ -23,8 +23,7 @@ using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
class AddIndexConstraintsOptimizer final
|
||||
{
|
||||
public:
|
||||
AddIndexConstraintsOptimizer(
|
||||
const StorageMetadataPtr & metadata_snapshot);
|
||||
explicit AddIndexConstraintsOptimizer(const StorageMetadataPtr & metadata_snapshot);
|
||||
|
||||
void perform(CNFQuery & cnf_query);
|
||||
|
||||
|
@ -42,9 +42,6 @@ using Sizes = std::vector<size_t>;
|
||||
/// 2,1,1
|
||||
///
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
constexpr auto getBitmapSize()
|
||||
{
|
||||
@ -62,8 +59,6 @@ constexpr auto getBitmapSize()
|
||||
0)));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
template<typename T, size_t step>
|
||||
void fillFixedBatch(size_t num_rows, const T * source, T * dest)
|
||||
{
|
||||
@ -255,7 +250,7 @@ static inline T ALWAYS_INLINE packFixed(
|
||||
|
||||
|
||||
/// Hash a set of keys into a UInt128 value.
|
||||
static inline UInt128 ALWAYS_INLINE hash128(
|
||||
static inline UInt128 ALWAYS_INLINE hash128( /// NOLINT
|
||||
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns)
|
||||
{
|
||||
UInt128 key;
|
||||
@ -269,29 +264,9 @@ static inline UInt128 ALWAYS_INLINE hash128(
|
||||
return key;
|
||||
}
|
||||
|
||||
|
||||
/// Copy keys to the pool. Then put into pool StringRefs to them and return the pointer to the first.
|
||||
static inline StringRef * ALWAYS_INLINE placeKeysInPool(
|
||||
size_t keys_size, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
for (size_t j = 0; j < keys_size; ++j)
|
||||
{
|
||||
char * place = pool.alloc(keys[j].size);
|
||||
memcpySmallAllowReadWriteOverflow15(place, keys[j].data, keys[j].size);
|
||||
keys[j].data = place;
|
||||
}
|
||||
|
||||
/// Place the StringRefs on the newly copied keys in the pool.
|
||||
char * res = pool.alignedAlloc(keys_size * sizeof(StringRef), alignof(StringRef));
|
||||
memcpySmallAllowReadWriteOverflow15(res, keys.data(), keys_size * sizeof(StringRef));
|
||||
|
||||
return reinterpret_cast<StringRef *>(res);
|
||||
}
|
||||
|
||||
|
||||
/** Serialize keys into a continuous chunk of memory.
|
||||
*/
|
||||
static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous(
|
||||
static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous( /// NOLINT
|
||||
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, Arena & pool)
|
||||
{
|
||||
const char * begin = nullptr;
|
||||
|
@ -660,7 +660,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
case Type::without_key: break;
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
case Type::NAME: NAME = std::make_unique<decltype(NAME)::element_type>(); break;
|
||||
case Type::NAME: (NAME) = std::make_unique<decltype(NAME)::element_type>(); break;
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
@ -677,7 +677,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
case Type::without_key: return 1;
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
case Type::NAME: return NAME->data.size() + (without_key != nullptr);
|
||||
case Type::NAME: return (NAME)->data.size() + (without_key != nullptr);
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
@ -694,7 +694,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
case Type::without_key: return 1;
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
case Type::NAME: return NAME->data.size();
|
||||
case Type::NAME: return (NAME)->data.size();
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
@ -753,6 +753,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
M(low_cardinality_key_string) \
|
||||
M(low_cardinality_key_fixed_string) \
|
||||
|
||||
/// NOLINTNEXTLINE
|
||||
#define APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \
|
||||
M(key8) \
|
||||
M(key16) \
|
||||
@ -766,6 +767,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
M(low_cardinality_key8) \
|
||||
M(low_cardinality_key16) \
|
||||
|
||||
/// NOLINTNEXTLINE
|
||||
#define APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) \
|
||||
APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \
|
||||
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M) \
|
||||
@ -787,6 +789,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
|
||||
void convertToTwoLevel();
|
||||
|
||||
/// NOLINTNEXTLINE
|
||||
#define APPLY_FOR_VARIANTS_TWO_LEVEL(M) \
|
||||
M(key32_two_level) \
|
||||
M(key64_two_level) \
|
||||
@ -1341,7 +1344,7 @@ private:
|
||||
template <typename Method> Method & getDataVariant(AggregatedDataVariants & variants);
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
template <> inline decltype(AggregatedDataVariants::NAME)::element_type & getDataVariant<decltype(AggregatedDataVariants::NAME)::element_type>(AggregatedDataVariants & variants) { return *variants.NAME; }
|
||||
template <> inline decltype(AggregatedDataVariants::NAME)::element_type & getDataVariant<decltype(AggregatedDataVariants::NAME)::element_type>(AggregatedDataVariants & variants) { return *variants.NAME; } /// NOLINT
|
||||
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
using UnderType = UInt64;
|
||||
using Container = std::vector<UnderType>;
|
||||
|
||||
BloomFilter(const BloomFilterParameters & params);
|
||||
explicit BloomFilter(const BloomFilterParameters & params);
|
||||
/// size -- size of filter in bytes.
|
||||
/// hashes -- number of used hash functions.
|
||||
/// seed -- random seed for hash functions generation.
|
||||
|
@ -19,7 +19,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
void ClientInfo::write(WriteBuffer & out, const UInt64 server_protocol_revision) const
|
||||
void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const
|
||||
{
|
||||
if (server_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO)
|
||||
throw Exception("Logical error: method ClientInfo::write is called for unsupported server revision", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -99,7 +99,7 @@ void ClientInfo::write(WriteBuffer & out, const UInt64 server_protocol_revision)
|
||||
}
|
||||
|
||||
|
||||
void ClientInfo::read(ReadBuffer & in, const UInt64 client_protocol_revision)
|
||||
void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision)
|
||||
{
|
||||
if (client_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO)
|
||||
throw Exception("Logical error: method ClientInfo::read is called for unsupported client revision", ErrorCodes::LOGICAL_ERROR);
|
||||
|
@ -119,8 +119,8 @@ public:
|
||||
* Only values that are not calculated automatically or passed separately are serialized.
|
||||
* Revisions are passed to use format that server will understand or client was used.
|
||||
*/
|
||||
void write(WriteBuffer & out, const UInt64 server_protocol_revision) const;
|
||||
void read(ReadBuffer & in, const UInt64 client_protocol_revision);
|
||||
void write(WriteBuffer & out, UInt64 server_protocol_revision) const;
|
||||
void read(ReadBuffer & in, UInt64 client_protocol_revision);
|
||||
|
||||
/// Initialize parameters on client initiating query.
|
||||
void setInitialQuery();
|
||||
|
@ -63,7 +63,6 @@ public:
|
||||
/// is used to set a limit on the size of the timeout
|
||||
static Poco::Timespan saturate(Poco::Timespan v, Poco::Timespan limit);
|
||||
|
||||
public:
|
||||
using SlotToShard = std::vector<UInt64>;
|
||||
|
||||
struct Address
|
||||
@ -192,7 +191,6 @@ public:
|
||||
/// Name of directory for asynchronous write to StorageDistributed if has_internal_replication
|
||||
const std::string & insertPathForInternalReplication(bool prefer_localhost_replica, bool use_compact_format) const;
|
||||
|
||||
public:
|
||||
ShardInfoInsertPathForInternalReplication insert_path_for_internal_replication;
|
||||
/// Number of the shard, the indexation begins with 1
|
||||
UInt32 shard_num = 0;
|
||||
|
@ -159,7 +159,7 @@ ComparisonGraph::CompareResult ComparisonGraph::pathToCompareResult(Path path, b
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
std::optional<ComparisonGraph::Path> ComparisonGraph::findPath(const size_t start, const size_t finish) const
|
||||
std::optional<ComparisonGraph::Path> ComparisonGraph::findPath(size_t start, size_t finish) const
|
||||
{
|
||||
const auto it = dists.find(std::make_pair(start, finish));
|
||||
if (it == std::end(dists))
|
||||
@ -232,7 +232,7 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con
|
||||
return CompareResult::UNKNOWN;
|
||||
}
|
||||
|
||||
bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const
|
||||
bool ComparisonGraph::isPossibleCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const
|
||||
{
|
||||
const auto result = compare(left, right);
|
||||
|
||||
@ -267,7 +267,7 @@ bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTP
|
||||
return possible_pairs.contains({expected, result});
|
||||
}
|
||||
|
||||
bool ComparisonGraph::isAlwaysCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const
|
||||
bool ComparisonGraph::isAlwaysCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const
|
||||
{
|
||||
const auto result = compare(left, right);
|
||||
|
||||
@ -324,12 +324,12 @@ std::optional<size_t> ComparisonGraph::getComponentId(const ASTPtr & ast) const
|
||||
}
|
||||
}
|
||||
|
||||
bool ComparisonGraph::hasPath(const size_t left, const size_t right) const
|
||||
bool ComparisonGraph::hasPath(size_t left, size_t right) const
|
||||
{
|
||||
return findPath(left, right) || findPath(right, left);
|
||||
}
|
||||
|
||||
ASTs ComparisonGraph::getComponent(const size_t id) const
|
||||
ASTs ComparisonGraph::getComponent(size_t id) const
|
||||
{
|
||||
return graph.vertices[id].asts;
|
||||
}
|
||||
@ -387,7 +387,7 @@ ComparisonGraph::CompareResult ComparisonGraph::functionNameToCompareResult(cons
|
||||
return it == std::end(relation_to_compare) ? CompareResult::UNKNOWN : it->second;
|
||||
}
|
||||
|
||||
ComparisonGraph::CompareResult ComparisonGraph::inverseCompareResult(const CompareResult result)
|
||||
ComparisonGraph::CompareResult ComparisonGraph::inverseCompareResult(CompareResult result)
|
||||
{
|
||||
static const std::unordered_map<CompareResult, CompareResult> inverse_relations =
|
||||
{
|
||||
@ -486,7 +486,7 @@ std::vector<ASTs> ComparisonGraph::getVertices() const
|
||||
|
||||
void ComparisonGraph::dfsComponents(
|
||||
const Graph & reversed_graph, size_t v,
|
||||
OptionalIndices & components, const size_t component)
|
||||
OptionalIndices & components, size_t component)
|
||||
{
|
||||
components[v] = component;
|
||||
for (const auto & edge : reversed_graph.edges[v])
|
||||
|
@ -17,7 +17,7 @@ class ComparisonGraph
|
||||
{
|
||||
public:
|
||||
/// atomic_formulas are extracted from constraints.
|
||||
ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas);
|
||||
explicit ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas);
|
||||
|
||||
enum class CompareResult
|
||||
{
|
||||
@ -32,15 +32,15 @@ public:
|
||||
|
||||
static CompareResult atomToCompareResult(const CNFQuery::AtomicFormula & atom);
|
||||
static CompareResult functionNameToCompareResult(const std::string & name);
|
||||
static CompareResult inverseCompareResult(const CompareResult result);
|
||||
static CompareResult inverseCompareResult(CompareResult result);
|
||||
|
||||
CompareResult compare(const ASTPtr & left, const ASTPtr & right) const;
|
||||
|
||||
/// It's possible that left <expected> right
|
||||
bool isPossibleCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const;
|
||||
bool isPossibleCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const;
|
||||
|
||||
/// It's always true that left <expected> right
|
||||
bool isAlwaysCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const;
|
||||
bool isAlwaysCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const;
|
||||
|
||||
/// Returns all expressions from component to which @ast belongs if any.
|
||||
std::vector<ASTPtr> getEqual(const ASTPtr & ast) const;
|
||||
@ -52,11 +52,11 @@ public:
|
||||
std::optional<std::size_t> getComponentId(const ASTPtr & ast) const;
|
||||
|
||||
/// Returns all expressions from component.
|
||||
std::vector<ASTPtr> getComponent(const std::size_t id) const;
|
||||
std::vector<ASTPtr> getComponent(size_t id) const;
|
||||
|
||||
size_t getNumOfComponents() const { return graph.vertices.size(); }
|
||||
|
||||
bool hasPath(const size_t left, const size_t right) const;
|
||||
bool hasPath(size_t left, size_t right) const;
|
||||
|
||||
/// Find constants lessOrEqual and greaterOrEqual.
|
||||
/// For int and double linear programming can be applied here.
|
||||
@ -131,7 +131,7 @@ private:
|
||||
/// Assigns index of component for each vertex.
|
||||
static void dfsComponents(
|
||||
const Graph & reversed_graph, size_t v,
|
||||
OptionalIndices & components, const size_t component);
|
||||
OptionalIndices & components, size_t component);
|
||||
|
||||
enum class Path
|
||||
{
|
||||
@ -140,7 +140,7 @@ private:
|
||||
};
|
||||
|
||||
static CompareResult pathToCompareResult(Path path, bool inverse);
|
||||
std::optional<Path> findPath(const size_t start, const size_t finish) const;
|
||||
std::optional<Path> findPath(size_t start, size_t finish) const;
|
||||
|
||||
/// Calculate @dists.
|
||||
static std::map<std::pair<size_t, size_t>, Path> buildDistsFromGraph(const Graph & g);
|
||||
|
@ -41,7 +41,7 @@ class CrashLog : public SystemLog<CrashLogElement>
|
||||
public:
|
||||
static void initialize(std::shared_ptr<CrashLog> crash_log_)
|
||||
{
|
||||
crash_log = std::move(crash_log_);
|
||||
crash_log = crash_log_;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -107,7 +107,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext
|
||||
|
||||
StoragePtr getTable() const;
|
||||
|
||||
operator bool () const { return id != UUIDHelpers::Nil; }
|
||||
operator bool () const { return id != UUIDHelpers::Nil; } /// NOLINT
|
||||
|
||||
IDatabase * temporary_tables = nullptr;
|
||||
UUID id = UUIDHelpers::Nil;
|
||||
|
@ -2,6 +2,9 @@
|
||||
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -47,7 +47,7 @@ public:
|
||||
/// then we can cache the scalars forever (for any query that doesn't use the virtual storage either), but if it does use the virtual
|
||||
/// storage then we can only keep the scalar result around while we are working with that source block
|
||||
/// You can find more details about this under ExecuteScalarSubqueriesMatcher::visit
|
||||
bool usesViewSource() { return uses_view_source; }
|
||||
bool usesViewSource() const { return uses_view_source; }
|
||||
|
||||
protected:
|
||||
ASTPtr query_ptr;
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
|
||||
/// Add block of data from right hand of JOIN.
|
||||
/// @returns false, if some limit was exceeded and you should not insert more data.
|
||||
virtual bool addJoinedBlock(const Block & block, bool check_limits = true) = 0;
|
||||
virtual bool addJoinedBlock(const Block & block, bool check_limits = true) = 0; /// NOLINT
|
||||
|
||||
virtual void checkTypesOfKeys(const Block & block) const = 0;
|
||||
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
/// These methods could be overridden for the need of the unit test.
|
||||
virtual bool hasAtLeastTwoShards(const IStorage & table) const;
|
||||
virtual std::pair<std::string, std::string> getRemoteDatabaseAndTableName(const IStorage & table) const;
|
||||
virtual ~CheckShardsAndTables() {}
|
||||
virtual ~CheckShardsAndTables() = default;
|
||||
};
|
||||
|
||||
InJoinSubqueriesPreprocessor(
|
||||
|
@ -29,7 +29,7 @@ class LogicalExpressionsOptimizer final
|
||||
{
|
||||
const UInt64 optimize_min_equality_disjunction_chain_length;
|
||||
|
||||
ExtractedSettings(UInt64 optimize_min_equality_disjunction_chain_length_)
|
||||
explicit ExtractedSettings(UInt64 optimize_min_equality_disjunction_chain_length_)
|
||||
: optimize_min_equality_disjunction_chain_length(optimize_min_equality_disjunction_chain_length_)
|
||||
{}
|
||||
};
|
||||
@ -68,7 +68,6 @@ private:
|
||||
using DisjunctiveEqualityChainsMap = std::map<OrWithExpression, Equalities>;
|
||||
using DisjunctiveEqualityChain = DisjunctiveEqualityChainsMap::value_type;
|
||||
|
||||
private:
|
||||
/** Collect information about all the equations in the OR chains (not necessarily homogeneous).
|
||||
* This information is grouped by the expression that is on the left side of the equation.
|
||||
*/
|
||||
@ -92,12 +91,10 @@ private:
|
||||
/// Restore the original column order after optimization.
|
||||
void reorderColumns();
|
||||
|
||||
private:
|
||||
using ParentNodes = std::vector<IAST *>;
|
||||
using FunctionParentMap = std::unordered_map<const IAST *, ParentNodes>;
|
||||
using ColumnToPosition = std::unordered_map<const IAST *, size_t>;
|
||||
|
||||
private:
|
||||
ASTSelectQuery * select_query;
|
||||
const ExtractedSettings settings;
|
||||
/// Information about the OR-chains inside the query.
|
||||
|
@ -25,7 +25,7 @@ struct OpenTelemetrySpan
|
||||
struct OpenTelemetrySpanLogElement : public OpenTelemetrySpan
|
||||
{
|
||||
OpenTelemetrySpanLogElement() = default;
|
||||
OpenTelemetrySpanLogElement(const OpenTelemetrySpan & span)
|
||||
explicit OpenTelemetrySpanLogElement(const OpenTelemetrySpan & span)
|
||||
: OpenTelemetrySpan(span) {}
|
||||
|
||||
static std::string name() { return "OpenTelemetrySpanLog"; }
|
||||
@ -44,7 +44,7 @@ public:
|
||||
|
||||
struct OpenTelemetrySpanHolder : public OpenTelemetrySpan
|
||||
{
|
||||
OpenTelemetrySpanHolder(const std::string & _operation_name);
|
||||
explicit OpenTelemetrySpanHolder(const std::string & _operation_name);
|
||||
void addAttribute(const std::string& name, UInt64 value);
|
||||
void addAttribute(const std::string& name, const std::string& value);
|
||||
void addAttribute(const Exception & e);
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
class OptimizeIfWithConstantConditionVisitor
|
||||
{
|
||||
public:
|
||||
OptimizeIfWithConstantConditionVisitor(Aliases & aliases_)
|
||||
explicit OptimizeIfWithConstantConditionVisitor(Aliases & aliases_)
|
||||
: aliases(aliases_)
|
||||
{}
|
||||
|
||||
|
@ -216,7 +216,7 @@ struct ProcessListForUserInfo
|
||||
/// Data about queries for one user.
|
||||
struct ProcessListForUser
|
||||
{
|
||||
ProcessListForUser(ProcessList * global_process_list);
|
||||
explicit ProcessListForUser(ProcessList * global_process_list);
|
||||
|
||||
/// query_id -> ProcessListElement(s). There can be multiple queries with the same query_id as long as all queries except one are cancelled.
|
||||
using QueryToElement = std::unordered_map<String, QueryStatus *>;
|
||||
|
@ -25,7 +25,7 @@ class QueryNormalizer
|
||||
bool prefer_column_name_to_alias;
|
||||
|
||||
template <typename T>
|
||||
ExtractedSettings(const T & settings)
|
||||
ExtractedSettings(const T & settings) /// NOLINT
|
||||
: max_ast_depth(settings.max_ast_depth)
|
||||
, max_expanded_ast_elements(settings.max_expanded_ast_elements)
|
||||
, prefer_column_name_to_alias(settings.prefer_column_name_to_alias)
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Names.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -26,7 +26,7 @@ struct RowRef
|
||||
const Block * block = nullptr;
|
||||
SizeT row_num = 0;
|
||||
|
||||
RowRef() {}
|
||||
RowRef() {} /// NOLINT
|
||||
RowRef(const Block * block_, size_t row_num_) : block(block_), row_num(row_num_) {}
|
||||
};
|
||||
|
||||
@ -42,7 +42,7 @@ struct RowRefList : RowRef
|
||||
Batch * next;
|
||||
RowRef row_refs[MAX_SIZE];
|
||||
|
||||
Batch(Batch * parent)
|
||||
explicit Batch(Batch * parent)
|
||||
: next(parent)
|
||||
{}
|
||||
|
||||
@ -52,7 +52,7 @@ struct RowRefList : RowRef
|
||||
{
|
||||
if (full())
|
||||
{
|
||||
auto batch = pool.alloc<Batch>();
|
||||
auto * batch = pool.alloc<Batch>();
|
||||
*batch = Batch(this);
|
||||
batch->insert(std::move(row_ref), pool);
|
||||
return batch;
|
||||
@ -66,7 +66,7 @@ struct RowRefList : RowRef
|
||||
class ForwardIterator
|
||||
{
|
||||
public:
|
||||
ForwardIterator(const RowRefList * begin)
|
||||
explicit ForwardIterator(const RowRefList * begin)
|
||||
: root(begin)
|
||||
, first(true)
|
||||
, batch(root->next)
|
||||
@ -115,7 +115,7 @@ struct RowRefList : RowRef
|
||||
size_t position;
|
||||
};
|
||||
|
||||
RowRefList() {}
|
||||
RowRefList() {} /// NOLINT
|
||||
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {}
|
||||
|
||||
ForwardIterator begin() const { return ForwardIterator(this); }
|
||||
@ -221,7 +221,7 @@ public:
|
||||
T asof_value;
|
||||
RowRef row_ref;
|
||||
|
||||
Entry(T v) : asof_value(v) {}
|
||||
explicit Entry(T v) : asof_value(v) {}
|
||||
Entry(T v, RowRef rr) : asof_value(v), row_ref(rr) {}
|
||||
};
|
||||
|
||||
@ -241,8 +241,8 @@ public:
|
||||
Entry<Decimal128>::LookupPtr,
|
||||
Entry<DateTime64>::LookupPtr>;
|
||||
|
||||
AsofRowRefs() {}
|
||||
AsofRowRefs(TypeIndex t);
|
||||
AsofRowRefs() = default;
|
||||
explicit AsofRowRefs(TypeIndex t);
|
||||
|
||||
static std::optional<TypeIndex> getTypeSize(const IColumn & asof_column, size_t & type_size);
|
||||
|
||||
|
@ -41,9 +41,9 @@ struct StorageID
|
||||
assertNotEmpty();
|
||||
}
|
||||
|
||||
StorageID(const ASTQueryWithTableAndOutput & query);
|
||||
StorageID(const ASTTableIdentifier & table_identifier_node);
|
||||
StorageID(const ASTPtr & node);
|
||||
StorageID(const ASTQueryWithTableAndOutput & query); /// NOLINT
|
||||
StorageID(const ASTTableIdentifier & table_identifier_node); /// NOLINT
|
||||
StorageID(const ASTPtr & node); /// NOLINT
|
||||
|
||||
String getDatabaseName() const;
|
||||
|
||||
|
@ -103,7 +103,7 @@ private:
|
||||
|
||||
friend class TreeRewriter;
|
||||
|
||||
const SizeLimits size_limits;
|
||||
SizeLimits size_limits;
|
||||
const size_t default_max_bytes = 0;
|
||||
const bool join_use_nulls = false;
|
||||
const size_t max_joined_block_rows = 0;
|
||||
@ -114,7 +114,7 @@ private:
|
||||
const String temporary_files_codec = "LZ4";
|
||||
|
||||
/// the limit has no technical reasons, it supposed to improve safety
|
||||
const size_t MAX_DISJUNCTS = 16;
|
||||
const size_t MAX_DISJUNCTS = 16; /// NOLINT
|
||||
|
||||
ASTs key_asts_left;
|
||||
ASTs key_asts_right;
|
||||
|
@ -18,7 +18,7 @@ class TraceLog;
|
||||
class TraceCollector
|
||||
{
|
||||
public:
|
||||
TraceCollector(std::shared_ptr<TraceLog> trace_log_);
|
||||
explicit TraceCollector(std::shared_ptr<TraceLog> trace_log_);
|
||||
~TraceCollector();
|
||||
|
||||
static inline void collect(TraceType trace_type, const StackTrace & stack_trace, Int64 size)
|
||||
|
@ -303,7 +303,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
}
|
||||
|
||||
/// 'select * from a join b using id' should result one 'id' column
|
||||
void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, Data & data)
|
||||
void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(ASTPtr ast, Data & data)
|
||||
{
|
||||
const auto & table_join = ast->as<ASTTableJoin &>();
|
||||
|
||||
|
@ -52,7 +52,7 @@ private:
|
||||
static void visit(ASTExpressionList &, const ASTPtr &, Data &);
|
||||
static void visit(ASTFunction &, const ASTPtr &, Data &);
|
||||
|
||||
static void extractJoinUsingColumns(const ASTPtr ast, Data & data);
|
||||
static void extractJoinUsingColumns(ASTPtr ast, Data & data);
|
||||
};
|
||||
|
||||
/// Visits AST for names qualification.
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
using OrGroup = std::set<AtomicFormula>;
|
||||
using AndGroup = std::set<OrGroup>;
|
||||
|
||||
CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { }
|
||||
CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { } /// NOLINT
|
||||
|
||||
template <typename P>
|
||||
CNFQuery & filterAlwaysTrueGroups(P predicate_is_unknown) /// delete always true groups
|
||||
@ -91,7 +91,7 @@ public:
|
||||
CNFQuery & appendGroup(AndGroup&& and_group)
|
||||
{
|
||||
for (auto && or_group : and_group)
|
||||
statements.emplace(std::move(or_group));
|
||||
statements.emplace(or_group);
|
||||
return *this;
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user