Added most of clang-static-analyzer checks

This commit is contained in:
Alexey Milovidov 2020-03-20 02:48:53 +03:00
parent 3f13464e3d
commit a5e3d4efb7
113 changed files with 226 additions and 608 deletions

View File

@ -136,7 +136,6 @@ Checks: '-*,
clang-analyzer-core.uninitialized.CapturedBlockVariable,
clang-analyzer-core.uninitialized.UndefReturn,
clang-analyzer-cplusplus.InnerPointer,
clang-analyzer-cplusplus.NewDelete,
clang-analyzer-cplusplus.NewDeleteLeaks,
clang-analyzer-cplusplus.PlacementNewChecker,
clang-analyzer-cplusplus.SelfAssignment,
@ -162,3 +161,6 @@ Checks: '-*,
boost-use-to-string,
'
WarningsAsErrors: '*'
# TODO:
# clang-analyzer-cplusplus.NewDelete,

View File

@ -292,3 +292,26 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end)
{
return const_cast<char *>(detail::find_last_symbols_sse2<false, detail::ReturnMode::Nullptr, symbols...>(begin, end));
}
/// Slightly resembles boost::split. The drawback of boost::split is that it fires a false positive in clang static analyzer.
/// See https://github.com/boostorg/algorithm/issues/63
/// And https://bugs.llvm.org/show_bug.cgi?id=41141
template <char... symbols, typename To>
inline void splitInto(To & to, const std::string & what, bool token_compress = false)
{
const char * pos = what.data();
const char * end = pos + what.size();
while (pos < end)
{
const char * delimiter_or_end = find_first_symbols<','>(pos, end);
if (!token_compress || pos + 1 < delimiter_or_end)
to.emplace_back(pos, delimiter_or_end);
if (delimiter_or_end < end)
pos = delimiter_or_end + 1;
else
pos = delimiter_or_end;
}
}

View File

@ -162,7 +162,7 @@ void TCPHandler::runImpl()
std::optional<DB::Exception> exception;
bool network_error = false;
bool send_exception_with_stack_trace = connection_context.getSettingsRef().calculate_text_stack_trace;
bool send_exception_with_stack_trace = true;
try
{
@ -1030,7 +1030,7 @@ void TCPHandler::receiveUnexpectedData()
last_block_in.header,
client_revision);
Block skip_block = skip_block_in->read();
skip_block_in->read();
throw NetException("Unexpected packet Data received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}

View File

@ -87,15 +87,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample(const std::string &
{
assertUnary(name, argument_types);
UInt64 max_elems = std::numeric_limits<UInt64>::max();
UInt64 seed = thread_local_rng();
UInt64 * params[2] = {&max_elems, &seed};
if (parameters.size() != 1 && parameters.size() != 2)
throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1 or 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto i = 0ul; i < parameters.size(); ++i)
auto get_parameter = [&](size_t i)
{
auto type = parameters[i].getType();
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
@ -105,8 +101,16 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample(const std::string &
(type == Field::Types::UInt64 && parameters[i].get<UInt64>() == 0))
throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS);
*params[i] = parameters[i].get<UInt64>();
}
return parameters[i].get<UInt64>();
};
UInt64 max_elems = get_parameter(0);
UInt64 seed;
if (parameters.size() >= 2)
seed = get_parameter(1);
else
seed = thread_local_rng();
return createAggregateFunctionGroupArrayImpl<GroupArrayTrait<true, Sampler::RNG>>(argument_types[0], max_elems, seed);
}

View File

