More GCC warnings

This commit is contained in:
kreuzerkrieg 2019-12-15 08:34:43 +02:00
parent fcca420e91
commit 9612bb0e79
307 changed files with 1195 additions and 896 deletions

View File

@ -49,6 +49,9 @@ set (CAPNP_SRCS
)
add_library(capnp ${CAPNP_SRCS})
set_target_properties(capnp
PROPERTIES LINKER_LANGUAGE CXX
)
target_link_libraries(capnp PUBLIC kj)
# The library has substandard code

View File

@ -1,3 +1,6 @@
set(ConfigIncludePath ${CMAKE_CURRENT_BINARY_DIR}/includes/configs CACHE INTERNAL "Path to generated configuration files.")
include_directories(${ConfigIncludePath})
if (USE_INCLUDE_WHAT_YOU_USE)
set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH})
endif ()
@ -72,8 +75,83 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
endif ()
endif ()
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wshadow")
endif()
# Add compiler options only to c++ compiler
function(add_cxx_compile_options option)
add_compile_options(
"$<$<STREQUAL:$<TARGET_PROPERTY:LINKER_LANGUAGE>,CXX>:${option}>"
)
endfunction()
# Warn about boolean expression compared with an integer value different from true/false
add_cxx_compile_options(-Wbool-compare)
# Warn whenever a pointer is cast such that the required alignment of the target is increased.
add_cxx_compile_options(-Wcast-align)
# Warn whenever a pointer is cast so as to remove a type qualifier from the target type.
add_cxx_compile_options(-Wcast-qual)
# Warn when deleting a pointer to incomplete type, which may cause undefined behavior at runtime
add_cxx_compile_options(-Wdelete-incomplete)
# Warn if a requested optimization pass is disabled. Code is too big or too complex
add_cxx_compile_options(-Wdisabled-optimization)
# Warn about duplicated conditions in an if-else-if chain
add_cxx_compile_options(-Wduplicated-cond)
# Warn about a comparison between values of different enumerated types
add_cxx_compile_options(-Wenum-compare)
# Warn about uninitialized variables that are initialized with themselves
add_cxx_compile_options(-Winit-self)
# Warn about logical not used on the left hand side operand of a comparison
add_cxx_compile_options(-Wlogical-not-parentheses)
# Warn about suspicious uses of logical operators in expressions
add_cxx_compile_options(-Wlogical-op)
# Warn if there exists a path from the function entry to a use of the variable that is uninitialized.
add_cxx_compile_options(-Wmaybe-uninitialized)
# Warn when the indentation of the code does not reflect the block structure
add_cxx_compile_options(-Wmisleading-indentation)
# Warn if a global function is defined without a previous declaration
add_cxx_compile_options(-Wmissing-declarations)
# Warn if a user-supplied include directory does not exist
# add_cxx_compile_options(-Wmissing-include-dirs)
# Obvious
add_cxx_compile_options(-Wnon-virtual-dtor)
# Obvious
add_cxx_compile_options(-Wno-return-local-addr)
# Obvious
add_cxx_compile_options(-Wnull-dereference)
# Obvious
add_cxx_compile_options(-Wodr)
# Obvious
add_cxx_compile_options(-Wold-style-cast)
# Warn when a function declaration hides virtual functions from a base class
# add_cxx_compile_options(-Woverloaded-virtual)
# Warn about placement new expressions with undefined behavior
add_cxx_compile_options(-Wplacement-new=2)
# Warn about anything that depends on the size of a function type or of void
add_cxx_compile_options(-Wpointer-arith)
# Warn if anything is declared more than once in the same scope
add_cxx_compile_options(-Wredundant-decls)
# Member initialization reordering
add_cxx_compile_options(-Wreorder)
# Obvious
add_cxx_compile_options(-Wshadow)
# Warn if left shifting a negative value
add_cxx_compile_options(-Wshift-negative-value)
# Warn about a definition of an unsized deallocation function
add_cxx_compile_options(-Wsized-deallocation)
# Warn when the sizeof operator is applied to a parameter that is declared as an array in a function definition
add_cxx_compile_options(-Wsizeof-array-argument)
# Warn for suspicious length parameters to certain string and memory built-in functions if the argument uses sizeof
add_cxx_compile_options(-Wsizeof-pointer-memaccess)
# Warn about overriding virtual functions that are not marked with the override keyword
# add_cxx_compile_options(-Wsuggest-override)
# Warn whenever a switch statement has an index of boolean type and the case values are outside the range of a boolean type
add_cxx_compile_options(-Wswitch-bool)
# Warn if a self-comparison always evaluates to true or false
add_cxx_compile_options(-Wtautological-compare)
# Warn about trampolines generated for pointers to nested functions
add_cxx_compile_options(-Wtrampolines)
# Obvious
add_cxx_compile_options(-Wunused)
# Warn if vector operation is not implemented via SIMD capabilities of the architecture
add_cxx_compile_options(-Wvector-operation-performance)
endif ()
if (USE_DEBUG_HELPERS)
set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/libs/libcommon/include -include ${ClickHouse_SOURCE_DIR}/dbms/src/Core/iostream_debug_helpers.h")

View File

@ -19,7 +19,7 @@ if(NOT (MAKE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES))
set(CLICKHOUSE_ONE_SHARED 1)
endif()
configure_file (config_tools.h.in ${CMAKE_CURRENT_BINARY_DIR}/config_tools.h)
configure_file (config_tools.h.in ${ConfigIncludePath}/config_tools.h)
macro(clickhouse_target_link_split_lib target name)