@ -228,13 +228,13 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
/// If initialized column is nullable.
bool is_nullable = false;
static const ColumnLowCardinality & getLowCardinalityColumn(const IColumn * low_cardinality_column)
static const ColumnLowCardinality & getLowCardinalityColumn(const IColumn * column)
{
auto column = typeid_cast<const ColumnLowCardinality *>(low_cardinality_column);
if (!column)
auto low_cardinality_column = typeid_cast<const ColumnLowCardinality *>(column);
if (!low_cardinality_column)
throw Exception("Invalid aggregation key type for HashMethodSingleLowCardinalityColumn method. "
"Excepted LowCardinality, got " + column->getName(), ErrorCodes::LOGICAL_ERROR);
return *column;
return *low_cardinality_column;
}
HashMethodSingleLowCardinalityColumn(

View File

@ -151,8 +151,8 @@ public:
void get128(char * out)
{
finalize();
reinterpret_cast<UInt64 *>(out)[0] = v0 ^ v1;
reinterpret_cast<UInt64 *>(out)[1] = v2 ^ v3;
unalignedStore<UInt64>(out, v0 ^ v1);
unalignedStore<UInt64>(out + 8, v2 ^ v3);
}
/// template for avoiding 'unsigned long long' vs 'unsigned long' problem on old poco in macos

View File

@ -61,21 +61,7 @@ void ZooKeeper::init(const std::string & implementation, const std::string & hos
throw KeeperException("No addresses passed to ZooKeeper constructor.", Coordination::ZBADARGUMENTS);
std::vector<std::string> addresses_strings;
const char * pos = hosts.data();
const char * end = pos + hosts.size();
while (pos < end)
{
const char * comma_or_end = find_first_symbols<','>(pos, end);
addresses_strings.emplace_back(pos, comma_or_end);
if (comma_or_end < end)
pos = comma_or_end + 1;
else
pos = comma_or_end;
}
splitInto<','>(addresses_strings, hosts);
Coordination::ZooKeeper::Addresses addresses;
addresses.reserve(addresses_strings.size());

View File

@ -4,7 +4,7 @@
#include <Common/ZooKeeper/ZooKeeperImpl.h>
#include <Common/typeid_cast.h>
#include <iostream>
#include <boost/algorithm/string.hpp>
#include <common/find_symbols.h>
using namespace Coordination;
@ -25,7 +25,7 @@ try
std::string addresses_arg = argv[1];
std::vector<std::string> addresses_strings;
boost::split(addresses_strings, addresses_arg, boost::is_any_of(","));
splitInto<','>(addresses_strings, addresses_arg);
ZooKeeper::Addresses addresses;
addresses.reserve(addresses_strings.size());
for (const auto & address_string : addresses_strings)

View File

@ -1,26 +0,0 @@
/// Taken from SMHasher.
#include "AvalancheTest.h"
//-----------------------------------------------------------------------------
double maxBias(std::vector<int> & counts, int reps)
{
double worst = 0;
for (auto count : counts)
{
double c = static_cast<double>(count) / static_cast<double>(reps);
double d = fabs(c * 2 - 1);
if (d > worst)
{
worst = d;
}
}
return worst;
}
//-----------------------------------------------------------------------------

View File

@ -1,264 +0,0 @@
/// Taken from SMHasher.
//-----------------------------------------------------------------------------
// Flipping a single bit of a key should cause an "avalanche" of changes in
// the hash function's output. Ideally, each output bits should flip 50% of
// the time - if the probability of an output bit flipping is not 50%, that bit
// is "biased". Too much bias means that patterns applied to the input will
// cause "echoes" of the patterns in the output, which in turn can cause the
// hash function to fail to create an even, random distribution of hash values.
#pragma once
#include "Random.h"
#include <vector>
#include <math.h>
#include <stdio.h>
// Avalanche fails if a bit is biased by more than 1%
#define AVALANCHE_FAIL 0.01
double maxBias(std::vector<int> & counts, int reps);
typedef void (*pfHash)(const void * blob, const int len, const uint32_t seed, void * out);
inline uint32_t getbit(const void * block, int len, uint32_t bit)
{
uint8_t * b = reinterpret_cast<uint8_t *>(const_cast<void *>(block));
int byte = bit >> 3;
bit = bit & 0x7;
if (byte < len)
return (b[byte] >> bit) & 1;
return 0;
}
template <typename T>
inline uint32_t getbit(T & blob, uint32_t bit)
{
return getbit(&blob, sizeof(blob), bit);
}
inline void flipbit(void * block, int len, uint32_t bit)
{
uint8_t * b = reinterpret_cast<uint8_t *>(block);
int byte = bit >> 3;
bit = bit & 0x7;
if (byte < len)
b[byte] ^= (1 << bit);
}
template <typename T>
inline void flipbit(T & blob, uint32_t bit)
{
flipbit(&blob, sizeof(blob), bit);
}
//-----------------------------------------------------------------------------
template <typename keytype, typename hashtype>
void calcBias(pfHash hash, std::vector<int> & counts, int reps, Rand & r)
{
const int keybytes = sizeof(keytype);
const int hashbytes = sizeof(hashtype);
const int keybits = keybytes * 8;
const int hashbits = hashbytes * 8;
keytype K;
hashtype A, B;
for (int irep = 0; irep < reps; irep++)
{
if (irep % (reps / 10) == 0)
printf(".");
r.rand_p(&K, keybytes);
hash(&K, keybytes, 0, &A);
int * cursor = counts.data();
for (int iBit = 0; iBit < keybits; iBit++)
{
flipbit(&K, keybytes, iBit);
hash(&K, keybytes, 0, &B);
flipbit(&K, keybytes, iBit);
for (int iOut = 0; iOut < hashbits; iOut++)
{
int bitA = getbit(&A, hashbytes, iOut);
int bitB = getbit(&B, hashbytes, iOut);
(*cursor++) += (bitA ^ bitB);
}
}
}
}
//-----------------------------------------------------------------------------
template <typename keytype, typename hashtype>
bool AvalancheTest(pfHash hash, const int reps)
{
Rand r(48273);
const int keybytes = sizeof(keytype);
const int hashbytes = sizeof(hashtype);
const int keybits = keybytes * 8;
const int hashbits = hashbytes * 8;
printf("Testing %3d-bit keys -> %3d-bit hashes, %8d reps", keybits, hashbits, reps);
//----------
std::vector<int> bins(keybits * hashbits, 0);
calcBias<keytype, hashtype>(hash, bins, reps, r);
//----------
bool result = true;
double b = maxBias(bins, reps);
printf(" worst bias is %f%%", b * 100.0);
if (b > AVALANCHE_FAIL)
{
printf(" !!!!! ");
result = false;
}
printf("\n");
return result;
}
//-----------------------------------------------------------------------------
// BIC test variant - store all intermediate data in a table, draw diagram
// afterwards (much faster)
template <typename keytype, typename hashtype>
void BicTest3(pfHash hash, const int reps, bool verbose = true)
{
const int keybytes = sizeof(keytype);
const int keybits = keybytes * 8;
const int hashbytes = sizeof(hashtype);
const int hashbits = hashbytes * 8;
const int pagesize = hashbits * hashbits * 4;
Rand r(11938);
double maxBias = 0;
int maxK = 0;
int maxA = 0;
int maxB = 0;
keytype key;
hashtype h1, h2;
std::vector<int> bins(keybits * pagesize, 0);
for (int keybit = 0; keybit < keybits; keybit++)
{
if (keybit % (keybits / 10) == 0)
printf(".");
int * page = &bins[keybit * pagesize];
for (int irep = 0; irep < reps; irep++)
{
r.rand_p(&key, keybytes);
hash(&key, keybytes, 0, &h1);
flipbit(key, keybit);
hash(&key, keybytes, 0, &h2);
hashtype d = h1 ^ h2;
for (int out1 = 0; out1 < hashbits - 1; out1++)
for (int out2 = out1 + 1; out2 < hashbits; out2++)
{
int * b = &page[(out1 * hashbits + out2) * 4];
uint32_t x = getbit(d, out1) | (getbit(d, out2) << 1);
b[x]++;
}
}
}
printf("\n");
for (int out1 = 0; out1 < hashbits - 1; out1++)
{
for (int out2 = out1 + 1; out2 < hashbits; out2++)
{
if (verbose)
printf("(%3d,%3d) - ", out1, out2);
for (int keybit = 0; keybit < keybits; keybit++)
{
int * page = &bins[keybit * pagesize];
int * bins_in_page = &page[(out1 * hashbits + out2) * 4];
double bias = 0;
for (int b = 0; b < 4; b++)
{
double b2 = static_cast<double>(bins_in_page[b]) / static_cast<double>(reps / 2);
b2 = fabs(b2 * 2 - 1);
if (b2 > bias)
bias = b2;
}
if (bias > maxBias)
{
maxBias = bias;
maxK = keybit;
maxA = out1;
maxB = out2;
}
if (verbose)
{
if (bias < 0.01)
printf(".");
else if (bias < 0.05)
printf("o");
else if (bias < 0.33)
printf("O");
else
printf("X");
}
}
// Finished keybit
if (verbose)
printf("\n");
}
if (verbose)
{
for (int i = 0; i < keybits + 12; i++)
printf("-");
printf("\n");
}
}
printf("Max bias %f - (%3d : %3d,%3d)\n", maxBias, maxK, maxA, maxB);
}
//-----------------------------------------------------------------------------

View File

@ -22,7 +22,7 @@ target_link_libraries (parallel_aggregation PRIVATE dbms)
add_executable (parallel_aggregation2 parallel_aggregation2.cpp)
target_link_libraries (parallel_aggregation2 PRIVATE dbms)
add_executable (int_hashes_perf int_hashes_perf.cpp AvalancheTest.cpp Random.cpp)
add_executable (int_hashes_perf int_hashes_perf.cpp)
target_link_libraries (int_hashes_perf PRIVATE clickhouse_common_io)
add_executable (simple_cache simple_cache.cpp)

View File

@ -1,9 +0,0 @@
/// Taken from SMHasher.
#include "Random.h"
Rand g_rand1(1);
Rand g_rand2(2);
Rand g_rand3(3);
Rand g_rand4(4);
//-----------------------------------------------------------------------------

View File

@ -1,129 +0,0 @@
/// Taken from SMHasher.
#pragma once
#include <cstdint>
//-----------------------------------------------------------------------------
// Xorshift RNG based on code by George Marsaglia
// http://en.wikipedia.org/wiki/Xorshift
struct Rand
{
uint32_t x;
uint32_t y;
uint32_t z;
uint32_t w;
Rand()
{
reseed(static_cast<uint32_t>(0));
}
explicit Rand(uint32_t seed)
{
reseed(seed);
}
void reseed(uint32_t seed)
{
x = 0x498b3bc5 ^ seed;
y = 0;
z = 0;
w = 0;
for (int i = 0; i < 10; i++)
mix();
}
void reseed(uint64_t seed)
{
x = 0x498b3bc5 ^ static_cast<uint32_t>(seed >> 0);
y = 0x5a05089a ^ static_cast<uint32_t>(seed >> 32);
z = 0;
w = 0;
for (int i = 0; i < 10; i++)
mix();
}
//-----------------------------------------------------------------------------
void mix(void)
{
uint32_t t = x ^ (x << 11);
x = y;
y = z;
z = w;
w = w ^ (w >> 19) ^ t ^ (t >> 8);
}
uint32_t rand_u32(void)
{
mix();
return x;
}
uint64_t rand_u64(void)
{
mix();
uint64_t a = x;
uint64_t b = y;
return (a << 32) | b;
}
void rand_p(void * blob, int bytes)
{
uint32_t * blocks = reinterpret_cast<uint32_t *>(blob);
while (bytes >= 4)
{
blocks[0] = rand_u32();
blocks++;
bytes -= 4;
}
uint8_t * tail = reinterpret_cast<uint8_t *>(blocks);
for (int i = 0; i < bytes; i++)
{
tail[i] = static_cast<uint8_t>(rand_u32());
}
}
};
//-----------------------------------------------------------------------------
extern Rand g_rand1;
inline uint32_t rand_u32(void)
{
return g_rand1.rand_u32();
}
inline uint64_t rand_u64(void)
{
return g_rand1.rand_u64();
}
inline void rand_p(void * blob, int bytes)
{
uint32_t * blocks = static_cast<uint32_t *>(blob);
while (bytes >= 4)
{
*blocks++ = rand_u32();
bytes -= 4;
}
uint8_t * tail = reinterpret_cast<uint8_t *>(blocks);
for (int i = 0; i < bytes; i++)
{
tail[i] = static_cast<uint8_t>(rand_u32());
}
}
//-----------------------------------------------------------------------------

View File

@ -8,7 +8,6 @@
#include <Common/HashTable/Hash.h>
#include <Common/Stopwatch.h>
#include <Core/Defines.h>
#include "AvalancheTest.h" /// Taken from SMHasher.
static void setAffinity()
@ -253,21 +252,6 @@ static inline void test(size_t n, const UInt64 * data, const char * name)
std::cerr << "Latency of ";
report(name, n, watch.elapsedSeconds(), tsc_diff, res);
}
/// quality. Methods are taken from SMHasher.
{
auto wrapper = [](const void * blob, const int, const uint32_t, void * out)
{
*reinterpret_cast<UInt32*>(out) = Func(*reinterpret_cast<const UInt64 *>(blob));
};
std::cerr << "Avalanche: " << std::endl;
AvalancheTest<UInt64, UInt32>(wrapper, 300000);
// std::cerr << "Bit Independence Criteria: " << std::endl;
// BicTest3<UInt64, UInt32>(wrapper, 2000000);
std::cerr << std::endl;
}
}

View File

@ -82,6 +82,7 @@ static void aggregate12(Map & map, Source::const_iterator begin, Source::const_i
{
if (prev_it != end && *it == *prev_it)
{
assert(found != nullptr);
++found->getMapped();
continue;
}
@ -89,6 +90,7 @@ static void aggregate12(Map & map, Source::const_iterator begin, Source::const_i
bool inserted;
map.emplace(*it, found, inserted);
assert(found != nullptr);
++found->getMapped();
}
}
@ -107,6 +109,7 @@ static void aggregate22(MapTwoLevel & map, Source::const_iterator begin, Source:
{
if (*it == *prev_it)
{
assert(found != nullptr);
++found->getMapped();
continue;
}
@ -114,6 +117,7 @@ static void aggregate22(MapTwoLevel & map, Source::const_iterator begin, Source:
bool inserted;
map.emplace(*it, found, inserted);
assert(found != nullptr);
++found->getMapped();
}
}

View File

@ -93,6 +93,7 @@ struct AggregateIndependentWithSequentialKeysOptimization
{
if (it != begin && *it == prev_key)
{
assert(place != nullptr);
updater(place->getMapped());
continue;
}
@ -100,6 +101,7 @@ struct AggregateIndependentWithSequentialKeysOptimization
bool inserted;
map.emplace(*it, place, inserted);
assert(place != nullptr);
if (inserted)
creator(place->getMapped());

View File

@ -14,7 +14,12 @@ bool CompressedReadBuffer::nextImpl()
if (!size_compressed)
return false;
memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer());
auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer();
/// This is for clang static analyzer.
assert(size_decompressed + additional_size_at_the_end_of_buffer > 0);
memory.resize(size_decompressed + additional_size_at_the_end_of_buffer);
working_buffer = Buffer(memory.data(), &memory[size_decompressed]);
decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum);
@ -39,8 +44,10 @@ size_t CompressedReadBuffer::readBig(char * to, size_t n)
if (!readCompressedData(size_decompressed, size_compressed_without_checksum))
return bytes_read;
/// If the decompressed block is placed entirely where it needs to be copied.
if (size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer() <= n - bytes_read)
auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer();
/// If the decompressed block fits entirely where it needs to be copied.
if (size_decompressed + additional_size_at_the_end_of_buffer <= n - bytes_read)
{
decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum);
bytes_read += size_decompressed;
@ -49,7 +56,11 @@ size_t CompressedReadBuffer::readBig(char * to, size_t n)
else
{
bytes += offset();
memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer());
/// This is for clang static analyzer.
assert(size_decompressed + additional_size_at_the_end_of_buffer > 0);
memory.resize(size_decompressed + additional_size_at_the_end_of_buffer);
working_buffer = Buffer(memory.data(), &memory[size_decompressed]);
pos = working_buffer.begin();

View File

@ -119,6 +119,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data());
size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data());
/// This is for clang static analyzer.
assert(size_decompressed > 0);
if (size_compressed_without_checksum > DBMS_MAX_COMPRESSED_SIZE)
@ -133,9 +134,11 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum));
auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer();
/// Is whole compressed block located in 'compressed_in->' buffer?
if (compressed_in->offset() >= header_size &&
compressed_in->position() + size_compressed_without_checksum + codec->getAdditionalSizeAtTheEndOfBuffer() - header_size <= compressed_in->buffer().end())
compressed_in->position() + size_compressed_without_checksum + additional_size_at_the_end_of_buffer - header_size <= compressed_in->buffer().end())
{
compressed_in->position() -= header_size;
compressed_buffer = compressed_in->position();
@ -143,7 +146,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
}
else
{
own_compressed_buffer.resize(size_compressed_without_checksum + codec->getAdditionalSizeAtTheEndOfBuffer());
own_compressed_buffer.resize(size_compressed_without_checksum + additional_size_at_the_end_of_buffer);
compressed_buffer = own_compressed_buffer.data();
compressed_in->readStrict(compressed_buffer + header_size, size_compressed_without_checksum - header_size);
}

View File

@ -24,9 +24,12 @@ bool CompressedReadBufferFromFile::nextImpl()
if (!size_compressed)
return false;
assert(size_decompressed > 0);
auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer();
memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer());
/// This is for clang static analyzer.
assert(size_decompressed + additional_size_at_the_end_of_buffer > 0);
memory.resize(size_decompressed + additional_size_at_the_end_of_buffer);
working_buffer = Buffer(memory.data(), &memory[size_decompressed]);
decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum);
@ -99,8 +102,10 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n)
if (!new_size_compressed)
return bytes_read;
auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer();
/// If the decompressed block fits entirely where it needs to be copied.
if (size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer() <= n - bytes_read)
if (size_decompressed + additional_size_at_the_end_of_buffer <= n - bytes_read)
{
decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum);
bytes_read += size_decompressed;
@ -110,7 +115,11 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n)
{
size_compressed = new_size_compressed;
bytes += offset();
memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer());
/// This is for clang static analyzer.
assert(size_decompressed + additional_size_at_the_end_of_buffer > 0);
memory.resize(size_decompressed + additional_size_at_the_end_of_buffer);
working_buffer = Buffer(memory.data(), &memory[size_decompressed]);
pos = working_buffer.begin();

View File