View File

@ -504,6 +504,7 @@ public:
#ifndef __clang__
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseBenchmark(int argc, char ** argv)
{

View File

@ -1,2 +1,2 @@
int mainEntryClickHouseBenchmark(int argc, char ** argv);
extern int mainEntryClickHouseBenchmark(int argc, char ** argv);
int main(int argc_, char ** argv_) { return mainEntryClickHouseBenchmark(argc_, argv_); }

View File

@ -12,7 +12,7 @@ endif ()
include(CheckSymbolExists)
check_symbol_exists(readpassphrase readpassphrase.h HAVE_READPASSPHRASE)
configure_file(config_client.h.in ${CMAKE_CURRENT_BINARY_DIR}/include/config_client.h)
configure_file(config_client.h.in ${ConfigIncludePath}/config_client.h)
if(NOT HAVE_READPASSPHRASE)
add_subdirectory(readpassphrase)

View File

@ -1942,6 +1942,9 @@ public:
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseClient(int argc, char ** argv)
{
try

View File

@ -6,5 +6,8 @@ set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-result -Wno-reserved-id-macro")
configure_file(includes.h.in ${CMAKE_CURRENT_BINARY_DIR}/include/includes.h)
add_library(readpassphrase ${CMAKE_CURRENT_SOURCE_DIR}/readpassphrase.c)
set_target_properties(readpassphrase
PROPERTIES LINKER_LANGUAGE C
)
# . to allow #include <readpassphrase.h>
target_include_directories(readpassphrase PUBLIC . ${CMAKE_CURRENT_BINARY_DIR}/include ${CMAKE_CURRENT_BINARY_DIR}/../include)

View File

@ -57,6 +57,8 @@ void checkAndWriteHeader(DB::ReadBuffer & in, DB::WriteBuffer & out)
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseCompressor(int argc, char ** argv)
{

View File

@ -2513,6 +2513,8 @@ int ClusterCopierApp::main(const std::vector<std::string> &)
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseClusterCopier(int argc, char ** argv)
{

View File

@ -44,6 +44,9 @@ static std::string extractFromConfig(
return configuration->getString(key);
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseExtractFromConfig(int argc, char ** argv)
{
bool print_stacktrace = false;

View File

@ -8,6 +8,9 @@
#include <Parsers/formatAST.h>
#include <Common/TerminalSize.h>
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseFormat(int argc, char ** argv)
{
using namespace DB;

View File

@ -499,6 +499,9 @@ void LocalServer::applyCmdOptions()
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseLocal(int argc, char ** argv)
{
DB::LocalServer app;

View File

@ -123,14 +123,14 @@ UInt64 hash(Ts... xs)
}
UInt64 maskBits(UInt64 x, size_t num_bits)
static UInt64 maskBits(UInt64 x, size_t num_bits)
{
return x & ((1ULL << num_bits) - 1);
}
/// Apply Feistel network round to least significant num_bits part of x.
UInt64 feistelRound(UInt64 x, size_t num_bits, UInt64 seed, size_t round)
static UInt64 feistelRound(UInt64 x, size_t num_bits, UInt64 seed, size_t round)
{
size_t num_bits_left_half = num_bits / 2;
size_t num_bits_right_half = num_bits - num_bits_left_half;
@ -146,7 +146,7 @@ UInt64 feistelRound(UInt64 x, size_t num_bits, UInt64 seed, size_t round)
/// Apply Feistel network with num_rounds to least significant num_bits part of x.
UInt64 feistelNetwork(UInt64 x, size_t num_bits, UInt64 seed, size_t num_rounds = 4)
static UInt64 feistelNetwork(UInt64 x, size_t num_bits, UInt64 seed, size_t num_rounds = 4)
{
UInt64 bits = maskBits(x, num_bits);
for (size_t i = 0; i < num_rounds; ++i)
@ -156,7 +156,7 @@ UInt64 feistelNetwork(UInt64 x, size_t num_bits, UInt64 seed, size_t num_rounds
/// Pseudorandom permutation within set of numbers with the same log2(x).
UInt64 transform(UInt64 x, UInt64 seed)
static UInt64 transform(UInt64 x, UInt64 seed)
{
/// Keep 0 and 1 as is.
if (x == 0 || x == 1)
@ -199,7 +199,7 @@ public:
/// Keep sign and apply pseudorandom permutation after converting to unsigned as above.
Int64 transformSigned(Int64 x, UInt64 seed)
static Int64 transformSigned(Int64 x, UInt64 seed)
{
if (x >= 0)
return transform(x, seed);
@ -298,7 +298,7 @@ public:
/// Pseudorandom function, but keep word characters as word characters.
void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UInt64 seed)
static void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UInt64 seed)
{
{
SipHash hash;
@ -943,6 +943,8 @@ public:
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseObfuscator(int argc, char ** argv)
try

View File

@ -35,7 +35,7 @@ using PocoSessionPoolConstructor = std::function<std::shared_ptr<Poco::Data::Ses
/** Is used to adjust max size of default Poco thread pool. See issue #750
* Acquire the lock, resize pool and construct new Session.
*/
std::shared_ptr<Poco::Data::SessionPool> createAndCheckResizePocoSessionPool(PocoSessionPoolConstructor pool_constr)
static std::shared_ptr<Poco::Data::SessionPool> createAndCheckResizePocoSessionPool(PocoSessionPoolConstructor pool_constr)
{
static std::mutex mutex;

View File

@ -193,6 +193,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
}
}
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseODBCBridge(int argc, char ** argv)
{
DB::ODBCBridge app;

View File

@ -294,7 +294,7 @@ static std::vector<std::string> getInputFiles(const po::variables_map & options,
return input_files;
}
std::unordered_map<std::string, std::vector<std::size_t>> getTestQueryIndexes(const po::basic_parsed_options<char> & parsed_opts)
static std::unordered_map<std::string, std::vector<std::size_t>> getTestQueryIndexes(const po::basic_parsed_options<char> & parsed_opts)
{
std::unordered_map<std::string, std::vector<std::size_t>> result;
const auto & options = parsed_opts.options;
@ -319,6 +319,9 @@ std::unordered_map<std::string, std::vector<std::size_t>> getTestQueryIndexes(co
return result;
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHousePerformanceTest(int argc, char ** argv)
try
{

View File

@ -30,7 +30,7 @@ void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVecto
/// Recursive method which goes through all substitution blocks in xml
/// and replaces property {names} by their values
void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left,
static void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left,
StringToVector::iterator substitutions_right,
const std::string & template_query,
Strings & out_queries)

View File

@ -974,6 +974,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
int mainEntryClickHouseServer(int argc, char ** argv)
{
DB::Server app;

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionArray.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionAvg.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionBitwise.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionBoundingRatio.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -3,6 +3,7 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,5 +1,6 @@
#include <Common/StringUtils/StringUtils.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionEntropy.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -16,6 +16,7 @@
#include <Common/typeid_cast.h>
#include <Poco/String.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionForEach.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include "registerAggregateFunctions.h"
// TODO include this last because of a broken roaring header. See the comment
// inside.

View File

@ -11,7 +11,10 @@
// garbage that breaks the build (e.g. it changes _POSIX_C_SOURCE).
// TODO: find out what it is. On github, they have proper inteface headers like
// this one: https://github.com/RoaringBitmap/CRoaring/blob/master/include/roaring/roaring.h
#pragma GCC diagnostic push
#pragma GCC diagnostic warning "-Wold-style-cast"
#include <roaring/roaring.h>
#pragma GCC diagnostic pop
namespace DB
{
@ -602,7 +605,7 @@ private:
/// To read and write the DB Buffer directly, migrate code from CRoaring
void db_roaring_bitmap_add_many(DB::ReadBuffer & dbBuf, roaring_bitmap_t * r, size_t n_args)
{
void * container = NULL; // hold value of last container touched
void * container = nullptr; // hold value of last container touched
uint8_t typecode = 0; // typecode of last container touched
uint32_t prev = 0; // previous valued inserted
size_t i = 0; // index of value
@ -647,7 +650,7 @@ private:
size_t ctr = 0;
for (Int32 i = 0; i < ra->size; ++i)
{
Int32 num_added = db_container_to_uint32_array(dbBuf, ra->containers[i], ra->typecodes[i], ((UInt32)ra->keys[i]) << 16);
Int32 num_added = db_container_to_uint32_array(dbBuf, ra->containers[i], ra->typecodes[i], (static_cast<UInt32>(ra->keys[i])) << 16);
ctr += num_added;
}
}
@ -658,18 +661,18 @@ private:
switch (typecode)
{
case BITSET_CONTAINER_TYPE_CODE:
return db_bitset_container_to_uint32_array(dbBuf, (const bitset_container_t *)container, base);
return db_bitset_container_to_uint32_array(dbBuf, static_cast<const bitset_container_t *>(container), base);
case ARRAY_CONTAINER_TYPE_CODE:
return db_array_container_to_uint32_array(dbBuf, (const array_container_t *)container, base);
return db_array_container_to_uint32_array(dbBuf, static_cast<const array_container_t *>(container), base);
case RUN_CONTAINER_TYPE_CODE:
return db_run_container_to_uint32_array(dbBuf, (const run_container_t *)container, base);
return db_run_container_to_uint32_array(dbBuf, static_cast<const run_container_t *>(container), base);
}
return 0;
}
UInt32 db_bitset_container_to_uint32_array(DB::WriteBuffer & dbBuf, const bitset_container_t * cont, UInt32 base) const
{
return (UInt32)db_bitset_extract_setbits(dbBuf, cont->array, BITSET_CONTAINER_SIZE_IN_WORDS, base);
return static_cast<UInt32>(db_bitset_extract_setbits(dbBuf, cont->array, BITSET_CONTAINER_SIZE_IN_WORDS, base));
}
size_t db_bitset_extract_setbits(DB::WriteBuffer & dbBuf, UInt64 * bitset, size_t length, UInt32 base) const

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/Helpers.h>
#include <Common/FieldVisitors.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -1,5 +1,6 @@
#include <AggregateFunctions/AggregateFunctionIf.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -11,6 +11,7 @@
#include "AggregateFunctionFactory.h"
#include "FactoryHelpers.h"
#include "Helpers.h"
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionMaxIntersections.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionMerge.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/HelpersMinMaxAny.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -3,6 +3,7 @@
#include <AggregateFunctions/AggregateFunctionNothing.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionOrFill.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/Helpers.h>
#include <Core/Field.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -1,6 +1,7 @@
#include <AggregateFunctions/AggregateFunctionResample.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionRetention.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -6,6 +6,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <ext/range.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <Core/TypeListNumber.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionMerge.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/AggregateFunctionStatistics.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/AggregateFunctionStatisticsSimple.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionSum.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <Functions/FunctionHelpers.h>
#include <IO/WriteHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -2,6 +2,7 @@
#include "AggregateFunctionFactory.h"
#include "FactoryHelpers.h"
#include "Helpers.h"
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
#define TOP_K_MAX_SIZE 0xFFFFFF

View File

@ -9,6 +9,7 @@
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -7,6 +7,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <functional>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -5,6 +5,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -6,6 +6,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <ext/range.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -7,45 +7,6 @@
namespace DB
{
void registerAggregateFunctionAvg(AggregateFunctionFactory &);
void registerAggregateFunctionCount(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory &);
void registerAggregateFunctionWindowFunnel(AggregateFunctionFactory &);
void registerAggregateFunctionRate(AggregateFunctionFactory &);
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory &);
void registerAggregateFunctionsStatisticsStable(AggregateFunctionFactory &);
void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &);
void registerAggregateFunctionSum(AggregateFunctionFactory &);
void registerAggregateFunctionSumMap(AggregateFunctionFactory &);
void registerAggregateFunctionsUniq(AggregateFunctionFactory &);
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory &);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &);
void registerAggregateFunctionTopK(AggregateFunctionFactory &);
void registerAggregateFunctionsBitwise(AggregateFunctionFactory &);
void registerAggregateFunctionsBitmap(AggregateFunctionFactory &);
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &);
void registerAggregateFunctionHistogram(AggregateFunctionFactory &);
void registerAggregateFunctionRetention(AggregateFunctionFactory &);
void registerAggregateFunctionTimeSeriesGroupSum(AggregateFunctionFactory &);
void registerAggregateFunctionMLMethod(AggregateFunctionFactory &);
void registerAggregateFunctionEntropy(AggregateFunctionFactory &);
void registerAggregateFunctionSimpleLinearRegression(AggregateFunctionFactory &);
void registerAggregateFunctionMoving(AggregateFunctionFactory &);
void registerAggregateFunctionCategoricalIV(AggregateFunctionFactory &);
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorForEach(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctions()
{
{

View File

@ -3,6 +3,47 @@
namespace DB
{
class AggregateFunctionFactory;
void registerAggregateFunctionAvg(AggregateFunctionFactory &);
void registerAggregateFunctionCount(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory &);
void registerAggregateFunctionWindowFunnel(AggregateFunctionFactory &);
void registerAggregateFunctionRate(AggregateFunctionFactory &);
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory &);
void registerAggregateFunctionsStatisticsStable(AggregateFunctionFactory &);
void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &);
void registerAggregateFunctionSum(AggregateFunctionFactory &);
void registerAggregateFunctionSumMap(AggregateFunctionFactory &);
void registerAggregateFunctionsUniq(AggregateFunctionFactory &);
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory &);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &);
void registerAggregateFunctionTopK(AggregateFunctionFactory &);
void registerAggregateFunctionsBitwise(AggregateFunctionFactory &);
void registerAggregateFunctionsBitmap(AggregateFunctionFactory &);
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &);
void registerAggregateFunctionHistogram(AggregateFunctionFactory &);
void registerAggregateFunctionRetention(AggregateFunctionFactory &);
void registerAggregateFunctionTimeSeriesGroupSum(AggregateFunctionFactory &);
void registerAggregateFunctionMLMethod(AggregateFunctionFactory &);
void registerAggregateFunctionEntropy(AggregateFunctionFactory &);
void registerAggregateFunctionSimpleLinearRegression(AggregateFunctionFactory &);
void registerAggregateFunctionMoving(AggregateFunctionFactory &);
void registerAggregateFunctionCategoricalIV(AggregateFunctionFactory &);
class AggregateFunctionCombinatorFactory;
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorForEach(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctions();
}

View File

@ -6,7 +6,7 @@
#include <Columns/ColumnVector.h>
#include <Common/typeid_cast.h>
#include <Common/HashTable/HashSet.h>
#include <Common/HashTable/HashMap.h>
#include "ColumnsCommon.h"
namespace DB

View File

@ -76,7 +76,7 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
}
}
void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg)
static void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg)
{
path = std::filesystem::absolute(path);
/// It's possible to get ENOSPC for non existent file (e.g. if there are no free inodes and creat() fails)
@ -97,7 +97,7 @@ void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg)
#endif
}
std::string getExtraExceptionInfo(const std::exception & e)
static std::string getExtraExceptionInfo(const std::exception & e)
{
String msg;
try

View File

@ -18,7 +18,13 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
UInt128 stringToUUID(const String &);
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wredundant-decls"
// Just dont mess with it. If the redundant redeclaration is removed then ReaderHelpers.h should be included.
// This leads to Arena.h inclusion which has a problem with ASAN stuff included properly and messing macro definition
// which intefrers with... You dont want to know, really.
UInt128 stringToUUID(const String & str);
#pragma GCC diagnostic pop
/** StaticVisitor (and its descendants) - class with overloaded operator() for all types of fields.

View File

@ -6,6 +6,7 @@
namespace DB
{
#pragma GCC diagnostic warning "-Wold-style-cast"
String getOpenSSLErrors()
{

View File

@ -71,6 +71,9 @@ extern const char EmptyPODArray[EmptyPODArraySize];
/** Base class that depend only on size of element, not on element itself.
* You can static_cast to this class if you want to insert some data regardless to the actual type T.
*/
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wnull-dereference"
template <size_t ELEMENT_SIZE, size_t initial_bytes, typename TAllocator, size_t pad_right_, size_t pad_left_>
class PODArrayBase : private boost::noncopyable, private TAllocator /// empty base optimization
{
@ -621,6 +624,6 @@ void swap(PODArray<T, initial_bytes, TAllocator, pad_right_> & lhs, PODArray<T,
{
lhs.swap(rhs);
}
#pragma GCC diagnostic pop
}

View File

@ -58,7 +58,7 @@ Exception::Exception(const Exception & exc)
using namespace DB;
void addRootPath(String & path, const String & root_path)
static void addRootPath(String & path, const String & root_path)
{
if (path.empty())
throw Exception("Path cannot be empty", ZBADARGUMENTS);
@ -75,7 +75,7 @@ void addRootPath(String & path, const String & root_path)
path = root_path + path;
}
void removeRootPath(String & path, const String & root_path)
static void removeRootPath(String & path, const String & root_path)
{
if (root_path.empty())
return;

View File

@ -264,24 +264,24 @@ using namespace DB;
/// Assuming we are at little endian.
void write(int64_t x, WriteBuffer & out)
static void write(int64_t x, WriteBuffer & out)
{
x = __builtin_bswap64(x);
writeBinary(x, out);
}
void write(int32_t x, WriteBuffer & out)
static void write(int32_t x, WriteBuffer & out)
{
x = __builtin_bswap32(x);
writeBinary(x, out);
}
void write(bool x, WriteBuffer & out)
static void write(bool x, WriteBuffer & out)
{
writeBinary(x, out);
}
void write(const String & s, WriteBuffer & out)
static void write(const String & s, WriteBuffer & out)
{
write(int32_t(s.size()), out);
out.write(s.data(), s.size());
@ -300,7 +300,7 @@ template <typename T> void write(const std::vector<T> & arr, WriteBuffer & out)
write(elem, out);
}
void write(const ACL & acl, WriteBuffer & out)
static void write(const ACL & acl, WriteBuffer & out)
{
write(acl.permissions, out);
write(acl.scheme, out);
@ -308,24 +308,24 @@ void write(const ACL & acl, WriteBuffer & out)
}
void read(int64_t & x, ReadBuffer & in)
static void read(int64_t & x, ReadBuffer & in)
{
readBinary(x, in);
x = __builtin_bswap64(x);
}
void read(int32_t & x, ReadBuffer & in)
static void read(int32_t & x, ReadBuffer & in)
{
readBinary(x, in);
x = __builtin_bswap32(x);
}
void read(bool & x, ReadBuffer & in)
static void read(bool & x, ReadBuffer & in)
{
readBinary(x, in);
}
void read(String & s, ReadBuffer & in)
static void read(String & s, ReadBuffer & in)
{
int32_t size = 0;
read(size, in);
@ -356,7 +356,7 @@ template <size_t N> void read(std::array<char, N> & s, ReadBuffer & in)
in.read(s.data(), N);
}
void read(Stat & stat, ReadBuffer & in)
static void read(Stat & stat, ReadBuffer & in)
{
read(stat.czxid, in);
read(stat.mzxid, in);
@ -1387,14 +1387,17 @@ void ZooKeeper::finalize(bool error_send, bool error_receive)
if (info.callback)
{
ResponsePtr response = info.request->makeResponse();
response->error = ZSESSIONEXPIRED;
try
if (response)
{
info.callback(*response);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
response->error = ZSESSIONEXPIRED;
try
{
info.callback(*response);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
if (info.watch)

View File

@ -4,6 +4,9 @@
/// Interposing these symbols explicitly. The idea works like this: malloc.cpp compiles to a
/// dedicated object (namely clickhouse_malloc.o), and it will show earlier in the link command
/// than malloc libs like libjemalloc.a. As a result, these symbols get picked in time right after.
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wredundant-decls"
extern "C"
{
void *malloc(size_t size);
@ -16,6 +19,7 @@ extern "C"
void *memalign(size_t alignment, size_t size);
void *pvalloc(size_t size);
}
#pragma GCC diagnostic pop
template<typename T>
inline void ignore(T x __attribute__((unused)))

View File

@ -4,40 +4,6 @@
//-----------------------------------------------------------------------------
void PrintAvalancheDiagram(int x, int y, int reps, double scale, int * bins)
{
const char * symbols = ".123456789X";
for (int i = 0; i < y; i++)
{
printf("[");
for (int j = 0; j < x; j++)
{
int k = (y - i) - 1;
int bin = bins[k + (j * y)];
double b = double(bin) / double(reps);
b = fabs(b * 2 - 1);
b *= scale;
int s = static_cast<int>(floor(b * 10));
if (s > 10)
s = 10;
if (s < 0)
s = 0;
printf("%c", symbols[s]);
}
printf("]\n");
}
}
//----------------------------------------------------------------------------
double maxBias(std::vector<int> & counts, int reps)
{
double worst = 0;

View File

@ -17,7 +17,7 @@
namespace fs = std::filesystem;
std::string createTmpPath(const std::string & filename)
static std::string createTmpPath(const std::string & filename)
{
char pattern[] = "/tmp/fileXXXXXX";
char * dir = mkdtemp(pattern);
@ -246,7 +246,7 @@ struct Generator3
}
};
void runTests()
static void runTests()
{
std::cout << "Test set 1\n";
TestSet<Generator1>::execute();

View File

@ -5,7 +5,7 @@
#include <gtest/gtest.h>
bool check()
static bool check()
{
ThreadPool pool(10);

View File

@ -12,7 +12,7 @@
#include <port/clock.h>
void setAffinity()
static void setAffinity()
{
#if !defined(__APPLE__) && !defined(__FreeBSD__)
cpu_set_t mask;
@ -200,7 +200,7 @@ const size_t BUF_SIZE = 1024;
using Source = std::vector<UInt64>;
void report(const char * name, size_t n, double elapsed, UInt64 tsc_diff, size_t res)
static void report(const char * name, size_t n, double elapsed, UInt64 tsc_diff, size_t res)
{
std::cerr << name << std::endl
<< "Done in " << elapsed

View File

@ -325,7 +325,7 @@ void NO_INLINE testForEachHash(const Key * data, size_t size, Init && init)
test<Map, Hashes::SipHash>(data, size, init);
}
void NO_INLINE testForEachMapAndHash(const Key * data, size_t size)
static void NO_INLINE testForEachMapAndHash(const Key * data, size_t size)
{
auto nothing = [](auto &){};

View File

@ -11,13 +11,13 @@ using MV = MultiVersion<T>;
using Results = std::vector<T>;
void thread1(MV & x, T & result)
static void thread1(MV & x, T & result)
{
MV::Version v = x.get();
result = *v;
}
void thread2(MV & x, const char * result)
static void thread2(MV & x, const char * result)
{
x.set(std::make_unique<T>(result));
}

View File

@ -63,7 +63,7 @@ using Mutex = std::mutex;
HashTableAllocator>;*/
void aggregate1(Map & map, Source::const_iterator begin, Source::const_iterator end)
static void aggregate1(Map & map, Source::const_iterator begin, Source::const_iterator end)
{
for (auto it = begin; it != end; ++it)
++map[*it];
@ -74,7 +74,7 @@ void aggregate1(Map & map, Source::const_iterator begin, Source::const_iterator
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
void aggregate12(Map & map, Source::const_iterator begin, Source::const_iterator end)
static void aggregate12(Map & map, Source::const_iterator begin, Source::const_iterator end)
{
Map::LookupResult found = nullptr;
auto prev_it = end;
@ -93,13 +93,13 @@ void aggregate12(Map & map, Source::const_iterator begin, Source::const_iterator
}
}
void aggregate2(MapTwoLevel & map, Source::const_iterator begin, Source::const_iterator end)
static void aggregate2(MapTwoLevel & map, Source::const_iterator begin, Source::const_iterator end)
{
for (auto it = begin; it != end; ++it)
++map[*it];
}
void aggregate22(MapTwoLevel & map, Source::const_iterator begin, Source::const_iterator end)
static void aggregate22(MapTwoLevel & map, Source::const_iterator begin, Source::const_iterator end)
{
MapTwoLevel::LookupResult found = nullptr;
auto prev_it = end;
@ -122,14 +122,14 @@ void aggregate22(MapTwoLevel & map, Source::const_iterator begin, Source::const_
#pragma GCC diagnostic pop
#endif
void merge2(MapTwoLevel * maps, size_t num_threads, size_t bucket)
static void merge2(MapTwoLevel * maps, size_t num_threads, size_t bucket)
{
for (size_t i = 1; i < num_threads; ++i)
for (auto it = maps[i].impls[bucket].begin(); it != maps[i].impls[bucket].end(); ++it)
maps[0].impls[bucket][it->getKey()] += it->getMapped();
}
void aggregate3(Map & local_map, Map & global_map, Mutex & mutex, Source::const_iterator begin, Source::const_iterator end)
static void aggregate3(Map & local_map, Map & global_map, Mutex & mutex, Source::const_iterator begin, Source::const_iterator end)
{
static constexpr size_t threshold = 65536;
@ -154,7 +154,7 @@ void aggregate3(Map & local_map, Map & global_map, Mutex & mutex, Source::const_
}
}
void aggregate33(Map & local_map, Map & global_map, Mutex & mutex, Source::const_iterator begin, Source::const_iterator end)
static void aggregate33(Map & local_map, Map & global_map, Mutex & mutex, Source::const_iterator begin, Source::const_iterator end)
{
static constexpr size_t threshold = 65536;
@ -176,7 +176,7 @@ void aggregate33(Map & local_map, Map & global_map, Mutex & mutex, Source::const
}
}
void aggregate4(Map & local_map, MapTwoLevel & global_map, Mutex * mutexes, Source::const_iterator begin, Source::const_iterator end)
static void aggregate4(Map & local_map, MapTwoLevel & global_map, Mutex * mutexes, Source::const_iterator begin, Source::const_iterator end)
{
static constexpr size_t threshold = 65536;
static constexpr size_t block_size = 8192;

View File

@ -14,7 +14,7 @@ do \
} \
while (0)
void test1()
static void test1()
{
using namespace DB;
@ -135,7 +135,7 @@ void test1()
std::cerr << "Some errors were found in test 1\n";
}
void test2()
static void test2()
{
using namespace DB;
@ -385,7 +385,7 @@ void test2()
std::cerr << "Some errors were found in test 2\n";
}
void test3()
static void test3()
{
using namespace DB;

View File

@ -9,17 +9,17 @@
using Key = double;
void NO_INLINE sort1(Key * data, size_t size)
static void NO_INLINE sort1(Key * data, size_t size)
{
std::sort(data, data + size);
}
void NO_INLINE sort2(Key * data, size_t size)
static void NO_INLINE sort2(Key * data, size_t size)
{
radixSortLSD(data, size);
}
void NO_INLINE sort3(Key * data, size_t size)
static void NO_INLINE sort3(Key * data, size_t size)
{
std::sort(data, data + size, [](Key a, Key b)
{

View File

@ -2,7 +2,7 @@
#include <common/SimpleCache.h>
int func(int x, int y)
static int func(int x, int y)
{
std::cerr << x << " + " << y << "\n";
return x + y;

View File

@ -88,7 +88,7 @@ uint8_t vectors[64][8] =
};
int test_vectors()
static int test_vectors()
{
#define MAXLEN 64
char in[MAXLEN];

View File

@ -6,11 +6,13 @@
#include <iostream>
#include <dlfcn.h>
NO_INLINE const void * getAddress()
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-function"
static NO_INLINE const void * getAddress()
{
return __builtin_return_address(0);
}
#pragma GCC diagnostic pop
int main(int argc, char ** argv)
{

View File

@ -10,8 +10,8 @@
int value = 0;
void f() { ++value; }
void * g(void *) { f(); return {}; }
static void f() { ++value; }
static void * g(void *) { f(); return {}; }
namespace DB

View File

@ -28,4 +28,7 @@ private:
UInt8 delta_bytes_size;
};
class CompressionCodecFactory;
void registerCodecDelta(CompressionCodecFactory & factory);
}

View File

@ -27,4 +27,7 @@ private:
UInt8 data_bytes_size;
};
class CompressionCodecFactory;
void registerCodecDoubleDelta(CompressionCodecFactory & factory);
}

View File

@ -27,4 +27,7 @@ private:
UInt8 data_bytes_size;
};
class CompressionCodecFactory;
void registerCodecGorilla(CompressionCodecFactory & factory);
}

View File

@ -37,12 +37,18 @@ String CompressionCodecLZ4::getCodecDesc() const
UInt32 CompressionCodecLZ4::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{
#pragma GCC diagnostic push
#pragma GCC diagnostic warning "-Wold-style-cast"
return LZ4_COMPRESSBOUND(uncompressed_size);
#pragma GCC diagnostic pop
}
UInt32 CompressionCodecLZ4::doCompressData(const char * source, UInt32 source_size, char * dest) const
{
#pragma GCC diagnostic push
#pragma GCC diagnostic warning "-Wold-style-cast"
return LZ4_compress_default(source, dest, source_size, LZ4_COMPRESSBOUND(source_size));
#pragma GCC diagnostic pop
}
void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
@ -66,7 +72,10 @@ String CompressionCodecLZ4HC::getCodecDesc() const
UInt32 CompressionCodecLZ4HC::doCompressData(const char * source, UInt32 source_size, char * dest) const
{
#pragma GCC diagnostic push
#pragma GCC diagnostic warning "-Wold-style-cast"
auto success = LZ4_compress_HC(source, dest, source_size, LZ4_COMPRESSBOUND(source_size), level);
#pragma GCC diagnostic pop
if (!success)
throw Exception("Cannot LZ4_compress_HC", ErrorCodes::CANNOT_COMPRESS);

View File

@ -29,6 +29,9 @@ private:
mutable LZ4::PerformanceStatistics lz4_stat;
};
class CompressionCodecFactory;
void registerCodecLZ4(CompressionCodecFactory & factory);
class CompressionCodecLZ4HC : public CompressionCodecLZ4
{
public:
@ -44,4 +47,7 @@ private:
const int level;
};
class CompressionCodecFactory;
void registerCodecLZ4HC(CompressionCodecFactory & factory);
}

View File

@ -29,4 +29,8 @@ private:
};
class CompressionCodecFactory;
void registerCodecMultiple(CompressionCodecFactory & factory);
}

View File

@ -22,4 +22,6 @@ protected:
};
class CompressionCodecFactory;
void registerCodecNone(CompressionCodecFactory & factory);
}

View File

@ -53,4 +53,7 @@ private:
Variant variant;
};
class CompressionCodecFactory;
void registerCodecT64(CompressionCodecFactory & factory);
}

View File

@ -30,4 +30,8 @@ private:
const int level;
};
class CompressionCodecFactory;
void registerCodecZSTD(CompressionCodecFactory & factory);
}

View File

@ -131,11 +131,8 @@ void CompressionCodecFactory::registerSimpleCompressionCodec(
}
void registerCodecLZ4(CompressionCodecFactory & factory);
void registerCodecNone(CompressionCodecFactory & factory);
void registerCodecZSTD(CompressionCodecFactory & factory);
void registerCodecMultiple(CompressionCodecFactory & factory);
void registerCodecLZ4HC(CompressionCodecFactory & factory);
void registerCodecDelta(CompressionCodecFactory & factory);
void registerCodecT64(CompressionCodecFactory & factory);
void registerCodecDoubleDelta(CompressionCodecFactory & factory);

View File

@ -14,7 +14,8 @@ class WriteBuffer;
* Column data could be nullptr - to represent just 'header' of column.
* Name could be either name from a table or some temporary generated name during expression evaluation.
*/
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wnull-dereference"
struct ColumnWithTypeAndName
{
ColumnPtr column;
@ -35,5 +36,6 @@ struct ColumnWithTypeAndName
void dumpStructure(WriteBuffer & out) const;
String dumpStructure() const;
};
#pragma GCC diagnostic pop
}

View File

@ -1040,7 +1040,10 @@ public:
throw Exception("Failed to write public key to memory. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
}
char * pem_buf = nullptr;
# pragma GCC diagnostic push
# pragma GCC diagnostic ignored "-Wold-style-cast"
long pem_size = BIO_get_mem_data(mem, &pem_buf);
# pragma GCC diagnostic pop
String pem(pem_buf, pem_size);
LOG_TRACE(log, "Key: " << pem);

View File

@ -25,6 +25,9 @@ std::ostream & operator<<(std::ostream & stream, const IStorage & what);
class TableStructureReadLock;
std::ostream & operator<<(std::ostream & stream, const TableStructureReadLock & what);
class IFunctionOverloadResolver;
std::ostream & operator<<(std::ostream & stream, const IFunctionOverloadResolver & what);
class IFunctionBase;
std::ostream & operator<<(std::ostream & stream, const IFunctionBase & what);

View File

@ -1,3 +1,4 @@
#pragma GCC diagnostic ignored "-Wmissing-declarations"
#include <gtest/gtest.h>
#include <Core/DecimalFunctions.h>

View File

@ -1,6 +1,7 @@
#include <random>
#include <Common/thread_local_rng.h>
#include <DataStreams/ConcatBlockInputStream.h>
#include "narrowBlockInputStreams.h"
namespace DB

View File

@ -8,7 +8,7 @@
using namespace DB;
Block getBlockWithSize(const std::vector<std::string> & columns, size_t rows, size_t stride, size_t & start)
static Block getBlockWithSize(const std::vector<std::string> & columns, size_t rows, size_t stride, size_t & start)
{
ColumnsWithTypeAndName cols;
@ -27,7 +27,7 @@ Block getBlockWithSize(const std::vector<std::string> & columns, size_t rows, si
}
BlockInputStreams getInputStreams(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
static BlockInputStreams getInputStreams(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
{
BlockInputStreams result;
for (auto [block_size_in_bytes, blocks_count, stride] : block_sizes)
@ -43,7 +43,7 @@ BlockInputStreams getInputStreams(const std::vector<std::string> & column_names,
}
BlockInputStreams getInputStreamsEqualStride(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
static BlockInputStreams getInputStreamsEqualStride(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
{
BlockInputStreams result;
size_t i = 0;
@ -61,7 +61,7 @@ BlockInputStreams getInputStreamsEqualStride(const std::vector<std::string> & co
}
SortDescription getSortDescription(const std::vector<std::string> & column_names)
static SortDescription getSortDescription(const std::vector<std::string> & column_names)
{
SortDescription descr;
for (const auto & column : column_names)

View File

@ -157,27 +157,6 @@ const DataTypeFactory::Creator& DataTypeFactory::findCreatorByName(const String
throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE);
}
void registerDataTypeNumbers(DataTypeFactory & factory);
void registerDataTypeDecimal(DataTypeFactory & factory);
void registerDataTypeDate(DataTypeFactory & factory);
void registerDataTypeDateTime(DataTypeFactory & factory);
void registerDataTypeDateTime64(DataTypeFactory & factory);
void registerDataTypeString(DataTypeFactory & factory);
void registerDataTypeFixedString(DataTypeFactory & factory);
void registerDataTypeEnum(DataTypeFactory & factory);
void registerDataTypeArray(DataTypeFactory & factory);
void registerDataTypeTuple(DataTypeFactory & factory);
void registerDataTypeNullable(DataTypeFactory & factory);
void registerDataTypeNothing(DataTypeFactory & factory);
void registerDataTypeUUID(DataTypeFactory & factory);
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
void registerDataTypeNested(DataTypeFactory & factory);
void registerDataTypeInterval(DataTypeFactory & factory);
void registerDataTypeLowCardinality(DataTypeFactory & factory);
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory);
void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory);
DataTypeFactory::DataTypeFactory()
{
registerDataTypeNumbers(*this);

View File

@ -65,4 +65,24 @@ private:
String getFactoryName() const override { return "DataTypeFactory"; }
};
void registerDataTypeNumbers(DataTypeFactory & factory);
void registerDataTypeDecimal(DataTypeFactory & factory);
void registerDataTypeDate(DataTypeFactory & factory);
void registerDataTypeDateTime(DataTypeFactory & factory);
void registerDataTypeString(DataTypeFactory & factory);
void registerDataTypeFixedString(DataTypeFactory & factory);
void registerDataTypeEnum(DataTypeFactory & factory);
void registerDataTypeArray(DataTypeFactory & factory);
void registerDataTypeTuple(DataTypeFactory & factory);
void registerDataTypeNullable(DataTypeFactory & factory);
void registerDataTypeNothing(DataTypeFactory & factory);
void registerDataTypeUUID(DataTypeFactory & factory);
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
void registerDataTypeNested(DataTypeFactory & factory);
void registerDataTypeInterval(DataTypeFactory & factory);
void registerDataTypeLowCardinality(DataTypeFactory & factory);
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory);
void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory);
void registerDataTypeDateTime64(DataTypeFactory & factory);
}

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