@ -102,14 +102,16 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour
{
UInt8 compression_method = source[idx + 1];
const auto codec = CompressionCodecFactory::instance().get(compression_method);
compressed_buf.resize(compressed_buf.size() + codec->getAdditionalSizeAtTheEndOfBuffer());
auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer();
compressed_buf.resize(compressed_buf.size() + additional_size_at_the_end_of_buffer);
UInt32 uncompressed_size = ICompressionCodec::readDecompressedBlockSize(compressed_buf.data());
if (idx == 0 && uncompressed_size != decompressed_size)
throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) +
", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA);
uncompressed_buf.resize(uncompressed_size + codec->getAdditionalSizeAtTheEndOfBuffer());
uncompressed_buf.resize(uncompressed_size + additional_size_at_the_end_of_buffer);
codec->decompress(compressed_buf.data(), source_size, uncompressed_buf.data());
uncompressed_buf.swap(compressed_buf);
source_size = uncompressed_size;

View File

@ -1,5 +1,4 @@
#include <boost/program_options.hpp>
#include <boost/algorithm/string.hpp>
#include <DataStreams/AsynchronousBlockInputStream.h>
#include <DataTypes/DataTypeFactory.h>
#include <Interpreters/Context.h>
@ -8,13 +7,13 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/LimitReadBuffer.h>
#include <Storages/StorageMemory.h>
#include <Poco/Net/MessageHeader.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Pipe.h>
#include <Core/ExternalTable.h>
#include <Processors/Sources/SinkToOutputStream.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Core/ExternalTable.h>
#include <Poco/Net/MessageHeader.h>
#include <common/find_symbols.h>
namespace DB
@ -42,11 +41,11 @@ ExternalTableDataPtr BaseExternalTable::getData(const Context & context)
void BaseExternalTable::clean()
{
name = "";
file = "";
format = "";
name.clear();
file.clear();
format.clear();
structure.clear();
sample_block = Block();
sample_block.clear();
read_buffer.reset();
}
@ -58,19 +57,13 @@ void BaseExternalTable::write()
std::cerr << "format " << format << std::endl;
std::cerr << "structure: \n";
for (const auto & elem : structure)
std::cerr << "\t" << elem.first << " " << elem.second << std::endl;
}
std::vector<std::string> BaseExternalTable::split(const std::string & s, const std::string & d)
{
std::vector<std::string> res;
boost::split(res, s, boost::algorithm::is_any_of(d), boost::algorithm::token_compress_on);
return res;
std::cerr << '\t' << elem.first << ' ' << elem.second << std::endl;
}
void BaseExternalTable::parseStructureFromStructureField(const std::string & argument)
{
std::vector<std::string> vals = split(argument, " ,");
std::vector<std::string> vals;
splitInto<' ', ','>(vals, argument, true);
if (vals.size() & 1)
throw Exception("Odd number of attributes in section structure", ErrorCodes::BAD_ARGUMENTS);
@ -81,7 +74,8 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg
void BaseExternalTable::parseStructureFromTypesField(const std::string & argument)
{
std::vector<std::string> vals = split(argument, " ,");
std::vector<std::string> vals;
splitInto<' ', ','>(vals, argument, true);
for (size_t i = 0; i < vals.size(); ++i)
structure.emplace_back("_" + toString(i + 1), vals[i]);

View File

@ -66,8 +66,6 @@ protected:
/// Function for debugging information output
void write();
static std::vector<std::string> split(const std::string & s, const std::string & d);
/// Construct the `structure` vector from the text field `structure`
virtual void parseStructureFromStructureField(const std::string & argument);

View File

@ -19,7 +19,7 @@ namespace DB
/* Database to store StorageDictionary tables
* automatically creates tables for all dictionaries
*/
class DatabaseDictionary : public IDatabase
class DatabaseDictionary final : public IDatabase
{
public:
DatabaseDictionary(const String & name_);

View File

@ -15,7 +15,7 @@ class DatabaseLazyIterator;
* Works like DatabaseOrdinary, but stores in memory only cache.
* Can be used only with *Log engines.
*/
class DatabaseLazy : public DatabaseOnDisk
class DatabaseLazy final : public DatabaseOnDisk
{
public:
DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_);

View File

@ -16,7 +16,7 @@ namespace DB
* All tables are created by calling code.
* TODO: Maybe DatabaseRuntime is more suitable class name.
*/
class DatabaseMemory : public DatabaseWithOwnTablesBase
class DatabaseMemory final : public DatabaseWithOwnTablesBase
{
public:
DatabaseMemory(const String & name_);

View File

@ -17,7 +17,7 @@ namespace DB
* It doesn't make any manipulations with filesystem.
* All tables are created by calling code after real-time pull-out structure from remote MySQL
*/
class DatabaseMySQL : public IDatabase
class DatabaseMySQL final : public IDatabase
{
public:
~DatabaseMySQL() override;

View File

@ -11,7 +11,7 @@ namespace DB
* It stores tables list in filesystem using list of .sql files,
* that contain declaration of table represented by SQL ATTACH TABLE query.
*/
class DatabaseOrdinary : public DatabaseWithDictionaries //DatabaseWithOwnTablesBase
class DatabaseOrdinary final : public DatabaseWithDictionaries
{
public:
DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context);

View File

@ -44,7 +44,7 @@ public:
};
/// Copies list of tables and iterates through such snapshot.
class DatabaseTablesSnapshotIterator : public IDatabaseTablesIterator
class DatabaseTablesSnapshotIterator final : public IDatabaseTablesIterator
{
private:
Tables tables;

View File

@ -187,8 +187,7 @@ void IPolygonDictionary::createAttributes()
appendNullValue(attr.underlying_type, attr.null_value);
if (attr.hierarchical)
throw Exception{name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
ErrorCodes::TYPE_MISMATCH};
throw Exception{name + ": hierarchical attributes not supported for dictionary of polygonal type", ErrorCodes::TYPE_MISMATCH};
}
}

View File

@ -21,7 +21,7 @@ namespace ErrorCodes
}
class DiskMemoryDirectoryIterator : public IDiskDirectoryIterator
class DiskMemoryDirectoryIterator final : public IDiskDirectoryIterator
{
public:
explicit DiskMemoryDirectoryIterator(std::vector<Poco::Path> && dir_file_paths_)
@ -42,8 +42,9 @@ private:
std::vector<Poco::Path>::iterator iter;
};
/// Adapter with actual behaviour as ReadBufferFromString.
class ReadIndirectBuffer : public ReadBufferFromFileBase
class ReadIndirectBuffer final : public ReadBufferFromFileBase
{
public:
ReadIndirectBuffer(String path_, const String & data_)
@ -71,8 +72,9 @@ private:
const String path;
};
/// This class is responsible to update files metadata after buffer is finalized.
class WriteIndirectBuffer : public WriteBufferFromFileBase
class WriteIndirectBuffer final : public WriteBufferFromFileBase
{
public:
WriteIndirectBuffer(DiskMemory * disk_, String path_, WriteMode mode_, size_t buf_size)

View File

@ -147,7 +147,7 @@ namespace
};
/// Reads data from S3 using stored paths in metadata.
class ReadIndirectBufferFromS3 : public ReadBufferFromFileBase
class ReadIndirectBufferFromS3 final : public ReadBufferFromFileBase
{
public:
ReadIndirectBufferFromS3(
@ -260,7 +260,7 @@ namespace
};
/// Stores data in S3 and adds the object key (S3 path) and object size to metadata file on local FS.
class WriteIndirectBufferFromS3 : public WriteBufferFromFileBase
class WriteIndirectBufferFromS3 final : public WriteBufferFromFileBase
{
public:
WriteIndirectBufferFromS3(
@ -333,7 +333,7 @@ namespace
}
class DiskS3DirectoryIterator : public IDiskDirectoryIterator
class DiskS3DirectoryIterator final : public IDiskDirectoryIterator
{
public:
DiskS3DirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {}
@ -360,7 +360,7 @@ private:
using DiskS3Ptr = std::shared_ptr<DiskS3>;
class DiskS3Reservation : public IReservation
class DiskS3Reservation final : public IReservation
{
public:
DiskS3Reservation(const DiskS3Ptr & disk_, UInt64 size_)

View File

@ -1,3 +1,5 @@
#ifndef __clang_analyzer__ // It's too hard to analyze.
#include "GatherUtils.h"
#include "Selectors.h"
#include "Algorithms.h"
@ -18,3 +20,5 @@ void sliceDynamicOffsetBounded(IArraySource & src, IArraySink & sink, const ICol
SliceDynamicOffsetBoundedSelectArraySource::select(src, sink, offset_column, length_column);
}
}
#endif

View File

@ -1,3 +1,5 @@
#ifndef __clang_analyzer__ // It's too hard to analyze.
#include "GatherUtils.h"
#include "Selectors.h"
#include "Algorithms.h"
@ -19,3 +21,5 @@ void sliceDynamicOffsetUnbounded(IArraySource & src, IArraySink & sink, const IC
SliceDynamicOffsetUnboundedSelectArraySource::select(src, sink, offset_column);
}
}
#endif

View File

@ -1,3 +1,5 @@
#ifndef __clang_analyzer__ // It's too hard to analyze.
#include "GatherUtils.h"
#include "Selectors.h"
#include "Algorithms.h"
@ -19,3 +21,5 @@ void sliceFromLeftConstantOffsetBounded(IArraySource & src, IArraySink & sink, s
SliceFromLeftConstantOffsetBoundedSelectArraySource::select(src, sink, offset, length);
}
}
#endif

View File

@ -1,3 +1,5 @@
#ifndef __clang_analyzer__ // It's too hard to analyze.
#include "GatherUtils.h"
#include "Selectors.h"
#include "Algorithms.h"
@ -19,3 +21,5 @@ void sliceFromLeftConstantOffsetUnbounded(IArraySource & src, IArraySink & sink,
SliceFromLeftConstantOffsetUnboundedSelectArraySource::select(src, sink, offset);
}
}
#endif

View File

@ -1,3 +1,5 @@
#ifndef __clang_analyzer__ // It's too hard to analyze.
#include "GatherUtils.h"
#include "Selectors.h"
#include "Algorithms.h"
@ -19,3 +21,5 @@ void sliceFromRightConstantOffsetBounded(IArraySource & src, IArraySink & sink,
SliceFromRightConstantOffsetBoundedSelectArraySource::select(src, sink, offset, length);
}
}
#endif

View File

@ -1,3 +1,5 @@
#ifndef __clang_analyzer__ // It's too hard to analyze.
#include "GatherUtils.h"
#include "Selectors.h"
#include "Algorithms.h"
@ -19,3 +21,5 @@ void sliceFromRightConstantOffsetUnbounded(IArraySource & src, IArraySink & sink
SliceFromRightConstantOffsetUnboundedSelectArraySource::select(src, sink, offset);
}
}
#endif

View File

@ -36,10 +36,7 @@ ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments)
if (type_array)
{
if (depths.size() < array_num && prev_array_depth)
{
depths.emplace_back(prev_array_depth);
prev_array_depth = 0;
}
prev_array_depth = type_array->getNumberOfDimensions();
++array_num;

View File

@ -41,6 +41,8 @@ int main(int argc, char ** argv)
using T = UInt8;
size_t n = std::stol(argv[1]);
assert(n > 0);
std::vector<T> data(n);
std::vector<T> data2(n);

View File

@ -28,7 +28,7 @@ namespace DB
/// Simplified version of the StorageDistributed class.
class StorageDistributedFake : public ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
class StorageDistributedFake final : public ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
{
friend struct ext::shared_ptr_helper<StorageDistributedFake>;
public:

View File

@ -9,9 +9,8 @@
#include <capnp/serialize.h>
#include <capnp/dynamic.h>
#include <capnp/common.h>
#include <boost/algorithm/string.hpp>
#include <boost/range/join.hpp>
#include <common/logger_useful.h>
#include <common/find_symbols.h>
namespace DB
@ -33,7 +32,7 @@ static CapnProtoRowInputFormat::NestedField split(const Block & header, size_t i
if (!name.empty() && name[0] == '.')
name.erase(0, 1);
boost::split(field.tokens, name, boost::is_any_of("._"));
splitInto<'.', '_'>(field.tokens, name);
return field;
}

View File

@ -93,7 +93,7 @@ void FilterTransform::removeFilterIfNeed(Chunk & chunk)
void FilterTransform::transform(Chunk & chunk)
{
size_t num_rows_before_filtration = chunk.getNumRows();
size_t num_rows_before_filtration;
auto columns = chunk.detachColumns();
{

View File

@ -24,7 +24,7 @@ namespace DB
/** Implements a Kafka queue table engine that can be used as a persistent queue / buffer,
* or as a basic building block for creating pipelines with a continuous insertion / ETL.
*/
class StorageKafka : public ext::shared_ptr_helper<StorageKafka>, public IStorage
class StorageKafka final : public ext::shared_ptr_helper<StorageKafka>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageKafka>;
public:

View File

@ -38,7 +38,7 @@ using ASTPtr = std::shared_ptr<IAST>;
using BlocksMetadataPtr = std::shared_ptr<BlocksMetadata>;
using MergeableBlocksPtr = std::shared_ptr<MergeableBlocks>;
class StorageLiveView : public ext::shared_ptr_helper<StorageLiveView>, public IStorage
class StorageLiveView final : public ext::shared_ptr_helper<StorageLiveView>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageLiveView>;
friend class LiveViewBlockInputStream;

View File

@ -3515,13 +3515,13 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con
}
}
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage * source_table) const
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table) const
{
MergeTreeData * src_data = dynamic_cast<MergeTreeData *>(source_table);
MergeTreeData * src_data = dynamic_cast<MergeTreeData *>(&source_table);
if (!src_data)
throw Exception("Table " + source_table->getStorageID().getNameForLogs() +
throw Exception("Table " + source_table.getStorageID().getNameForLogs() +
" supports attachPartitionFrom only for MergeTree family of table engines."
" Got " + source_table->getName(), ErrorCodes::NOT_IMPLEMENTED);
" Got " + source_table.getName(), ErrorCodes::NOT_IMPLEMENTED);
if (getColumns().getAllPhysical().sizeOfDifference(src_data->getColumns().getAllPhysical()))
throw Exception("Tables have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS);
@ -3545,7 +3545,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage * sour
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const
{
return checkStructureAndGetMergeTreeData(source_table.get());
return checkStructureAndGetMergeTreeData(*source_table);
}
MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(const MergeTreeData::DataPartPtr & src_part,

View File

@ -644,7 +644,7 @@ public:
/// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM
/// Tables structure should be locked.
MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const;
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage * source_table) const;
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table) const;
MergeTreeData::MutableDataPartPtr cloneAndLoadDataPartOnSameDisk(
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info);

View File

@ -13,7 +13,7 @@ namespace DB
{
/// A Storage that allows reading from a single MergeTree data part.
class StorageFromMergeTreeDataPart : public ext::shared_ptr_helper<StorageFromMergeTreeDataPart>, public IStorage
class StorageFromMergeTreeDataPart final : public ext::shared_ptr_helper<StorageFromMergeTreeDataPart>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageFromMergeTreeDataPart>;
public:

View File

@ -37,7 +37,7 @@ class Context;
* When you destroy a Buffer table, all remaining data is flushed to the subordinate table.
* The data in the buffer is not replicated, not logged to disk, not indexed. With a rough restart of the server, the data is lost.
*/
class StorageBuffer : public ext::shared_ptr_helper<StorageBuffer>, public IStorage
class StorageBuffer final : public ext::shared_ptr_helper<StorageBuffer>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageBuffer>;
friend class BufferSource;

View File

@ -19,7 +19,7 @@ struct DictionaryStructure;
struct IDictionaryBase;
class ExternalDictionaries;
class StorageDictionary : public ext::shared_ptr_helper<StorageDictionary>, public IStorage
class StorageDictionary final : public ext::shared_ptr_helper<StorageDictionary>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageDictionary>;
public:

View File

@ -31,7 +31,7 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
* You can pass one address, not several.
* In this case, the table can be considered remote, rather than distributed.
*/
class StorageDistributed : public ext::shared_ptr_helper<StorageDistributed>, public IStorage
class StorageDistributed final : public ext::shared_ptr_helper<StorageDistributed>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageDistributed>;
friend class DistributedBlockOutputStream;

View File

@ -18,7 +18,7 @@ namespace DB
class StorageFileBlockInputStream;
class StorageFileBlockOutputStream;
class StorageFile : public ext::shared_ptr_helper<StorageFile>, public IStorage
class StorageFile final : public ext::shared_ptr_helper<StorageFile>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageFile>;
public:

View File

@ -9,7 +9,7 @@ namespace DB
{
/* Generates random data for given schema.
*/
class StorageGenerateRandom : public ext::shared_ptr_helper<StorageGenerateRandom>, public IStorage
class StorageGenerateRandom final : public ext::shared_ptr_helper<StorageGenerateRandom>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageGenerateRandom>;
public:

View File

@ -13,7 +13,7 @@ namespace DB
* This class represents table engine for external hdfs files.
* Read method is supported for now.
*/
class StorageHDFS : public ext::shared_ptr_helper<StorageHDFS>, public IStorage
class StorageHDFS final : public ext::shared_ptr_helper<StorageHDFS>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageHDFS>;
public:

View File

@ -8,7 +8,7 @@ namespace DB
/** Internal temporary storage for table function input(...)
*/
class StorageInput : public ext::shared_ptr_helper<StorageInput>, public IStorage
class StorageInput final : public ext::shared_ptr_helper<StorageInput>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageInput>;
public:

View File

@ -21,7 +21,7 @@ using HashJoinPtr = std::shared_ptr<Join>;
*
* When using, JOIN must be of the appropriate type (ANY|ALL LEFT|INNER ...).
*/
class StorageJoin : public ext::shared_ptr_helper<StorageJoin>, public StorageSetOrJoinBase
class StorageJoin final : public ext::shared_ptr_helper<StorageJoin>, public StorageSetOrJoinBase
{
friend struct ext::shared_ptr_helper<StorageJoin>;
public:

View File

@ -15,7 +15,7 @@ namespace DB
/** Implements simple table engine without support of indices.
* The data is stored in a compressed form.
*/
class StorageLog : public ext::shared_ptr_helper<StorageLog>, public IStorage
class StorageLog final : public ext::shared_ptr_helper<StorageLog>, public IStorage
{
friend class LogSource;
friend class LogBlockOutputStream;

View File

@ -12,7 +12,7 @@
namespace DB
{
class StorageMaterializedView : public ext::shared_ptr_helper<StorageMaterializedView>, public IStorage
class StorageMaterializedView final : public ext::shared_ptr_helper<StorageMaterializedView>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageMaterializedView>;
public:

View File

@ -17,7 +17,7 @@ namespace DB
* It does not support keys.
* Data is stored as a set of blocks and is not stored anywhere else.
*/
class StorageMemory : public ext::shared_ptr_helper<StorageMemory>, public IStorage
class StorageMemory final : public ext::shared_ptr_helper<StorageMemory>, public IStorage
{
friend class MemoryBlockInputStream;
friend class MemoryBlockOutputStream;

View File

@ -13,7 +13,7 @@ namespace DB
/** A table that represents the union of an arbitrary number of other tables.
* All tables must have the same structure.
*/
class StorageMerge : public ext::shared_ptr_helper<StorageMerge>, public IStorage
class StorageMerge final : public ext::shared_ptr_helper<StorageMerge>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageMerge>;
public:

View File

@ -1238,7 +1238,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
dest_table_storage->getStorageID().getNameForLogs() + ": " + dest_table_storage->getStoragePolicy()->getName(), ErrorCodes::LOGICAL_ERROR);
Stopwatch watch;
MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(this);
MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this);
String partition_id = getPartitionIDFromQuery(partition, context);
DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);

View File

@ -23,7 +23,7 @@ namespace DB
/** See the description of the data structure in MergeTreeData.
*/
class StorageMergeTree : public ext::shared_ptr_helper<StorageMergeTree>, public MergeTreeData
class StorageMergeTree final : public ext::shared_ptr_helper<StorageMergeTree>, public MergeTreeData
{
friend struct ext::shared_ptr_helper<StorageMergeTree>;
public:

View File

@ -17,7 +17,7 @@ namespace DB
* Use ENGINE = mysql(host_port, database_name, table_name, user_name, password)
* Read only.
*/
class StorageMySQL : public ext::shared_ptr_helper<StorageMySQL>, public IStorage
class StorageMySQL final : public ext::shared_ptr_helper<StorageMySQL>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageMySQL>;
public:

View File

@ -16,7 +16,7 @@ namespace DB
/** When writing, does nothing.
* When reading, returns nothing.
*/
class StorageNull : public ext::shared_ptr_helper<StorageNull>, public IStorage
class StorageNull final : public ext::shared_ptr_helper<StorageNull>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageNull>;
public:

View File

@ -5051,7 +5051,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
getStorageID().getNameForLogs() + ": " + dest_table_storage->getStoragePolicy()->getName(), ErrorCodes::LOGICAL_ERROR);
Stopwatch watch;
MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(this);
MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this);
auto src_data_id = src_data.getStorageID();
String partition_id = getPartitionIDFromQuery(partition, context);

View File

@ -73,7 +73,7 @@ namespace DB
* as the time will take the time of creation the appropriate part on any of the replicas.
*/
class StorageReplicatedMergeTree : public ext::shared_ptr_helper<StorageReplicatedMergeTree>, public MergeTreeData
class StorageReplicatedMergeTree final : public ext::shared_ptr_helper<StorageReplicatedMergeTree>, public MergeTreeData
{
friend struct ext::shared_ptr_helper<StorageReplicatedMergeTree>;
public:

View File

@ -22,7 +22,7 @@ namespace DB
* It sends HTTP GET to server when select is called and
* HTTP PUT when insert is called.
*/
class StorageS3 : public ext::shared_ptr_helper<StorageS3>, public IStorage
class StorageS3 final : public ext::shared_ptr_helper<StorageS3>, public IStorage
{
public:
StorageS3(const S3::URI & uri,

View File

@ -57,7 +57,7 @@ private:
* and also written to a file-backup, for recovery after a restart.
* Reading from the table is not possible directly - it is possible to specify only the right part of the IN statement.
*/
class StorageSet : public ext::shared_ptr_helper<StorageSet>, public StorageSetOrJoinBase
class StorageSet final : public ext::shared_ptr_helper<StorageSet>, public StorageSetOrJoinBase
{
friend struct ext::shared_ptr_helper<StorageSet>;

View File

@ -16,7 +16,7 @@ namespace DB
/** Implements a table engine that is suitable for small chunks of the log.
* In doing so, stores all the columns in a single Native file, with a nearby index.
*/
class StorageStripeLog : public ext::shared_ptr_helper<StorageStripeLog>, public IStorage
class StorageStripeLog final : public ext::shared_ptr_helper<StorageStripeLog>, public IStorage
{
friend class StripeLogSource;
friend class StripeLogBlockOutputStream;

View File

@ -15,7 +15,7 @@ namespace DB
/** Implements a table engine that is suitable for small chunks of the log.
* It differs from StorageLog in the absence of mark files.
*/
class StorageTinyLog : public ext::shared_ptr_helper<StorageTinyLog>, public IStorage
class StorageTinyLog final : public ext::shared_ptr_helper<StorageTinyLog>, public IStorage
{
friend class TinyLogSource;
friend class TinyLogBlockOutputStream;

View File

@ -63,7 +63,7 @@ private:
};
class StorageURL : public ext::shared_ptr_helper<StorageURL>, public IStorageURLBase
class StorageURL final : public ext::shared_ptr_helper<StorageURL>, public IStorageURLBase
{
friend struct ext::shared_ptr_helper<StorageURL>;
public:

View File

@ -9,7 +9,7 @@ namespace DB
/* One block storage used for values table function
* It's structure is similar to IStorageSystemOneBlock
*/
class StorageValues : public ext::shared_ptr_helper<StorageValues>, public IStorage
class StorageValues final : public ext::shared_ptr_helper<StorageValues>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageValues>;
public:

View File

@ -10,7 +10,7 @@
namespace DB
{
class StorageView : public ext::shared_ptr_helper<StorageView>, public IStorage
class StorageView final : public ext::shared_ptr_helper<StorageView>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageView>;
public:

View File

@ -7,7 +7,7 @@
namespace DB
{
class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper<StorageSystemAggregateFunctionCombinators>,
class StorageSystemAggregateFunctionCombinators final : public ext::shared_ptr_helper<StorageSystemAggregateFunctionCombinators>,
public IStorageSystemOneBlock<StorageSystemAggregateFunctionCombinators>
{
friend struct ext::shared_ptr_helper<StorageSystemAggregateFunctionCombinators>;

View File

@ -12,7 +12,7 @@ class Context;
/** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics.
*/
class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>,
class StorageSystemAsynchronousMetrics final : public ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>,
public IStorageSystemOneBlock<StorageSystemAsynchronousMetrics>
{
friend struct ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>;

View File

@ -12,7 +12,7 @@ class Context;
/** System table "build_options" with many params used for clickhouse building
*/
class StorageSystemBuildOptions : public ext::shared_ptr_helper<StorageSystemBuildOptions>, public IStorageSystemOneBlock<StorageSystemBuildOptions>
class StorageSystemBuildOptions final : public ext::shared_ptr_helper<StorageSystemBuildOptions>, public IStorageSystemOneBlock<StorageSystemBuildOptions>
{
friend struct ext::shared_ptr_helper<StorageSystemBuildOptions>;
protected:

View File

@ -15,7 +15,7 @@ class Context;
* that allows to obtain information about available clusters
* (which may be specified in Distributed tables).
*/
class StorageSystemClusters : public ext::shared_ptr_helper<StorageSystemClusters>, public IStorageSystemOneBlock<StorageSystemClusters>
class StorageSystemClusters final : public ext::shared_ptr_helper<StorageSystemClusters>, public IStorageSystemOneBlock<StorageSystemClusters>
{
friend struct ext::shared_ptr_helper<StorageSystemClusters>;
public:

View File

@ -5,7 +5,7 @@
namespace DB
{
class StorageSystemCollations : public ext::shared_ptr_helper<StorageSystemCollations>,
class StorageSystemCollations final : public ext::shared_ptr_helper<StorageSystemCollations>,
public IStorageSystemOneBlock<StorageSystemCollations>
{
friend struct ext::shared_ptr_helper<StorageSystemCollations>;

View File

@ -11,7 +11,7 @@ class Context;
/** Implements system table 'columns', that allows to get information about columns for every table.
*/
class StorageSystemColumns : public ext::shared_ptr_helper<StorageSystemColumns>, public IStorage
class StorageSystemColumns final : public ext::shared_ptr_helper<StorageSystemColumns>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemColumns>;
public:

View File

@ -11,7 +11,7 @@ class Context;
/** System table "contributors" with list of clickhouse contributors
*/
class StorageSystemContributors : public ext::shared_ptr_helper<StorageSystemContributors>,
class StorageSystemContributors final : public ext::shared_ptr_helper<StorageSystemContributors>,
public IStorageSystemOneBlock<StorageSystemContributors>
{
friend struct ext::shared_ptr_helper<StorageSystemContributors>;

View File

@ -6,7 +6,7 @@
namespace DB
{
class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper<StorageSystemDataTypeFamilies>,
class StorageSystemDataTypeFamilies final : public ext::shared_ptr_helper<StorageSystemDataTypeFamilies>,
public IStorageSystemOneBlock<StorageSystemDataTypeFamilies>
{
friend struct ext::shared_ptr_helper<StorageSystemDataTypeFamilies>;

View File

@ -12,7 +12,7 @@ class Context;
/** Implements `databases` system table, which allows you to get information about all databases.
*/
class StorageSystemDatabases : public ext::shared_ptr_helper<StorageSystemDatabases>, public IStorageSystemOneBlock<StorageSystemDatabases>
class StorageSystemDatabases final : public ext::shared_ptr_helper<StorageSystemDatabases>, public IStorageSystemOneBlock<StorageSystemDatabases>
{
friend struct ext::shared_ptr_helper<StorageSystemDatabases>;
public:

View File

@ -18,7 +18,7 @@ namespace DB
* We don't use StorageSystemPartsBase, because it introduces virtual _state
* column and column aliases which we don't need.
*/
class StorageSystemDetachedParts :
class StorageSystemDetachedParts final :
public ext::shared_ptr_helper<StorageSystemDetachedParts>,
public IStorage
{

View File

@ -10,7 +10,7 @@ namespace DB
class Context;
class StorageSystemDictionaries : public ext::shared_ptr_helper<StorageSystemDictionaries>, public IStorageSystemOneBlock<StorageSystemDictionaries>
class StorageSystemDictionaries final : public ext::shared_ptr_helper<StorageSystemDictionaries>, public IStorageSystemOneBlock<StorageSystemDictionaries>
{
friend struct ext::shared_ptr_helper<StorageSystemDictionaries>;
public:

View File

@ -14,7 +14,7 @@ class Context;
/** Implements the system table `disks`, which allows you to get information about all disks.
*/
class StorageSystemDisks : public ext::shared_ptr_helper<StorageSystemDisks>, public IStorage
class StorageSystemDisks final : public ext::shared_ptr_helper<StorageSystemDisks>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemDisks>;
public:

View File

@ -11,7 +11,7 @@ class Context;
/** Implements `events` system table, which allows you to obtain information for profiling.
*/
class StorageSystemEvents : public ext::shared_ptr_helper<StorageSystemEvents>, public IStorageSystemOneBlock<StorageSystemEvents>
class StorageSystemEvents final : public ext::shared_ptr_helper<StorageSystemEvents>, public IStorageSystemOneBlock<StorageSystemEvents>
{
friend struct ext::shared_ptr_helper<StorageSystemEvents>;
public:

View File

@ -5,7 +5,7 @@
namespace DB
{
class StorageSystemFormats : public ext::shared_ptr_helper<StorageSystemFormats>, public IStorageSystemOneBlock<StorageSystemFormats>
class StorageSystemFormats final : public ext::shared_ptr_helper<StorageSystemFormats>, public IStorageSystemOneBlock<StorageSystemFormats>
{
friend struct ext::shared_ptr_helper<StorageSystemFormats>;
protected:

View File

@ -13,7 +13,7 @@ class Context;
/** Implements `functions`system table, which allows you to get a list
* all normal and aggregate functions.
*/
class StorageSystemFunctions : public ext::shared_ptr_helper<StorageSystemFunctions>, public IStorageSystemOneBlock<StorageSystemFunctions>
class StorageSystemFunctions final : public ext::shared_ptr_helper<StorageSystemFunctions>, public IStorageSystemOneBlock<StorageSystemFunctions>
{
friend struct ext::shared_ptr_helper<StorageSystemFunctions>;
public:

View File

@ -11,7 +11,7 @@ namespace DB
{
/// Provides information about Graphite configuration.
class StorageSystemGraphite : public ext::shared_ptr_helper<StorageSystemGraphite>, public IStorageSystemOneBlock<StorageSystemGraphite>
class StorageSystemGraphite final : public ext::shared_ptr_helper<StorageSystemGraphite>, public IStorageSystemOneBlock<StorageSystemGraphite>
{
friend struct ext::shared_ptr_helper<StorageSystemGraphite>;
public:

View File

@ -13,7 +13,7 @@ class Context;
/** Information about macros for introspection.
*/
class StorageSystemMacros : public ext::shared_ptr_helper<StorageSystemMacros>, public IStorageSystemOneBlock<StorageSystemMacros>
class StorageSystemMacros final : public ext::shared_ptr_helper<StorageSystemMacros>, public IStorageSystemOneBlock<StorageSystemMacros>
{
friend struct ext::shared_ptr_helper<StorageSystemMacros>;
public:

View File

@ -13,7 +13,7 @@ class Context;
/** implements system table "merge_tree_settings", which allows to get information about the current MergeTree settings.
*/
class SystemMergeTreeSettings : public ext::shared_ptr_helper<SystemMergeTreeSettings>, public IStorageSystemOneBlock<SystemMergeTreeSettings>
class SystemMergeTreeSettings final : public ext::shared_ptr_helper<SystemMergeTreeSettings>, public IStorageSystemOneBlock<SystemMergeTreeSettings>
{
friend struct ext::shared_ptr_helper<SystemMergeTreeSettings>;
public:

View File

@ -13,7 +13,7 @@ namespace DB
class Context;
class StorageSystemMerges : public ext::shared_ptr_helper<StorageSystemMerges>, public IStorageSystemOneBlock<StorageSystemMerges>
class StorageSystemMerges final : public ext::shared_ptr_helper<StorageSystemMerges>, public IStorageSystemOneBlock<StorageSystemMerges>
{
friend struct ext::shared_ptr_helper<StorageSystemMerges>;
public:

View File

@ -12,7 +12,7 @@ class Context;
/** Implements `metrics` system table, which provides information about the operation of the server.
*/
class StorageSystemMetrics : public ext::shared_ptr_helper<StorageSystemMetrics>, public IStorageSystemOneBlock<StorageSystemMetrics>
class StorageSystemMetrics final : public ext::shared_ptr_helper<StorageSystemMetrics>, public IStorageSystemOneBlock<StorageSystemMetrics>
{
friend struct ext::shared_ptr_helper<StorageSystemMetrics>;
public:

View File

@ -10,7 +10,7 @@ namespace DB
class Context;
class StorageSystemModels : public ext::shared_ptr_helper<StorageSystemModels>, public IStorageSystemOneBlock<StorageSystemModels>
class StorageSystemModels final : public ext::shared_ptr_helper<StorageSystemModels>, public IStorageSystemOneBlock<StorageSystemModels>
{
friend struct ext::shared_ptr_helper<StorageSystemModels>;
public:

View File

@ -12,7 +12,7 @@ class Context;
/// Implements the `mutations` system table, which provides information about the status of mutations
/// in the MergeTree tables.
class StorageSystemMutations : public ext::shared_ptr_helper<StorageSystemMutations>, public IStorageSystemOneBlock<StorageSystemMutations>
class StorageSystemMutations final : public ext::shared_ptr_helper<StorageSystemMutations>, public IStorageSystemOneBlock<StorageSystemMutations>
{
friend struct ext::shared_ptr_helper<StorageSystemMutations>;
public:

View File

@ -23,7 +23,7 @@ class Context;
* In multithreaded case, if even_distributed is False, implementation with atomic is used,
* and result is always in [0 ... limit - 1] range.
*/
class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
class StorageSystemNumbers final : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemNumbers>;
public:

View File

@ -15,7 +15,7 @@ class Context;
* Used when the table is not specified in the query.
* Analog of the DUAL table in Oracle and MySQL.
*/
class StorageSystemOne : public ext::shared_ptr_helper<StorageSystemOne>, public IStorage
class StorageSystemOne final : public ext::shared_ptr_helper<StorageSystemOne>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemOne>;
public:

View File

@ -12,7 +12,7 @@ class Context;
/** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family.
*/
class StorageSystemParts : public ext::shared_ptr_helper<StorageSystemParts>, public StorageSystemPartsBase
class StorageSystemParts final : public ext::shared_ptr_helper<StorageSystemParts>, public StorageSystemPartsBase
{
friend struct ext::shared_ptr_helper<StorageSystemParts>;
public:

View File

@ -13,7 +13,7 @@ class Context;
/** Implements system table 'parts_columns' which allows to get information about
* columns in data parts for tables of MergeTree family.
*/
class StorageSystemPartsColumns
class StorageSystemPartsColumns final
: public ext::shared_ptr_helper<StorageSystemPartsColumns>, public StorageSystemPartsBase
{
friend struct ext::shared_ptr_helper<StorageSystemPartsColumns>;

View File

@ -12,7 +12,7 @@ class Context;
/** Implements `processes` system table, which allows you to get information about the queries that are currently executing.
*/
class StorageSystemProcesses : public ext::shared_ptr_helper<StorageSystemProcesses>, public IStorageSystemOneBlock<StorageSystemProcesses>
class StorageSystemProcesses final : public ext::shared_ptr_helper<StorageSystemProcesses>, public IStorageSystemOneBlock<StorageSystemProcesses>
{
friend struct ext::shared_ptr_helper<StorageSystemProcesses>;
public:

View File

@ -13,7 +13,7 @@ class Context;
/** Implements the `quota_usage` system tables, which allows you to get information about
* how the quotas are used by all users.
*/
class StorageSystemQuotaUsage : public ext::shared_ptr_helper<StorageSystemQuotaUsage>, public IStorageSystemOneBlock<StorageSystemQuotaUsage>
class StorageSystemQuotaUsage final : public ext::shared_ptr_helper<StorageSystemQuotaUsage>, public IStorageSystemOneBlock<StorageSystemQuotaUsage>
{
public:
std::string getName() const override { return "SystemQuotaUsage"; }

Some files were not shown because too many files have changed in this diff Show More