Merge pull request #3986 from yandex/weverything

Enabled most of the warnings from -Weverything
This commit is contained in:
alexey-milovidov 2019-01-06 15:29:18 +03:00 committed by GitHub
commit a0d8743c4c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
199 changed files with 1020 additions and 1073 deletions

View File

@ -1,7 +1,7 @@
# Third-party libraries may have substandard code.
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-stringop-overflow")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-stringop-overflow -Wno-implicit-function-declaration -Wno-return-type -Wno-array-bounds -Wno-bool-compare -Wno-int-conversion -Wno-switch")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -Wno-sign-compare -std=c++1z")
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-format -Wno-parentheses-equality -Wno-tautological-constant-compare -Wno-tautological-constant-out-of-range-compare -Wno-implicit-function-declaration -Wno-return-type -Wno-pointer-bool-conversion -Wno-enum-conversion -Wno-int-conversion -Wno-switch")

View File

@ -2,8 +2,7 @@
// MurmurHash2 was written by Austin Appleby, and is placed in the public
// domain. The author hereby disclaims copyright to this source code.
#ifndef _MURMURHASH2_H_
#define _MURMURHASH2_H_
#pragma once
//-----------------------------------------------------------------------------
// Platform-specific functions and macros
@ -30,6 +29,3 @@ uint64_t MurmurHash64B (const void * key, int len, uint64_t seed);
uint32_t MurmurHash2A (const void * key, int len, uint32_t seed);
uint32_t MurmurHashNeutral2 (const void * key, int len, uint32_t seed);
uint32_t MurmurHashAligned2 (const void * key, int len, uint32_t seed);
#endif // _MURMURHASH2_H_

View File

@ -2,8 +2,7 @@
// MurmurHash3 was written by Austin Appleby, and is placed in the public
// domain. The author hereby disclaims copyright to this source code.
#ifndef _MURMURHASH3_H_
#define _MURMURHASH3_H_
#pragma once
//-----------------------------------------------------------------------------
// Platform-specific functions and macros
@ -33,5 +32,3 @@ void MurmurHash3_x86_128 ( const void * key, int len, uint32_t seed, void * out
void MurmurHash3_x64_128 ( const void * key, int len, uint32_t seed, void * out );
//-----------------------------------------------------------------------------
#endif // _MURMURHASH3_H_

View File

@ -29,13 +29,35 @@ if (NOT NO_WERROR)
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Werror")
endif ()
# Add some warnings that are not available even with -Wall -Wextra.
# Add some warnings that are not available even with -Wall -Wextra -Wpedantic.
option (WEVERYTHING "Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang." ON)
if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra-semi -Wcomma -Winconsistent-missing-destructor-override -Wunused-exception-parameter -Wshadow-uncaptured-local")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wpedantic -Wno-vla-extension -Wno-zero-length-array -Wno-gnu-anonymous-struct -Wno-nested-anon-types")
if (NOT CMAKE_CXX_COMPILER_VERSION VERSION_LESS 6)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wredundant-parens -Wzero-as-null-pointer-constant")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra-semi -Wcomma -Winconsistent-missing-destructor-override -Wunused-exception-parameter -Wshadow-uncaptured-local -Wcovered-switch-default -Wshadow -Wold-style-cast -Wrange-loop-analysis -Wunused-member-function -Wunreachable-code -Wunreachable-code-return -Wnewline-eof -Wembedded-directive -Wgnu-case-range -Wunused-macros -Wconditional-uninitialized -Wdeprecated -Wundef -Wreserved-id-macro -Wredundant-parens -Wzero-as-null-pointer-constant")
if (WEVERYTHING)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Weverything -Wno-c++98-compat -Wno-c++98-compat-pedantic -Wno-missing-noreturn -Wno-padded -Wno-switch-enum -Wno-shadow-field-in-constructor -Wno-deprecated-dynamic-exception-spec -Wno-float-equal -Wno-weak-vtables -Wno-shift-sign-overflow -Wno-sign-conversion -Wno-conversion -Wno-exit-time-destructors -Wno-undefined-func-template -Wno-documentation-unknown-command -Wno-missing-variable-declarations -Wno-unused-template -Wno-global-constructors -Wno-c99-extensions -Wno-missing-prototypes -Wno-weak-template-vtables -Wno-zero-length-array -Wno-gnu-anonymous-struct -Wno-nested-anon-types -Wno-double-promotion -Wno-disabled-macro-expansion -Wno-used-but-marked-unused -Wno-vla-extension -Wno-vla -Wno-packed")
# TODO Enable conversion, sign-conversion, double-promotion warnings.
endif ()
if (NOT CMAKE_CXX_COMPILER_VERSION VERSION_LESS 7)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}")
if (WEVERYTHING)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-return-std-move-in-c++11")
endif ()
endif ()
if (NOT CMAKE_CXX_COMPILER_VERSION VERSION_LESS 8)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra-semi-stmt")
if (WEVERYTHING)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-shadow-field") # TODO Enable
endif ()
endif ()
endif ()

View File

@ -229,7 +229,7 @@ private:
report(info_per_interval);
delay_watch.restart();
}
};
}
return true;
}
@ -324,7 +324,7 @@ private:
double seconds = watch.elapsedSeconds();
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
info_per_interval.add(seconds, progress.rows, progress.bytes, info.rows, info.bytes);
info_total.add(seconds, progress.rows, progress.bytes, info.rows, info.bytes);
}
@ -332,7 +332,7 @@ private:
void report(Stats & info)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
/// Avoid zeros, nans or exceptions
if (0 == info.queries)
@ -369,7 +369,7 @@ private:
{
WriteBufferFromFile json_out(filename);
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
auto print_key_value = [&](auto key, auto value, bool with_comma = true)
{
@ -503,6 +503,4 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
std::cerr << getCurrentExceptionMessage(print_stacktrace, true) << std::endl;
return getCurrentExceptionCode();
}
return 0;
}

View File

@ -74,8 +74,8 @@
/// http://en.wikipedia.org/wiki/ANSI_escape_code
/// Similar codes \e[s, \e[u don't work in VT100 and Mosh.
#define SAVE_CURSOR_POSITION "\e7"
#define RESTORE_CURSOR_POSITION "\e8"
#define SAVE_CURSOR_POSITION "\033""7"
#define RESTORE_CURSOR_POSITION "\033""8"
#define CLEAR_TO_END_OF_LINE "\033[K"
@ -554,10 +554,10 @@ private:
void loop()
{
String query;
String prev_query;
String input;
String prev_input;
while (char * line_ = readline(query.empty() ? prompt().c_str() : ":-] "))
while (char * line_ = readline(input.empty() ? prompt().c_str() : ":-] "))
{
String line = line_;
free(line_);
@ -577,17 +577,17 @@ private:
if (ends_with_backslash)
line = line.substr(0, ws - 1);
query += line;
input += line;
if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || (!config().has("multiline") && !hasDataInSTDIN())))
{
if (query != prev_query)
if (input != prev_input)
{
/// Replace line breaks with spaces to prevent the following problem.
/// Every line of multi-line query is saved to history file as a separate line.
/// If the user restarts the client then after pressing the "up" button
/// every line of the query will be displayed separately.
std::string logged_query = query;
std::string logged_query = input;
std::replace(logged_query.begin(), logged_query.end(), '\n', ' ');
add_history(logged_query.c_str());
@ -596,18 +596,18 @@ private:
throwFromErrno("Cannot append history to file " + history_file, ErrorCodes::CANNOT_APPEND_HISTORY);
#endif
prev_query = query;
prev_input = input;
}
if (has_vertical_output_suffix)
query = query.substr(0, query.length() - 2);
input = input.substr(0, input.length() - 2);
try
{
/// Determine the terminal size.
ioctl(0, TIOCGWINSZ, &terminal_size);
if (!process(query))
if (!process(input))
break;
}
catch (const Exception & e)
@ -633,11 +633,11 @@ private:
connect();
}
query = "";
input = "";
}
else
{
query += '\n';
input += '\n';
}
}
}
@ -675,8 +675,6 @@ private:
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.
String query;
const char * begin = text.data();
const char * end = begin + text.size();
@ -708,19 +706,19 @@ private:
insert->end = pos;
}
query = text.substr(begin - text.data(), pos - begin);
String str = text.substr(begin - text.data(), pos - begin);
begin = pos;
while (isWhitespaceASCII(*begin) || *begin == ';')
++begin;
TestHint test_hint(test_mode, query);
TestHint test_hint(test_mode, str);
expected_client_error = test_hint.clientError();
expected_server_error = test_hint.serverError();
try
{
if (!processSingleQuery(query, ast) && !ignore_error)
if (!processSingleQuery(str, ast) && !ignore_error)
return false;
}
catch (...)
@ -728,7 +726,7 @@ private:
last_exception = std::make_unique<Exception>(getCurrentExceptionMessage(true), getCurrentExceptionCode());
actual_client_error = last_exception->code();
if (!ignore_error && (!actual_client_error || actual_client_error != expected_client_error))
std::cerr << "Error on processing query: " << query << std::endl << last_exception->message();
std::cerr << "Error on processing query: " << str << std::endl << last_exception->message();
got_exception = true;
}
@ -904,8 +902,6 @@ private:
ParserQuery parser(end, true);
ASTPtr res;
const auto ignore_error = config().getBool("ignore-error", false);
if (is_interactive || ignore_error)
{
String message;
@ -1616,10 +1612,10 @@ public:
for (size_t i = 0; i < external_tables_arguments.size(); ++i)
{
/// Parse commandline options related to external tables.
po::parsed_options parsed = po::command_line_parser(
po::parsed_options parsed_tables = po::command_line_parser(
external_tables_arguments[i].size(), external_tables_arguments[i].data()).options(external_description).run();
po::variables_map external_options;
po::store(parsed, external_options);
po::store(parsed_tables, external_options);
try
{

View File

@ -56,7 +56,7 @@ private:
{
std::string prefix_str(prefix);
std::tie(pos, end) = std::equal_range(words.begin(), words.end(), prefix_str,
[prefix_length](const std::string & s, const std::string & prefix) { return strncmp(s.c_str(), prefix.c_str(), prefix_length) < 0; });
[prefix_length](const std::string & s, const std::string & prefix_searched) { return strncmp(s.c_str(), prefix_searched.c_str(), prefix_length) < 0; });
}
/// Iterates through matched range.

View File

@ -66,11 +66,11 @@ void LocalServer::initialize(Poco::Util::Application & self)
}
}
void LocalServer::applyCmdSettings(Context & context)
void LocalServer::applyCmdSettings()
{
#define EXTRACT_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \
if (cmd_settings.NAME.changed) \
context.getSettingsRef().NAME = cmd_settings.NAME;
context->getSettingsRef().NAME = cmd_settings.NAME;
APPLY_FOR_SETTINGS(EXTRACT_SETTING)
#undef EXTRACT_SETTING
}
@ -179,7 +179,7 @@ try
std::string default_database = config().getString("default_database", "_local");
context->addDatabase(default_database, std::make_shared<DatabaseMemory>(default_database));
context->setCurrentDatabase(default_database);
applyCmdOptions(*context);
applyCmdOptions();
if (!context->getPath().empty())
{
@ -274,7 +274,7 @@ void LocalServer::processQueries()
context->setUser("default", "", Poco::Net::SocketAddress{}, "");
context->setCurrentQueryId("");
applyCmdSettings(*context);
applyCmdSettings();
/// Use the same query_id (and thread group) for all queries
CurrentThread::QueryScope query_scope_holder(*context);
@ -494,10 +494,10 @@ void LocalServer::init(int argc, char ** argv)
config().setBool("ignore-error", true);
}
void LocalServer::applyCmdOptions(Context & context)
void LocalServer::applyCmdOptions()
{
context.setDefaultFormat(config().getString("output-format", config().getString("format", "TSV")));
applyCmdSettings(context);
context->setDefaultFormat(config().getString("output-format", config().getString("format", "TSV")));
applyCmdSettings();
}
}

View File

@ -34,8 +34,8 @@ private:
std::string getInitialCreateTableQuery();
void tryInitPath();
void applyCmdOptions(Context & context);
void applyCmdSettings(Context & context);
void applyCmdOptions();
void applyCmdSettings();
void attachSystemTables();
void processQueries();
void setupUsers();

View File

@ -317,8 +317,8 @@ void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UInt64 se
if (size >= 16)
{
char * dst = reinterpret_cast<char *>(std::min(pos, end - 16));
hash.get128(dst);
char * hash_dst = reinterpret_cast<char *>(std::min(pos, end - 16));
hash.get128(hash_dst);
}
else
{

View File

@ -19,4 +19,4 @@ namespace DB
std::string getIdentifierQuote(SQLHDBC hdbc);
}
#endif
#endif

View File

@ -112,7 +112,8 @@ public:
{
return asString(padding);
}
String asString(size_t padding) const
String asString(size_t cur_padding) const
{
String repr = "{";
@ -121,10 +122,10 @@ public:
if (it != content.begin())
repr += ',';
/// construct "key": "value" string with padding
repr += "\n" + pad(padding) + '"' + it->first + '"' + ": " + it->second;
repr += "\n" + pad(cur_padding) + '"' + it->first + '"' + ": " + it->second;
}
repr += "\n" + pad(padding - 1) + '}';
repr += "\n" + pad(cur_padding - 1) + '}';
return repr;
}
};
@ -762,13 +763,13 @@ private:
return true;
}
void processTestsConfigurations(const Paths & input_files)
void processTestsConfigurations(const Paths & paths)
{
tests_configurations.resize(input_files.size());
tests_configurations.resize(paths.size());
for (size_t i = 0; i != input_files.size(); ++i)
for (size_t i = 0; i != paths.size(); ++i)
{
const String path = input_files[i];
const String path = paths[i];
tests_configurations[i] = XMLConfigurationPtr(new XMLConfiguration(path));
}
@ -881,8 +882,6 @@ private:
}
}
Query query;
if (!test_config->has("query") && !test_config->has("query_file"))
{
throw DB::Exception("Missing query fields in test's config: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS);
@ -907,6 +906,7 @@ private:
bool tsv = fs::path(filename).extension().string() == ".tsv";
ReadBufferFromFile query_file(filename);
Query query;
if (tsv)
{
@ -1024,7 +1024,7 @@ private:
}
if (lite_output)
return minOutput(main_metric);
return minOutput();
else
return constructTotalInfo(metrics);
}
@ -1053,11 +1053,8 @@ private:
void runQueries(const QueriesWithIndexes & queries_with_indexes)
{
for (const std::pair<Query, const size_t> & query_and_index : queries_with_indexes)
for (const auto & [query, run_index] : queries_with_indexes)
{
Query query = query_and_index.first;
const size_t run_index = query_and_index.second;
TestStopConditions & stop_conditions = stop_conditions_by_run[run_index];
Stats & statistics = statistics_by_run[run_index];
@ -1139,7 +1136,7 @@ private:
}
}
void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVector & substitutions)
void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVector & out_substitutions)
{
Keys xml_substitutions;
substitutions_view->keys(xml_substitutions);
@ -1157,21 +1154,16 @@ private:
for (size_t j = 0; j != xml_values.size(); ++j)
{
substitutions[name].push_back(xml_substitution->getString("values.value[" + std::to_string(j) + "]"));
out_substitutions[name].push_back(xml_substitution->getString("values.value[" + std::to_string(j) + "]"));
}
}
}
std::vector<String> formatQueries(const String & query, StringToVector substitutions)
std::vector<String> formatQueries(const String & query, StringToVector substitutions_to_generate)
{
std::vector<String> queries;
StringToVector::iterator substitutions_first = substitutions.begin();
StringToVector::iterator substitutions_last = substitutions.end();
runThroughAllOptionsAndPush(substitutions_first, substitutions_last, query, queries);
return queries;
std::vector<String> queries_res;
runThroughAllOptionsAndPush(substitutions_to_generate.begin(), substitutions_to_generate.end(), query, queries_res);
return queries_res;
}
/// Recursive method which goes through all substitution blocks in xml
@ -1179,11 +1171,11 @@ private:
void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left,
StringToVector::iterator substitutions_right,
const String & template_query,
std::vector<String> & queries)
std::vector<String> & out_queries)
{
if (substitutions_left == substitutions_right)
{
queries.push_back(template_query); /// completely substituted query
out_queries.push_back(template_query); /// completely substituted query
return;
}
@ -1191,7 +1183,7 @@ private:
if (template_query.find(substitution_mask) == String::npos) /// nothing to substitute here
{
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, template_query, queries);
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, template_query, out_queries);
return;
}
@ -1209,7 +1201,7 @@ private:
query.replace(substr_pos, substitution_mask.length(), value);
}
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, query, queries);
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, query, out_queries);
}
}
@ -1343,7 +1335,7 @@ public:
return json_output.asString();
}
String minOutput(const String & main_metric)
String minOutput()
{
String output;
@ -1465,7 +1457,7 @@ try
input_files = options["input-files"].as<Strings>();
Strings collected_files;
for (const String filename : input_files)
for (const String & filename : input_files)
{
fs::path file(filename);

View File

@ -31,20 +31,16 @@ public:
Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override
{
LOG_TRACE(log,
"HTTP Request for " << name << ". "
<< "Method: "
<< request.getMethod()
<< ", Address: "
<< request.clientAddress().toString()
<< ", User-Agent: "
<< (request.has("User-Agent") ? request.get("User-Agent") : "none")
<< (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : (""))
#if !NDEBUG
<< ", Content Type: " << request.getContentType()
<< ", Transfer Encoding: " << request.getTransferEncoding()
#endif
);
LOG_TRACE(log, "HTTP Request for " << name << ". "
<< "Method: "
<< request.getMethod()
<< ", Address: "
<< request.clientAddress().toString()
<< ", User-Agent: "
<< (request.has("User-Agent") ? request.get("User-Agent") : "none")
<< (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : (""))
<< ", Content Type: " << request.getContentType()
<< ", Transfer Encoding: " << request.getTransferEncoding());
const auto & uri = request.getURI();

View File

@ -102,16 +102,14 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
{
String name = getAliasToOrName(name_param);
/// Find by exact match.
auto it = aggregate_functions.find(name);
if (it != aggregate_functions.end())
if (auto it = aggregate_functions.find(name); it != aggregate_functions.end())
return it->second(name, argument_types, parameters);
/// Find by case-insensitive name.
/// Combinators cannot apply for case insensitive (SQL-style) aggregate function names. Only for native names.
if (recursion_level == 0)
{
auto it = case_insensitive_aggregate_functions.find(Poco::toLower(name));
if (it != case_insensitive_aggregate_functions.end())
if (auto it = case_insensitive_aggregate_functions.find(Poco::toLower(name)); it != case_insensitive_aggregate_functions.end())
return it->second(name, argument_types, parameters);
}

View File

@ -45,8 +45,6 @@ AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name,
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
return nullptr;
}
}

View File

@ -80,11 +80,11 @@ public:
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
// Column 0 contains array of keys of known type
const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[0]);
const IColumn::Offsets & offsets = array_column.getOffsets();
const auto & keys_vec = static_cast<const ColVecType &>(array_column.getData());
const size_t keys_vec_offset = offsets[row_num - 1];
const size_t keys_vec_size = (offsets[row_num] - keys_vec_offset);
const ColumnArray & array_column0 = static_cast<const ColumnArray &>(*columns[0]);
const IColumn::Offsets & offsets0 = array_column0.getOffsets();
const auto & keys_vec = static_cast<const ColVecType &>(array_column0.getData());
const size_t keys_vec_offset = offsets0[row_num - 1];
const size_t keys_vec_size = (offsets0[row_num] - keys_vec_offset);
// Columns 1..n contain arrays of numeric values to sum
auto & merged_maps = this->data(place).merged_maps;

View File

@ -382,13 +382,13 @@ namespace detail
if (index == BIG_THRESHOLD)
break;
UInt64 count = 0;
readBinary(count, buf);
UInt64 elem_count = 0;
readBinary(elem_count, buf);
if (index < SMALL_THRESHOLD)
count_small[index] = count;
count_small[index] = elem_count;
else
count_big[index - SMALL_THRESHOLD] = count;
count_big[index - SMALL_THRESHOLD] = elem_count;
}
}
}

View File

@ -732,10 +732,10 @@ std::unique_ptr<Exception> Connection::receiveException()
std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type)
{
size_t num = Protocol::Server::stringsInMessage(msg_type);
std::vector<String> out(num);
std::vector<String> strings(num);
for (size_t i = 0; i < num; ++i)
readStringBinary(out[i], *in);
return out;
readStringBinary(strings[i], *in);
return strings;
}

View File

@ -9,7 +9,7 @@
#include <IO/WriteHelpers.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#endif
@ -169,7 +169,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
const UInt8 * filt_end = filt_pos + col_size;
const UInt8 * data_pos = chars.data();
#if __SSE2__
#ifdef __SSE2__
/** A slightly more optimized version.
* Based on the assumption that often pieces of consecutive values
* completely pass or do not pass the filter.

View File

@ -635,11 +635,11 @@ void ColumnLowCardinality::Dictionary::checkColumn(const IColumn & column)
throw Exception("ColumnUnique expected as an argument of ColumnLowCardinality.", ErrorCodes::ILLEGAL_COLUMN);
}
void ColumnLowCardinality::Dictionary::setShared(const ColumnPtr & dictionary)
void ColumnLowCardinality::Dictionary::setShared(const ColumnPtr & column_unique_)
{
checkColumn(*dictionary);
checkColumn(*column_unique_);
column_unique = dictionary;
column_unique = column_unique_;
shared = true;
}

View File

@ -66,9 +66,9 @@ public:
Int64 getInt(size_t n) const override { return getNestedColumn()->getInt(n); }
bool isNullAt(size_t n) const override { return is_nullable && n == getNullValueIndex(); }
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override
void updateHashWithValue(size_t n, SipHash & hash_func) const override
{
return getNestedColumn()->updateHashWithValue(n, hash);
return getNestedColumn()->updateHashWithValue(n, hash_func);
}
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override;

View File

@ -13,7 +13,7 @@
#include <DataStreams/ColumnGathererStream.h>
#include <ext/bit_cast.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#include <Columns/ColumnsCommon.h>
@ -162,7 +162,7 @@ ColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_s
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = data.data();
#if __SSE2__
#ifdef __SSE2__
/** A slightly more optimized version.
* Based on the assumption that often pieces of consecutive values
* completely pass or do not pass the filter.

View File

@ -1,4 +1,4 @@
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#endif
@ -24,7 +24,7 @@ size_t countBytesInFilter(const IColumn::Filter & filt)
const Int8 * pos = reinterpret_cast<const Int8 *>(filt.data());
const Int8 * end = pos + filt.size();
#if __SSE2__ && __POPCNT__
#if defined(__SSE2__) && defined(__POPCNT__)
const __m128i zero16 = _mm_setzero_si128();
const Int8 * end64 = pos + filt.size() / 64 * 64;
@ -69,7 +69,7 @@ bool memoryIsZero(const void * data, size_t size)
const Int8 * pos = reinterpret_cast<const Int8 *>(data);
const Int8 * end = pos + size;
#if __SSE2__
#ifdef __SSE2__
const __m128 zero16 = _mm_setzero_ps();
const Int8 * end64 = pos + size / 64 * 64;
@ -205,17 +205,17 @@ namespace
/// copy array ending at *end_offset_ptr
const auto copy_array = [&] (const IColumn::Offset * offset_ptr)
{
const auto offset = offset_ptr == offsets_begin ? 0 : offset_ptr[-1];
const auto size = *offset_ptr - offset;
const auto arr_offset = offset_ptr == offsets_begin ? 0 : offset_ptr[-1];
const auto arr_size = *offset_ptr - arr_offset;
result_offsets_builder.insertOne(size);
result_offsets_builder.insertOne(arr_size);
const auto elems_size_old = res_elems.size();
res_elems.resize(elems_size_old + size);
memcpy(&res_elems[elems_size_old], &src_elems[offset], size * sizeof(T));
res_elems.resize(elems_size_old + arr_size);
memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T));
};
#if __SSE2__
#ifdef __SSE2__
const __m128i zero_vec = _mm_setzero_si128();
static constexpr size_t SIMD_BYTES = 16;
const auto filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES;

View File

@ -330,7 +330,9 @@ public:
virtual bool lowCardinality() const { return false; }
virtual ~IColumn() {}
virtual ~IColumn() = default;
IColumn() = default;
IColumn(const IColumn &) = default;
/** Print column name, size, and recursively print all subcolumns.
*/

View File

@ -393,10 +393,10 @@ UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
if constexpr (use_saved_hash)
{
auto & data = saved_hash->getData();
if (data.size() <= num_rows)
data.resize(num_rows + 1);
data[num_rows] = hash;
auto & column_data = saved_hash->getData();
if (column_data.size() <= num_rows)
column_data.resize(num_rows + 1);
column_data[num_rows] = hash;
}
else
column->insertData(data.data, data.size);

View File

@ -10,6 +10,7 @@
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>

View File

@ -94,7 +94,11 @@ private:
Key key;
Payload payload;
void * ptr;
union
{
void * ptr;
char * char_ptr;
};
size_t size;
size_t refcount = 0;
void * chunk;
@ -286,7 +290,7 @@ private:
++token->refcount;
}
void cleanup([[maybe_unused]] std::lock_guard & token_lock, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
void cleanup([[maybe_unused]] std::lock_guard<std::mutex> & token_lock, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
{
token->cache.insert_tokens.erase(*key);
token->cleaned_up = true;
@ -349,7 +353,7 @@ private:
if (left_it->chunk == region.chunk && left_it->isFree())
{
region.size += left_it->size;
*reinterpret_cast<char **>(&region.ptr) -= left_it->size;
region.char_ptr-= left_it->size;
size_multimap.erase(size_multimap.iterator_to(*left_it));
adjacency_list.erase_and_dispose(left_it, [](RegionMetadata * elem) { elem->destroy(); });
}
@ -479,7 +483,7 @@ private:
size_multimap.erase(size_multimap.iterator_to(free_region));
free_region.size -= size;
*reinterpret_cast<char **>(&free_region.ptr) += size;
free_region.char_ptr += size;
size_multimap.insert(free_region);
adjacency_list.insert(adjacency_list.iterator_to(free_region), *allocated_region);

View File

@ -22,7 +22,7 @@ namespace ErrorCodes
* simulates an array of `content_width`-bit values.
*/
template <typename BucketIndex, UInt8 content_width, size_t bucket_count>
class __attribute__ ((packed)) CompactArray final
class CompactArray final
{
public:
class Reader;

View File

@ -600,9 +600,9 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
}
preprocessed_path = preprocessed_dir + new_path;
auto path = Poco::Path(preprocessed_path).makeParent();
if (!path.toString().empty())
Poco::File(path).createDirectories();
auto preprocessed_path_parent = Poco::Path(preprocessed_path).makeParent();
if (!preprocessed_path_parent.toString().empty())
Poco::File(preprocessed_path_parent).createDirectories();
}
try
{

View File

@ -146,7 +146,7 @@ void FileChecker::load(Map & local_map, const std::string & path)
JSON json(out.str());
JSON files = json["yandex"];
for (const auto & name_value : files)
for (const JSON name_value : files)
local_map[unescapeForFileName(name_value.getName())] = name_value.getValue()["size"].toUInt();
}

View File

@ -35,20 +35,20 @@ inline DB::UInt64 intHash64(DB::UInt64 x)
* due to high speed (latency 3 + 1 clock cycle, throughput 1 clock cycle).
* Works only with SSE 4.2 support.
*/
#if __SSE4_2__
#ifdef __SSE4_2__
#include <nmmintrin.h>
#endif
#if __aarch64__ && __ARM_FEATURE_CRC32
#if defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
#include <arm_acle.h>
#include <arm_neon.h>
#endif
inline DB::UInt64 intHashCRC32(DB::UInt64 x)
{
#if __SSE4_2__
#ifdef __SSE4_2__
return _mm_crc32_u64(-1ULL, x);
#elif __aarch64__ && __ARM_FEATURE_CRC32
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cd(-1U, x);
#else
/// On other platforms we do not have CRC32. NOTE This can be confusing.

View File

@ -278,7 +278,7 @@ template <
typename BiasEstimator = TrivialBiasEstimator,
HyperLogLogMode mode = HyperLogLogMode::FullFeatured,
DenominatorMode denominator_mode = DenominatorMode::StableIfBig>
class __attribute__ ((packed)) HyperLogLogCounter : private Hash
class HyperLogLogCounter : private Hash
{
private:
/// Number of buckets.

View File

@ -28,37 +28,29 @@ namespace
{
struct Pipe
{
union
{
int fds[2];
struct
{
int read_fd;
int write_fd;
};
};
int fds_rw[2];
Pipe()
{
#ifndef __APPLE__
if (0 != pipe2(fds, O_CLOEXEC))
if (0 != pipe2(fds_rw, O_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
#else
if (0 != pipe(fds))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
if (0 != fcntl(fds[0], F_SETFD, FD_CLOEXEC))
if (0 != fcntl(fds_rw[0], F_SETFD, FD_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
if (0 != fcntl(fds[1], F_SETFD, FD_CLOEXEC))
if (0 != fcntl(fds_rw[1], F_SETFD, FD_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
#endif
}
~Pipe()
{
if (read_fd >= 0)
close(read_fd);
if (write_fd >= 0)
close(write_fd);
if (fds_rw[0] >= 0)
close(fds_rw[0]);
if (fds_rw[1] >= 0)
close(fds_rw[1]);
}
};
@ -125,15 +117,15 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
/// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens.
/// Replace the file descriptors with the ends of our pipes.
if (STDIN_FILENO != dup2(pipe_stdin.read_fd, STDIN_FILENO))
if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO))
_exit(int(ReturnCodes::CANNOT_DUP_STDIN));
if (!pipe_stdin_only)
{
if (STDOUT_FILENO != dup2(pipe_stdout.write_fd, STDOUT_FILENO))
if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO))
_exit(int(ReturnCodes::CANNOT_DUP_STDOUT));
if (STDERR_FILENO != dup2(pipe_stderr.write_fd, STDERR_FILENO))
if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO))
_exit(int(ReturnCodes::CANNOT_DUP_STDERR));
}
@ -143,12 +135,12 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
_exit(int(ReturnCodes::CANNOT_EXEC));
}
std::unique_ptr<ShellCommand> res(new ShellCommand(pid, pipe_stdin.write_fd, pipe_stdout.read_fd, pipe_stderr.read_fd, terminate_in_destructor));
std::unique_ptr<ShellCommand> res(new ShellCommand(pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor));
/// Now the ownership of the file descriptors is passed to the result.
pipe_stdin.write_fd = -1;
pipe_stdout.read_fd = -1;
pipe_stderr.read_fd = -1;
pipe_stdin.fds_rw[1] = -1;
pipe_stdout.fds_rw[0] = -1;
pipe_stderr.fds_rw[0] = -1;
return res;
}

View File

@ -7,11 +7,11 @@
#include <stdint.h>
#include <string.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#endif
#if __SSE4_1__
#ifdef __SSE4_1__
#include <smmintrin.h>
#endif
@ -32,7 +32,7 @@ namespace ErrorCodes
struct StringSearcherBase
{
#if __SSE2__
#ifdef __SSE2__
static constexpr auto n = sizeof(__m128i);
const int page_size = getpagesize();
@ -63,7 +63,7 @@ private:
UInt8 l{};
UInt8 u{};
#if __SSE4_1__
#ifdef __SSE4_1__
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
__m128i patl, patu;
/// lower and uppercase vectors of first 16 characters of `needle`
@ -102,7 +102,7 @@ public:
u = u_seq[0];
}
#if __SSE4_1__
#ifdef __SSE4_1__
/// for detecting leftmost position of the first symbol
patl = _mm_set1_epi8(l);
patu = _mm_set1_epi8(u);
@ -160,7 +160,7 @@ public:
{
static const Poco::UTF8Encoding utf8;
#if __SSE4_1__
#ifdef __SSE4_1__
if (pageSafe(pos))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
@ -227,7 +227,7 @@ public:
while (haystack < haystack_end)
{
#if __SSE4_1__
#ifdef __SSE4_1__
if (haystack + n <= haystack_end && pageSafe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
@ -249,15 +249,15 @@ public:
if (haystack < haystack_end && haystack + n <= haystack_end && pageSafe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
const auto v_haystack_offset = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_l_offset = _mm_cmpeq_epi8(v_haystack_offset, cachel);
const auto v_against_u_offset = _mm_cmpeq_epi8(v_haystack_offset, cacheu);
const auto v_against_l_or_u_offset = _mm_or_si128(v_against_l_offset, v_against_u_offset);
const auto mask_offset = _mm_movemask_epi8(v_against_l_or_u_offset);
if (0xffff == cachemask)
{
if (mask == cachemask)
if (mask_offset == cachemask)
{
auto haystack_pos = haystack + cache_valid_len;
auto needle_pos = needle + cache_valid_len;
@ -276,7 +276,7 @@ public:
return haystack;
}
}
else if ((mask & cachemask) == cachemask)
else if ((mask_offset & cachemask) == cachemask)
return haystack;
/// first octet was ok, but not the first 16, move to start of next sequence and reapply
@ -334,7 +334,7 @@ private:
UInt8 l{};
UInt8 u{};
#if __SSE4_1__
#ifdef __SSE4_1__
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
__m128i patl, patu;
/// lower and uppercase vectors of first 16 characters of `needle`
@ -352,7 +352,7 @@ public:
l = static_cast<UInt8>(std::tolower(*needle));
u = static_cast<UInt8>(std::toupper(*needle));
#if __SSE4_1__
#ifdef __SSE4_1__
patl = _mm_set1_epi8(l);
patu = _mm_set1_epi8(u);
@ -376,7 +376,7 @@ public:
bool compare(const UInt8 * pos) const
{
#if __SSE4_1__
#ifdef __SSE4_1__
if (pageSafe(pos))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
@ -434,7 +434,7 @@ public:
while (haystack < haystack_end)
{
#if __SSE4_1__
#ifdef __SSE4_1__
if (haystack + n <= haystack_end && pageSafe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
@ -455,15 +455,15 @@ public:
if (haystack < haystack_end && haystack + n <= haystack_end && pageSafe(haystack))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
const auto v_haystack_offset = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_l_offset = _mm_cmpeq_epi8(v_haystack_offset, cachel);
const auto v_against_u_offset = _mm_cmpeq_epi8(v_haystack_offset, cacheu);
const auto v_against_l_or_u_offset = _mm_or_si128(v_against_l_offset, v_against_u_offset);
const auto mask_offset = _mm_movemask_epi8(v_against_l_or_u_offset);
if (0xffff == cachemask)
{
if (mask == cachemask)
if (mask_offset == cachemask)
{
auto haystack_pos = haystack + n;
auto needle_pos = needle + n;
@ -479,7 +479,7 @@ public:
return haystack;
}
}
else if ((mask & cachemask) == cachemask)
else if ((mask_offset & cachemask) == cachemask)
return haystack;
++haystack;
@ -532,7 +532,7 @@ private:
/// first character in `needle`
UInt8 first{};
#if __SSE4_1__
#ifdef __SSE4_1__
/// vector filled `first` for determining leftmost position of the first symbol
__m128i pattern;
/// vector of first 16 characters of `needle`
@ -549,7 +549,7 @@ public:
first = *needle;
#if __SSE4_1__
#ifdef __SSE4_1__
pattern = _mm_set1_epi8(first);
auto needle_pos = needle;
@ -570,7 +570,7 @@ public:
bool compare(const UInt8 * pos) const
{
#if __SSE4_1__
#ifdef __SSE4_1__
if (pageSafe(pos))
{
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
@ -620,7 +620,7 @@ public:
while (haystack < haystack_end)
{
#if __SSE4_1__
#ifdef __SSE4_1__
if (haystack + n <= haystack_end && pageSafe(haystack))
{
/// find first character
@ -642,13 +642,13 @@ public:
if (haystack < haystack_end && haystack + n <= haystack_end && pageSafe(haystack))
{
/// check for first 16 octets
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_cache = _mm_cmpeq_epi8(v_haystack, cache);
const auto mask = _mm_movemask_epi8(v_against_cache);
const auto v_haystack_offset = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
const auto v_against_cache = _mm_cmpeq_epi8(v_haystack_offset, cache);
const auto mask_offset = _mm_movemask_epi8(v_against_cache);
if (0xffff == cachemask)
{
if (mask == cachemask)
if (mask_offset == cachemask)
{
auto haystack_pos = haystack + n;
auto needle_pos = needle + n;
@ -661,7 +661,7 @@ public:
return haystack;
}
}
else if ((mask & cachemask) == cachemask)
else if ((mask_offset & cachemask) == cachemask)
return haystack;
++haystack;

View File

@ -56,7 +56,7 @@ std::string getOrdinalSuffix(T n)
case 2: return "nd";
case 3: return "rd";
default: return "th";
};
}
}
/// More efficient than libc, because doesn't respect locale. But for some functions table implementation could be better.

View File

@ -80,11 +80,6 @@ struct NetlinkMessage
::nlmsgerr error;
};
size_t payload_size() const
{
return header.nlmsg_len - sizeof(header) - sizeof(generic_header);
}
const Attribute * end() const
{
return reinterpret_cast<const Attribute *>(reinterpret_cast<const char *>(this) + header.nlmsg_len);

View File

@ -6,7 +6,7 @@
#include <Core/Types.h>
#if __SSE4_2__
#ifdef __SSE4_2__
#include <nmmintrin.h>
#endif
@ -75,7 +75,7 @@ struct UInt128Hash
}
};
#if __SSE4_2__
#ifdef __SSE4_2__
struct UInt128HashCRC32
{
@ -153,7 +153,7 @@ struct UInt256Hash
}
};
#if __SSE4_2__
#ifdef __SSE4_2__
struct UInt256HashCRC32
{

View File

@ -3,7 +3,7 @@
#include <Core/Types.h>
#include <Common/BitHelpers.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#endif
@ -55,7 +55,7 @@ inline size_t countCodePoints(const UInt8 * data, size_t size)
size_t res = 0;
const auto end = data + size;
#if __SSE2__
#ifdef __SSE2__
constexpr auto bytes_sse = sizeof(__m128i);
const auto src_end_sse = data + size / bytes_sse * bytes_sse;

View File

@ -59,7 +59,9 @@ using Requests = std::vector<RequestPtr>;
struct Request
{
virtual ~Request() {}
Request() = default;
Request(const Request &) = default;
virtual ~Request() = default;
virtual String getPath() const = 0;
virtual void addRootPath(const String & /* root_path */) {}
};
@ -72,7 +74,9 @@ using ResponseCallback = std::function<void(const Response &)>;
struct Response
{
int32_t error = 0;
virtual ~Response() {}
Response() = default;
Response(const Response &) = default;
virtual ~Response() = default;
virtual void removeRootPath(const String & /* root_path */) {}
};

View File

@ -17,7 +17,6 @@
#define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000
#define ZOOKEEPER_OPERATION_TIMEOUT_MS 10000
namespace DB

View File

@ -1181,9 +1181,9 @@ void ZooKeeper::receiveEvent()
ProfileEvents::increment(ProfileEvents::ZooKeeperWatchResponse);
response = std::make_shared<ZooKeeperWatchResponse>();
request_info.callback = [this](const Response & response)
request_info.callback = [this](const Response & response_)
{
const WatchResponse & watch_response = dynamic_cast<const WatchResponse &>(response);
const WatchResponse & watch_response = dynamic_cast<const WatchResponse &>(response_);
std::lock_guard lock(watches_mutex);

View File

@ -252,7 +252,9 @@ struct ZooKeeperRequest : virtual Request
/// If the request was sent and we didn't get the response and the error happens, then we cannot be sure was it processed or not.
bool probably_sent = false;
virtual ~ZooKeeperRequest() {}
ZooKeeperRequest() = default;
ZooKeeperRequest(const ZooKeeperRequest &) = default;
virtual ~ZooKeeperRequest() = default;
virtual ZooKeeper::OpNum getOpNum() const = 0;

View File

@ -8,6 +8,7 @@
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>

View File

@ -63,6 +63,4 @@ int main(int argc, char ** argv)
std::cerr << "Some exception" << std::endl;
return 2;
}
return 0;
}

View File

@ -67,6 +67,4 @@ int main(int argc, char ** argv)
std::cerr << "Some exception: " << DB::getCurrentExceptionMessage(true) << std::endl;
return 2;
}
return 0;
}

View File

@ -3,7 +3,7 @@
#include <string.h>
#include <Core/Defines.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>

View File

@ -60,7 +60,7 @@ int main(int argc, char ** argv)
{
pcg64 generator(randomSeed());
for (size_t i = 0; i < num_iterations; ++i)
for (size_t j = 0; j < num_iterations; ++j)
{
size_t size = std::uniform_int_distribution<size_t>(1, region_max_size)(generator);
int key = std::uniform_int_distribution<int>(1, max_key)(generator);
@ -70,8 +70,8 @@ int main(int argc, char ** argv)
[=]{ return size; },
[=](void * /*ptr*/, int & payload)
{
payload = i;
// memset(ptr, i, size);
payload = j;
// memset(ptr, j, size);
},
nullptr);

View File

@ -9,7 +9,9 @@ private:
virtual MutablePtr clone() const = 0;
public:
virtual ~IColumn() {}
IColumn() = default;
IColumn(const IColumn &) = default;
virtual ~IColumn() = default;
virtual int get() const = 0;
virtual void set(int value) = 0;

View File

@ -1,6 +1,7 @@
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>

View File

@ -3,6 +3,7 @@
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>

View File

@ -171,7 +171,7 @@ namespace Hashes
}
};
#if __SSE4_2__
#ifdef __SSE4_2__
#include <nmmintrin.h>
#endif
@ -179,7 +179,7 @@ namespace Hashes
{
size_t operator()(Key x) const
{
#if __SSE4_2__
#ifdef __SSE4_2__
return _mm_crc32_u64(-1ULL, x);
#else
/// On other platforms we do not have CRC32. NOTE This can be confusing.

View File

@ -186,8 +186,8 @@ struct MergeParallelForTwoLevelTable
for (size_t i = 0; i < num_maps; ++i)
section[i] = &source_maps[i]->impls[bucket];
typename Map::Impl * result_map;
ImplMerge::execute(section.data(), num_maps, result_map, merger, pool);
typename Map::Impl * res;
ImplMerge::execute(section.data(), num_maps, res, merger, pool);
});
pool.wait();

View File

@ -9,6 +9,9 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#ifdef __clang__
#pragma clang diagnostic ignored "-Wold-style-cast"
#endif
namespace DB

View File

@ -10,15 +10,15 @@
#include <common/Types.h>
#include <common/unaligned.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#endif
#if __SSSE3__
#ifdef __SSSE3__
#include <tmmintrin.h>
#endif
#if __aarch64__
#ifdef __aarch64__
#include <arm_neon.h>
#endif
@ -213,7 +213,7 @@ template <> void inline copyOverlap<8, true>(UInt8 * op, const UInt8 *& match, c
inline void copy16(UInt8 * dst, const UInt8 * src)
{
#if __SSE2__
#ifdef __SSE2__
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(src)));
#else
@ -351,8 +351,8 @@ void NO_INLINE decompressImpl(
char * const dest,
size_t dest_size)
{
const UInt8 * ip = (UInt8 *)source;
UInt8 * op = (UInt8 *)dest;
const UInt8 * ip = reinterpret_cast<const UInt8 *>(source);
UInt8 * op = reinterpret_cast<UInt8 *>(dest);
UInt8 * const output_end = op + dest_size;
while (1)
@ -528,8 +528,8 @@ void statistics(
size_t dest_size,
StreamStatistics & stat)
{
const UInt8 * ip = (UInt8 *)source;
UInt8 * op = (UInt8 *)dest;
const UInt8 * ip = reinterpret_cast<const UInt8 *>(source);
UInt8 * op = reinterpret_cast<UInt8 *>(dest);
UInt8 * const output_end = op + dest_size;
while (1)

View File

@ -25,7 +25,7 @@ void BlockInfo::write(WriteBuffer & out) const
writeVarUInt(FIELD_NUM, out); \
writeBinary(NAME, out);
APPLY_FOR_BLOCK_INFO_FIELDS(WRITE_FIELD);
APPLY_FOR_BLOCK_INFO_FIELDS(WRITE_FIELD)
#undef WRITE_FIELD
writeVarUInt(0, out);
@ -49,7 +49,7 @@ void BlockInfo::read(ReadBuffer & in)
readBinary(NAME, in); \
break;
APPLY_FOR_BLOCK_INFO_FIELDS(READ_FIELD);
APPLY_FOR_BLOCK_INFO_FIELDS(READ_FIELD)
#undef READ_FIELD
default:

View File

@ -75,7 +75,7 @@ namespace DB
x.push_back(value);
break;
}
};
}
}
}
@ -128,7 +128,7 @@ namespace DB
DB::writeBinary(get<Tuple>(*it), buf);
break;
}
};
}
}
}
@ -209,7 +209,7 @@ namespace DB
x.push_back(value);
break;
}
};
}
}
}
@ -262,7 +262,7 @@ namespace DB
DB::writeBinary(get<Tuple>(*it), buf);
break;
}
};
}
}
}

View File

@ -449,9 +449,6 @@ private:
case Types::Decimal32: f(field.template get<DecimalField<Decimal32>>()); return;
case Types::Decimal64: f(field.template get<DecimalField<Decimal64>>()); return;
case Types::Decimal128: f(field.template get<DecimalField<Decimal128>>()); return;
default:
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
}
}

View File

@ -30,12 +30,14 @@ int main(int argc, char ** argv)
field2 = field;
std::cerr << applyVisitor(to_string, field2) << std::endl;
Array array;
array.push_back(UInt64(123));
array.push_back(Int64(-123));
array.push_back(String("Hello"));
field = array;
std::cerr << applyVisitor(to_string, field) << std::endl;
{
Array array;
array.push_back(UInt64(123));
array.push_back(Int64(-123));
array.push_back(String("Hello"));
field = array;
std::cerr << applyVisitor(to_string, field) << std::endl;
}
get<Array &>(field).push_back(field);
std::cerr << applyVisitor(to_string, field) << std::endl;
@ -54,7 +56,7 @@ int main(int argc, char ** argv)
Array array(n);
{
Stopwatch watch;
watch.restart();
for (size_t i = 0; i < n; ++i)
array[i] = String(i % 32, '!');
@ -67,7 +69,7 @@ int main(int argc, char ** argv)
}
{
Stopwatch watch;
watch.restart();
size_t sum = 0;
for (size_t i = 0; i < n; ++i)

View File

@ -5,5 +5,7 @@ namespace DB
{
BlockIO::~BlockIO() = default;
BlockIO::BlockIO() = default;
BlockIO::BlockIO(const BlockIO &) = default;
}

View File

@ -60,6 +60,8 @@ struct BlockIO
}
~BlockIO();
BlockIO();
BlockIO(const BlockIO &);
};
}

View File

@ -230,9 +230,6 @@ void IProfilingBlockInputStream::checkQuota(Block & block)
prev_elapsed = total_elapsed;
break;
}
default:
throw Exception("Logical error: unknown limits mode.", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -56,8 +56,6 @@ static inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings
case FormatSettings::DateTimeInputFormat::BestEffort:
parseDateTimeBestEffort(x, istr, time_zone, utc_time_zone);
return;
default:
__builtin_unreachable();
}
}

View File

@ -49,8 +49,9 @@ public:
case Month: return "Month";
case Quarter: return "Quarter";
case Year: return "Year";
default: __builtin_unreachable();
}
__builtin_unreachable();
}
DataTypeInterval(Kind kind) : kind(kind) {}

View File

@ -766,7 +766,7 @@ namespace
void operator()()
{
if (typeid_cast<const DataTypeNumber<T> *>(&keys_type))
column = creator((ColumnVector<T> *)(nullptr));
column = creator(static_cast<ColumnVector<T> *>(nullptr));
}
};
}
@ -780,13 +780,13 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
type = nullable_type->getNestedType().get();
if (isString(type))
return creator((ColumnString *)(nullptr));
return creator(static_cast<ColumnString *>(nullptr));
if (isFixedString(type))
return creator((ColumnFixedString *)(nullptr));
return creator(static_cast<ColumnFixedString *>(nullptr));
if (typeid_cast<const DataTypeDate *>(type))
return creator((ColumnVector<UInt16> *)(nullptr));
return creator(static_cast<ColumnVector<UInt16> *>(nullptr));
if (typeid_cast<const DataTypeDateTime *>(type))
return creator((ColumnVector<UInt32> *)(nullptr));
return creator(static_cast<ColumnVector<UInt32> *>(nullptr));
if (isNumber(type))
{
MutableColumnUniquePtr column;

View File

@ -14,7 +14,7 @@
#include <IO/WriteHelpers.h>
#include <IO/VarInt.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#endif
@ -129,7 +129,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt
if (size)
{
#if __SSE2__
#ifdef __SSE2__
/// An optimistic branch in which more efficient copying is possible.
if (offset + 16 * UNROLL_TIMES <= data.allocated_bytes() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end())
{

View File

@ -100,7 +100,7 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_
void DataTypeTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
const auto & tuple = get<const Tuple &>(field).toUnderType();
for (const auto & idx_elem : ext::enumerate(elems))
for (const auto idx_elem : ext::enumerate(elems))
idx_elem.second->serializeBinary(tuple[idx_elem.first], ostr);
}
@ -115,7 +115,7 @@ void DataTypeTuple::deserializeBinary(Field & field, ReadBuffer & istr) const
void DataTypeTuple::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
for (const auto & idx_elem : ext::enumerate(elems))
for (const auto idx_elem : ext::enumerate(elems))
idx_elem.second->serializeBinary(extractElementColumn(column, idx_elem.first), row_num, ostr);
}

View File

@ -50,7 +50,7 @@ inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type)
return 20;
default:
break;
};
}
return 0;
}

View File

@ -7,6 +7,7 @@
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>

View File

@ -45,7 +45,5 @@ void ClickHouseLibrary::log(ClickHouseLibrary::LogLevel level, ClickHouseLibrary
if (logger.fatal())
logger.fatal(msg);
break;
default:
break;
}
}

View File

@ -1,7 +1,6 @@
#include "TrieDictionary.h"
#include <iostream>
#include <stack>
#include <btrie.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeFixedString.h>
@ -16,6 +15,13 @@
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wold-style-cast"
#pragma clang diagnostic ignored "-Wnewline-eof"
#endif
#include <btrie.h>
namespace DB
{
@ -761,13 +767,13 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
{
using BlockInputStreamType = DictionaryBlockInputStream<TrieDictionary, UInt64>;
auto getKeys = [](const Columns & columns, const std::vector<DictionaryAttribute> & attributes)
auto getKeys = [](const Columns & columns, const std::vector<DictionaryAttribute> & dict_attributes)
{
const auto & attr = attributes.front();
const auto & attr = dict_attributes.front();
return ColumnsWithTypeAndName(
{ColumnWithTypeAndName(columns.front(), std::make_shared<DataTypeFixedString>(IPV6_BINARY_LENGTH), attr.name)});
};
auto getView = [](const Columns & columns, const std::vector<DictionaryAttribute> & attributes)
auto getView = [](const Columns & columns, const std::vector<DictionaryAttribute> & dict_attributes)
{
auto column = ColumnString::create();
const auto & ip_column = static_cast<const ColumnFixedString &>(*columns.front());
@ -783,7 +789,7 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
column->insertData(buffer, ptr - buffer);
}
return ColumnsWithTypeAndName{
ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), attributes.front().name)};
ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), dict_attributes.front().name)};
};
return std::make_shared<BlockInputStreamType>(
shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(getKeys), std::move(getView));

View File

@ -88,7 +88,7 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp)
}
bool TSKVRowInputStream::read(MutableColumns & columns, RowReadExtension &)
bool TSKVRowInputStream::read(MutableColumns & columns, RowReadExtension & ext)
{
if (istr.eof())
return false;
@ -96,9 +96,7 @@ bool TSKVRowInputStream::read(MutableColumns & columns, RowReadExtension &)
size_t num_columns = columns.size();
/// Set of columns for which the values were read. The rest will be filled with default values.
/// TODO Ability to provide your DEFAULTs.
bool read_columns[num_columns];
memset(read_columns, 0, num_columns);
read_columns.assign(num_columns, false);
if (unlikely(*istr.position() == '\n'))
{
@ -180,6 +178,9 @@ bool TSKVRowInputStream::read(MutableColumns & columns, RowReadExtension &)
if (!read_columns[i])
header.getByPosition(i).type->insertDefaultInto(*columns[i]);
/// return info about defaults set
ext.read_columns = read_columns;
return true;
}

View File

@ -41,6 +41,8 @@ private:
/// Hash table matching `field name -> position in the block`. NOTE You can use perfect hash map.
using NameMap = HashMap<StringRef, size_t, StringRefHash>;
NameMap name_map;
std::vector<UInt8> read_columns;
};
}

View File

@ -225,10 +225,10 @@ struct DateTimeAddIntervalImpl
block.getByPosition(result).column = std::move(col_to);
}
else if (const auto * sources = checkAndGetColumnConst<ColumnVector<FromType>>(source_col.get()))
else if (const auto * sources_const = checkAndGetColumnConst<ColumnVector<FromType>>(source_col.get()))
{
auto col_to = ColumnVector<ToType>::create();
Op::constant_vector(sources->template getValue<FromType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
Op::constant_vector(sources_const->template getValue<FromType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
block.getByPosition(result).column = std::move(col_to);
}
else

View File

@ -146,7 +146,7 @@ private:
block.getByPosition(result).column = std::move(dst);
return true;
}
else if (const auto right_arg_typed = checkAndGetColumnConst<ColumnVector<RightType>>(right_arg))
if (const auto right_arg_typed = checkAndGetColumnConst<ColumnVector<RightType>>(right_arg))
{
auto dst = ColumnVector<Float64>::create();
@ -205,7 +205,7 @@ private:
throw Exception{"Illegal column " + right_arg->getName() + " of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
else if (const auto left_arg_typed = checkAndGetColumnConst<ColVecLeft>(left_arg))
if (const auto left_arg_typed = checkAndGetColumnConst<ColVecLeft>(left_arg))
{
if (executeTyped<LeftType, RightType>(block, result, left_arg_typed, right_arg))
return true;

View File

@ -75,12 +75,12 @@ public:
if (const ColumnString * haystack = checkAndGetColumn<ColumnString>(haystack_column))
dispatch<StringSource>(StringSource(*haystack), needle_column, vec_res);
else if (const ColumnFixedString * haystack = checkAndGetColumn<ColumnFixedString>(haystack_column))
dispatch<FixedStringSource>(FixedStringSource(*haystack), needle_column, vec_res);
else if (const ColumnConst * haystack = checkAndGetColumnConst<ColumnString>(haystack_column))
dispatch<ConstSource<StringSource>>(ConstSource<StringSource>(*haystack), needle_column, vec_res);
else if (const ColumnConst * haystack = checkAndGetColumnConst<ColumnFixedString>(haystack_column))
dispatch<ConstSource<FixedStringSource>>(ConstSource<FixedStringSource>(*haystack), needle_column, vec_res);
else if (const ColumnFixedString * haystack_fixed = checkAndGetColumn<ColumnFixedString>(haystack_column))
dispatch<FixedStringSource>(FixedStringSource(*haystack_fixed), needle_column, vec_res);
else if (const ColumnConst * haystack_const = checkAndGetColumnConst<ColumnString>(haystack_column))
dispatch<ConstSource<StringSource>>(ConstSource<StringSource>(*haystack_const), needle_column, vec_res);
else if (const ColumnConst * haystack_const_fixed = checkAndGetColumnConst<ColumnFixedString>(haystack_column))
dispatch<ConstSource<FixedStringSource>>(ConstSource<FixedStringSource>(*haystack_const_fixed), needle_column, vec_res);
else
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
@ -93,12 +93,12 @@ private:
{
if (const ColumnString * needle = checkAndGetColumn<ColumnString>(needle_column))
execute<HaystackSource, StringSource>(haystack_source, StringSource(*needle), res_data);
else if (const ColumnFixedString * needle = checkAndGetColumn<ColumnFixedString>(needle_column))
execute<HaystackSource, FixedStringSource>(haystack_source, FixedStringSource(*needle), res_data);
else if (const ColumnConst * needle = checkAndGetColumnConst<ColumnString>(needle_column))
execute<HaystackSource, ConstSource<StringSource>>(haystack_source, ConstSource<StringSource>(*needle), res_data);
else if (const ColumnConst * needle = checkAndGetColumnConst<ColumnFixedString>(needle_column))
execute<HaystackSource, ConstSource<FixedStringSource>>(haystack_source, ConstSource<FixedStringSource>(*needle), res_data);
else if (const ColumnFixedString * needle_fixed = checkAndGetColumn<ColumnFixedString>(needle_column))
execute<HaystackSource, FixedStringSource>(haystack_source, FixedStringSource(*needle_fixed), res_data);
else if (const ColumnConst * needle_const = checkAndGetColumnConst<ColumnString>(needle_column))
execute<HaystackSource, ConstSource<StringSource>>(haystack_source, ConstSource<StringSource>(*needle_const), res_data);
else if (const ColumnConst * needle_const_fixed = checkAndGetColumnConst<ColumnFixedString>(needle_column))
execute<HaystackSource, ConstSource<FixedStringSource>>(haystack_source, ConstSource<FixedStringSource>(*needle_const_fixed), res_data);
else
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -62,33 +62,33 @@ public:
block.getByPosition(result).column = std::move(col_res);
}
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column.get()))
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
{
if (Impl::is_fixed_to_constant)
{
ResultType res = 0;
Impl::vector_fixed_to_constant(col->getChars(), col->getN(), res);
Impl::vector_fixed_to_constant(col_fixed->getChars(), col_fixed->getN(), res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col->size(), toField(res));
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_fixed->size(), toField(res));
}
else
{
auto col_res = ColumnVector<ResultType>::create();
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
vec_res.resize(col->size());
Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res);
vec_res.resize(col_fixed->size());
Impl::vector_fixed_to_vector(col_fixed->getChars(), col_fixed->getN(), vec_res);
block.getByPosition(result).column = std::move(col_res);
}
}
else if (const ColumnArray * col = checkAndGetColumn<ColumnArray>(column.get()))
else if (const ColumnArray * col_arr = checkAndGetColumn<ColumnArray>(column.get()))
{
auto col_res = ColumnVector<ResultType>::create();
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
vec_res.resize(col->size());
Impl::array(col->getOffsets(), vec_res);
vec_res.resize(col_arr->size());
Impl::array(col_arr->getOffsets(), vec_res);
block.getByPosition(result).column = std::move(col_res);
}

View File

@ -60,10 +60,10 @@ public:
Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
block.getByPosition(result).column = std::move(col_res);
}
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column.get()))
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
{
auto col_res = ColumnFixedString::create(col->getN());
Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars());
auto col_res = ColumnFixedString::create(col_fixed->getN());
Impl::vector_fixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars());
block.getByPosition(result).column = std::move(col_res);
}
else

View File

@ -1056,17 +1056,17 @@ public:
block.getByPosition(result).column = std::move(col_res);
}
else if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
else if (const auto col_in_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
{
if (col_in->getN() != uuid_text_length)
if (col_in_fixed->getN() != uuid_text_length)
throw Exception("Illegal type " + col_type_name.type->getName() +
" of column " + col_in->getName() +
" of column " + col_in_fixed->getName() +
" argument of function " + getName() +
", expected FixedString(" + toString(uuid_text_length) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto size = col_in->size();
const auto & vec_in = col_in->getChars();
const auto size = col_in_fixed->size();
const auto & vec_in = col_in_fixed->getChars();
auto col_res = ColumnFixedString::create(uuid_bytes_length);
@ -1087,8 +1087,7 @@ public:
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
+ " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
}
};

View File

@ -371,7 +371,7 @@ struct StringEqualsImpl
&& !memcmp(&a_data[a_offsets[i - 1]], b_data, b_n)));
}
#if __SSE2__
#ifdef __SSE2__
static void NO_INLINE fixed_string_vector_fixed_string_vector_16(
const ColumnString::Chars & a_data,
const ColumnString::Chars & b_data,
@ -428,7 +428,7 @@ struct StringEqualsImpl
/** Specialization if both sizes are 16.
* To more efficient comparison of IPv6 addresses stored in FixedString(16).
*/
#if __SSE2__
#ifdef __SSE2__
if (a_n == 16 && b_n == 16)
{
fixed_string_vector_fixed_string_vector_16(a_data, b_data, c);
@ -448,7 +448,7 @@ struct StringEqualsImpl
PaddedPODArray<UInt8> & c)
{
ColumnString::Offset b_n = b.size();
#if __SSE2__
#ifdef __SSE2__
if (a_n == 16 && b_n == 16)
{
fixed_string_vector_constant_16(a_data, b, c);

View File

@ -1662,7 +1662,7 @@ private:
element_wrappers.reserve(from_element_types.size());
/// Create conversion wrapper for each element in tuple
for (const auto & idx_type : ext::enumerate(from_type->getElements()))
for (const auto idx_type : ext::enumerate(from_type->getElements()))
element_wrappers.push_back(prepareUnpackDictionaries(idx_type.second, to_element_types[idx_type.first]));
return [element_wrappers, from_element_types, to_element_types]
@ -1688,7 +1688,7 @@ private:
element_block.insert({ nullptr, std::make_shared<DataTypeTuple>(to_element_types), "" });
/// invoke conversion for each element
for (const auto & idx_element_wrapper : ext::enumerate(element_wrappers))
for (const auto idx_element_wrapper : ext::enumerate(element_wrappers))
idx_element_wrapper.second(element_block, { idx_element_wrapper.first },
tuple_size + idx_element_wrapper.first, input_rows_count);
@ -2098,35 +2098,35 @@ private:
{
if (const auto type = checkAndGetDataType<DataTypeUInt8>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeUInt16>(to_type))
if (const auto type = checkAndGetDataType<DataTypeUInt16>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeUInt32>(to_type))
if (const auto type = checkAndGetDataType<DataTypeUInt32>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeUInt64>(to_type))
if (const auto type = checkAndGetDataType<DataTypeUInt64>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeInt8>(to_type))
if (const auto type = checkAndGetDataType<DataTypeInt8>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeInt16>(to_type))
if (const auto type = checkAndGetDataType<DataTypeInt16>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeInt32>(to_type))
if (const auto type = checkAndGetDataType<DataTypeInt32>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeInt64>(to_type))
if (const auto type = checkAndGetDataType<DataTypeInt64>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeFloat32>(to_type))
if (const auto type = checkAndGetDataType<DataTypeFloat32>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeFloat64>(to_type))
if (const auto type = checkAndGetDataType<DataTypeFloat64>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeDate>(to_type))
if (const auto type = checkAndGetDataType<DataTypeDate>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeDateTime>(to_type))
if (const auto type = checkAndGetDataType<DataTypeDateTime>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeString>(to_type))
if (const auto type = checkAndGetDataType<DataTypeString>(to_type))
return monotonicityForType(type);
else if (isEnum(from_type))
if (isEnum(from_type))
{
if (const auto type = checkAndGetDataType<DataTypeEnum8>(to_type))
return monotonicityForType(type);
else if (const auto type = checkAndGetDataType<DataTypeEnum16>(to_type))
if (const auto type = checkAndGetDataType<DataTypeEnum16>(to_type))
return monotonicityForType(type);
}
/// other types like Null, FixedString, Array and Tuple have no monotonicity defined

View File

@ -489,8 +489,8 @@ private:
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col_const);
else
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
@ -512,12 +512,12 @@ private:
dictionary->getString(attr_name, ids, default_col, out.get());
block.getByPosition(result).column = std::move(out);
}
else if (const auto default_col = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
{
/// vector ids, const defaults
auto out = ColumnString::create();
const auto & ids = id_col->getData();
String def = default_col->getValue<String>();
String def = default_col_const->getValue<String>();
dictionary->getString(attr_name, ids, def, out.get());
block.getByPosition(result).column = std::move(out);
}
@ -547,12 +547,12 @@ private:
else
block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column
}
else if (const auto default_col = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
{
/// const ids, const defaults
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
auto out = ColumnString::create();
String def = default_col->getValue<String>();
String def = default_col_const->getValue<String>();
dictionary->getString(attr_name, ids, def, out.get());
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
}
@ -588,9 +588,9 @@ private:
{
dict->getString(attr_name, key_columns, key_types, default_col, out.get());
}
else if (const auto default_col = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
{
String def = default_col->getValue<String>();
String def = default_col_const->getValue<String>();
dict->getString(attr_name, key_columns, key_types, def, out.get());
}
else
@ -775,12 +775,12 @@ private:
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
block.getByPosition(result).column = std::move(out);
}
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
{
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
const PaddedPODArray<UInt64> ids(1, id_col_const->getValue<UInt64>());
PaddedPODArray<Type> data(1);
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col_const->size(), toField(data.front()));
}
else
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
@ -982,8 +982,8 @@ private:
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col_const);
else
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
@ -1007,13 +1007,13 @@ private:
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
block.getByPosition(result).column = std::move(out);
}
else if (const auto default_col = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
else if (const auto default_col_const = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
{
/// vector ids, const defaults
auto out = ColumnVector<Type>::create(id_col->size());
const auto & ids = id_col->getData();
auto & data = out->getData();
const auto def = default_col->template getValue<Type>();
const auto def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
block.getByPosition(result).column = std::move(out);
}
@ -1043,12 +1043,12 @@ private:
else
block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column
}
else if (const auto default_col = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
else if (const auto default_col_const = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
{
/// const ids, const defaults
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
PaddedPODArray<Type> data(1);
const auto & def = default_col->template getValue<Type>();
const auto & def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
}
@ -1091,9 +1091,9 @@ private:
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, defs, data);
}
else if (const auto default_col = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
else if (const auto default_col_const = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
{
const auto def = default_col->template getValue<Type>();
const auto def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, def, data);
}
@ -1489,14 +1489,14 @@ private:
get_hierarchies(in, backend->getData(), offsets->getData());
block.getByPosition(result).column = ColumnArray::create(std::move(backend), std::move(offsets));
}
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
{
const PaddedPODArray<UInt64> in(1, id_col->getValue<UInt64>());
const PaddedPODArray<UInt64> in(1, id_col_const->getValue<UInt64>());
auto backend = ColumnUInt64::create();
auto offsets = ColumnArray::ColumnOffsets::create();
get_hierarchies(in, backend->getData(), offsets->getData());
auto array = ColumnArray::create(std::move(backend), std::move(offsets));
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(id_col->size(), (*array)[0].get<Array>());
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(id_col_const->size(), (*array)[0].get<Array>());
}
else
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
@ -1585,8 +1585,8 @@ private:
if (const auto child_id_col = checkAndGetColumn<ColumnUInt64>(child_id_col_untyped))
execute(block, result, dict, child_id_col, ancestor_id_col_untyped);
else if (const auto child_id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(child_id_col_untyped))
execute(block, result, dict, child_id_col, ancestor_id_col_untyped);
else if (const auto child_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(child_id_col_untyped))
execute(block, result, dict, child_id_col_const, ancestor_id_col_untyped);
else
throw Exception{"Illegal column " + child_id_col_untyped->getName()
+ " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
@ -1611,12 +1611,12 @@ private:
dictionary->isInVectorVector(child_ids, ancestor_ids, data);
block.getByPosition(result).column = std::move(out);
}
else if (const auto ancestor_id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
{
auto out = ColumnUInt8::create();
const auto & child_ids = child_id_col->getData();
const auto ancestor_id = ancestor_id_col->getValue<UInt64>();
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
auto & data = out->getData();
const auto size = child_id_col->size();
data.resize(size);
@ -1650,10 +1650,10 @@ private:
dictionary->isInConstantVector(child_id, ancestor_ids, data);
block.getByPosition(result).column = std::move(out);
}
else if (const auto ancestor_id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
{
const auto child_id = child_id_col->getValue<UInt64>();
const auto ancestor_id = ancestor_id_col->getValue<UInt64>();
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
UInt8 res = 0;
dictionary->isInConstantConstant(child_id, ancestor_id, res);

View File

@ -86,9 +86,9 @@ private:
out_const = false;
result[arg_idx] = instr_t{instr_type::get_float_64, col};
}
else if (const auto col = checkAndGetColumnConst<ColumnVector<Float64>>(column))
else if (const auto col_const = checkAndGetColumnConst<ColumnVector<Float64>>(column))
{
result[arg_idx] = instr_t{instr_type::get_const_float_64, col};
result[arg_idx] = instr_t{instr_type::get_const_float_64, col_const};
}
else
throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(),
@ -233,7 +233,7 @@ private:
/// Prepare array of ellipses.
size_t ellipses_count = (arguments.size() - 2) / 4;
Ellipse ellipses[ellipses_count];
std::vector<Ellipse> ellipses(ellipses_count);
for (const auto ellipse_idx : ext::range(0, ellipses_count))
{
@ -285,7 +285,7 @@ private:
size_t start_index = 0;
for (const auto row : ext::range(0, size))
{
dst_data[row] = isPointInEllipses(col_vec_x->getData()[row], col_vec_y->getData()[row], ellipses, ellipses_count, start_index);
dst_data[row] = isPointInEllipses(col_vec_x->getData()[row], col_vec_y->getData()[row], ellipses.data(), ellipses_count, start_index);
}
block.getByPosition(result).column = std::move(dst);
@ -295,7 +295,7 @@ private:
const auto col_const_x = static_cast<const ColumnConst *> (col_x);
const auto col_const_y = static_cast<const ColumnConst *> (col_y);
size_t start_index = 0;
UInt8 res = isPointInEllipses(col_const_x->getValue<Float64>(), col_const_y->getValue<Float64>(), ellipses, ellipses_count, start_index);
UInt8 res = isPointInEllipses(col_const_x->getValue<Float64>(), col_const_y->getValue<Float64>(), ellipses.data(), ellipses_count, start_index);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(size, res);
}
else

View File

@ -622,9 +622,9 @@ private:
vec_to[i] = Impl::combineHashes(vec_to[i], h);
}
}
else if (auto col_from = checkAndGetColumnConst<ColumnVector<FromType>>(column))
else if (auto col_from_const = checkAndGetColumnConst<ColumnVector<FromType>>(column))
{
auto value = col_from->template getValue<FromType>();
auto value = col_from_const->template getValue<FromType>();
ToType hash;
if constexpr (std::is_same_v<ToType, UInt64>)
hash = IntHash64Impl::apply(ext::bit_cast<UInt64>(value));
@ -672,10 +672,10 @@ private:
current_offset = offsets[i];
}
}
else if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(column))
else if (const ColumnFixedString * col_from_fixed = checkAndGetColumn<ColumnFixedString>(column))
{
const typename ColumnString::Chars & data = col_from->getChars();
size_t n = col_from->getN();
const typename ColumnString::Chars & data = col_from_fixed->getChars();
size_t n = col_from_fixed->getN();
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
@ -687,9 +687,9 @@ private:
vec_to[i] = Impl::combineHashes(vec_to[i], h);
}
}
else if (const ColumnConst * col_from = checkAndGetColumnConstStringOrFixedString(column))
else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column))
{
String value = col_from->getValue<String>().data();
String value = col_from_const->getValue<String>().data();
const ToType hash = Impl::apply(value.data(), value.size());
const size_t size = vec_to.size();
@ -749,10 +749,10 @@ private:
current_offset = offsets[i];
}
}
else if (const ColumnConst * col_from = checkAndGetColumnConst<ColumnArray>(column))
else if (const ColumnConst * col_from_const = checkAndGetColumnConst<ColumnArray>(column))
{
/// NOTE: here, of course, you can do without the materialization of the column.
ColumnPtr full_column = col_from->convertToFullColumn();
ColumnPtr full_column = col_from_const->convertToFullColumn();
executeArray<first>(type, &*full_column, vec_to);
}
else
@ -799,9 +799,9 @@ private:
for (size_t i = 0; i < tuple_size; ++i)
executeForArgument(tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first);
}
else if (const ColumnTuple * tuple = checkAndGetColumnConstData<ColumnTuple>(column))
else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData<ColumnTuple>(column))
{
const Columns & tuple_columns = tuple->getColumns();
const Columns & tuple_columns = tuple_const->getColumns();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
size_t tuple_size = tuple_columns.size();
for (size_t i = 0; i < tuple_size; ++i)

View File

@ -25,9 +25,6 @@ namespace
/// And this is from `head -c8 /dev/urandom | xxd -p`
UInt64 current = 0x09826f4a081cee35ULL;
LinearCongruentialGenerator() {}
LinearCongruentialGenerator(UInt64 value) : current(value) {}
void seed(UInt64 value)
{
current = value;

View File

@ -198,12 +198,12 @@ public:
block.getByPosition(result).column = std::move(col_res);
}
else if (const ColumnFixedString * col_from = typeid_cast<const ColumnFixedString *>(block.getByPosition(arguments[0]).column.get()))
else if (const ColumnFixedString * col_from_fixed = typeid_cast<const ColumnFixedString *>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnVector<ToFieldType>::create();
const ColumnString::Chars & data_from = col_from->getChars();
size_t step = col_from->getN();
const ColumnString::Chars & data_from = col_from_fixed->getChars();
size_t step = col_from_fixed->getN();
size_t size = data_from.size() / step;
typename ColumnVector<ToFieldType>::Container & vec_res = col_res->getData();
vec_res.resize(size);

View File

@ -16,7 +16,7 @@
#include <ext/bit_cast.h>
#include <algorithm>
#if __SSE4_1__
#ifdef __SSE4_1__
#include <smmintrin.h>
#endif
@ -62,7 +62,7 @@ enum class ScaleMode
enum class RoundingMode
{
#if __SSE4_1__
#ifdef __SSE4_1__
Round = _MM_FROUND_TO_NEAREST_INT | _MM_FROUND_NO_EXC,
Floor = _MM_FROUND_TO_NEG_INF | _MM_FROUND_NO_EXC,
Ceil = _MM_FROUND_TO_POS_INF | _MM_FROUND_NO_EXC,
@ -118,8 +118,6 @@ struct IntegerRoundingComputation
x = -x;
return x;
}
default:
__builtin_unreachable();
}
}
@ -133,8 +131,6 @@ struct IntegerRoundingComputation
return x;
case ScaleMode::Negative:
return computeImpl(x, scale);
default:
__builtin_unreachable();
}
}
@ -149,7 +145,7 @@ struct IntegerRoundingComputation
};
#if __SSE4_1__
#ifdef __SSE4_1__
template <typename T>
class BaseFloatRoundingComputation;

View File

@ -995,10 +995,10 @@ public:
Impl::vector(col->getChars(), col->getOffsets(), needle, replacement, col_res->getChars(), col_res->getOffsets());
block.getByPosition(result).column = std::move(col_res);
}
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column_src.get()))
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_src.get()))
{
auto col_res = ColumnString::create();
Impl::vector_fixed(col->getChars(), col->getN(), needle, replacement, col_res->getChars(), col_res->getOffsets());
Impl::vector_fixed(col_fixed->getChars(), col_fixed->getN(), needle, replacement, col_res->getChars(), col_res->getOffsets());
block.getByPosition(result).column = std::move(col_res);
}
else

View File

@ -100,20 +100,23 @@ struct ExtractRaw
if (*pos == current_expect_end)
{
expects_end.pop_back();
current_expect_end = (UInt8) (expects_end.empty() ? 0 : expects_end.back());
current_expect_end = expects_end.empty() ? 0 : expects_end.back();
}
else
{
switch (*pos)
{
case '[':
expects_end.push_back((current_expect_end = ']'));
current_expect_end = ']';
expects_end.push_back(current_expect_end);
break;
case '{':
expects_end.push_back((current_expect_end = '}'));
current_expect_end = '}';
expects_end.push_back(current_expect_end);
break;
case '"' :
expects_end.push_back((current_expect_end = '"'));
current_expect_end = '"';
expects_end.push_back(current_expect_end);
break;
case '\\':
/// skip backslash

View File

@ -133,6 +133,7 @@ struct ConstSource : public Base
{
}
ConstSource(const ConstSource &) = default;
virtual ~ConstSource() = default;
virtual void accept(ArraySourceVisitor & visitor) // override

View File

@ -199,8 +199,8 @@ UInt64 PointInPolygonWithGrid<CoordinateType>::getAllocatedBytes() const
size += polygons.capacity() * sizeof(MultiPolygon);
size += getPolygonAllocatedBytes(polygon);
for (const auto & polygon : polygons)
size += getMultiPolygonAllocatedBytes(polygon);
for (const auto & elem : polygons)
size += getMultiPolygonAllocatedBytes(elem);
return size;
}
@ -312,24 +312,23 @@ bool PointInPolygonWithGrid<CoordinateType>::contains(CoordinateType x, Coordina
return cell.half_planes[0].contains(x, y);
case CellType::pairOfLinesSingleConvexPolygon:
return cell.half_planes[0].contains(x, y) && cell.half_planes[1].contains(x, y);
case CellType::pairOfLinesDifferentPolygons:
case CellType::pairOfLinesDifferentPolygons: [[fallthrough]];
case CellType::pairOfLinesSingleNonConvexPolygons:
return cell.half_planes[0].contains(x, y) || cell.half_planes[1].contains(x, y);
case CellType::complexPolygon:
return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]);
default:
return false;
}
__builtin_unreachable();
}
template <typename CoordinateType>
typename PointInPolygonWithGrid<CoordinateType>::Distance
PointInPolygonWithGrid<CoordinateType>::distance(
const PointInPolygonWithGrid<CoordinateType>::Point & point,
const PointInPolygonWithGrid<CoordinateType>::Polygon & polygon)
const PointInPolygonWithGrid<CoordinateType>::Polygon & poly)
{
const auto & outer = polygon.outer();
const auto & outer = poly.outer();
Distance distance = 0;
for (auto i : ext::range(0, outer.size() - 1))
{
@ -341,9 +340,9 @@ PointInPolygonWithGrid<CoordinateType>::distance(
}
template <typename CoordinateType>
bool PointInPolygonWithGrid<CoordinateType>::isConvex(const PointInPolygonWithGrid<CoordinateType>::Polygon & polygon)
bool PointInPolygonWithGrid<CoordinateType>::isConvex(const PointInPolygonWithGrid<CoordinateType>::Polygon & poly)
{
const auto & outer = polygon.outer();
const auto & outer = poly.outer();
/// Segment or point.
if (outer.size() < 4)
return false;

View File

@ -28,7 +28,7 @@ private:
{
const auto flip_case_mask = 'A' ^ 'a';
#if __SSE2__
#ifdef __SSE2__
const auto bytes_sse = sizeof(__m128i);
const auto src_end_sse = src_end - (src_end - src) % bytes_sse;

View File

@ -2,7 +2,7 @@
#include <Poco/UTF8Encoding.h>
#include <Common/UTF8Helpers.h>
#if __SSE2__
#ifdef __SSE2__
#include <emmintrin.h>
#endif
@ -165,7 +165,7 @@ private:
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
{
#if __SSE2__
#ifdef __SSE2__
static constexpr auto bytes_sse = sizeof(__m128i);
auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;

View File

@ -58,7 +58,7 @@ public:
*/
Columns columns_holder(num_elements);
const IColumn * columns[num_elements];
ColumnRawPtrs columns(num_elements);
for (size_t i = 0; i < num_elements; ++i)
{

View File

@ -328,7 +328,7 @@ bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
size_t rank = 0;
for (size_t j = prev_off; j < off; ++j)
{
auto &idx = indices[packFixed<UInt128>(j, count, columns, key_sizes)];;
auto & idx = indices[packFixed<UInt128>(j, count, columns, key_sizes)];
if (!idx)
idx = ++rank;
res_values[j] = idx;

View File

@ -132,9 +132,9 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
else if (const ColumnConst * const_arr = checkAndGetColumnConst<ColumnArray>(col))
{
materialized_columns.emplace_back(const_arr->convertToFullColumn());
const auto & arr = typeid_cast<const ColumnArray &>(*materialized_columns.back().get());
aggregate_arguments_vec[i] = &arr.getData();
offsets_i = &arr.getOffsets();
const auto & materialized_arr = typeid_cast<const ColumnArray &>(*materialized_columns.back().get());
aggregate_arguments_vec[i] = &materialized_arr.getData();
offsets_i = &materialized_arr.getOffsets();
}
else
throw Exception("Illegal column " + col->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -165,17 +165,17 @@ bool FunctionArrayReverse::executeFixedString(const IColumn & src_data, const Co
if (const ColumnFixedString * src_data_concrete = checkAndGetColumn<ColumnFixedString>(&src_data))
{
const size_t n = src_data_concrete->getN();
const ColumnFixedString::Chars & src_data = src_data_concrete->getChars();
const ColumnFixedString::Chars & src_data_chars = src_data_concrete->getChars();
ColumnFixedString::Chars & res_chars = typeid_cast<ColumnFixedString &>(res_data).getChars();
size_t size = src_offsets.size();
res_chars.resize(src_data.size());
res_chars.resize(src_data_chars.size());
ColumnArray::Offset src_prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
const UInt8 * src = &src_data[src_prev_offset * n];
const UInt8 * src_end = &src_data[src_offsets[i] * n];
const UInt8 * src = &src_data_chars[src_prev_offset * n];
const UInt8 * src_end = &src_data_chars[src_offsets[i] * n];
if (src == src_end)
continue;
@ -205,12 +205,12 @@ bool FunctionArrayReverse::executeString(const IColumn & src_data, const ColumnA
const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets();
ColumnString::Offsets & res_string_offsets = typeid_cast<ColumnString &>(res_data).getOffsets();
const ColumnString::Chars & src_data = src_data_concrete->getChars();
const ColumnString::Chars & src_data_chars = src_data_concrete->getChars();
ColumnString::Chars & res_chars = typeid_cast<ColumnString &>(res_data).getChars();
size_t size = src_array_offsets.size();
res_string_offsets.resize(src_string_offsets.size());
res_chars.resize(src_data.size());
res_chars.resize(src_data_chars.size());
ColumnArray::Offset src_array_prev_offset = 0;
ColumnString::Offset res_string_prev_offset = 0;
@ -228,7 +228,7 @@ bool FunctionArrayReverse::executeString(const IColumn & src_data, const ColumnA
auto src_pos = src_string_offsets[src_array_prev_offset + j_reversed - 1];
size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos;
memcpySmallAllowReadWriteOverflow15(&res_chars[res_string_prev_offset], &src_data[src_pos], string_size);
memcpySmallAllowReadWriteOverflow15(&res_chars[res_string_prev_offset], &src_data_chars[src_pos], string_size);
res_string_prev_offset += string_size;
res_string_offsets[src_array_prev_offset + j] = res_string_prev_offset;

View File

@ -125,14 +125,14 @@ private:
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
if (auto * x_vec = checkAndGetColumn<ColumnUInt16>(&x))
dispatchForSecondColumn<Transform>(*x_vec, y, timezone_x, timezone_y, result);
else if (auto * x_vec = checkAndGetColumn<ColumnUInt32>(&x))
dispatchForSecondColumn<Transform>(*x_vec, y, timezone_x, timezone_y, result);
else if (auto * x_const = checkAndGetColumnConst<ColumnUInt16>(&x))
dispatchConstForSecondColumn<Transform>(x_const->getValue<UInt16>(), y, timezone_x, timezone_y, result);
else if (auto * x_const = checkAndGetColumnConst<ColumnUInt32>(&x))
dispatchConstForSecondColumn<Transform>(x_const->getValue<UInt32>(), y, timezone_x, timezone_y, result);
if (auto * x_vec_16 = checkAndGetColumn<ColumnUInt16>(&x))
dispatchForSecondColumn<Transform>(*x_vec_16, y, timezone_x, timezone_y, result);
else if (auto * x_vec_32 = checkAndGetColumn<ColumnUInt32>(&x))
dispatchForSecondColumn<Transform>(*x_vec_32, y, timezone_x, timezone_y, result);
else if (auto * x_const_16 = checkAndGetColumnConst<ColumnUInt16>(&x))
dispatchConstForSecondColumn<Transform>(x_const_16->getValue<UInt16>(), y, timezone_x, timezone_y, result);
else if (auto * x_const_32 = checkAndGetColumnConst<ColumnUInt32>(&x))
dispatchConstForSecondColumn<Transform>(x_const_32->getValue<UInt32>(), y, timezone_x, timezone_y, result);
else
throw Exception("Illegal column for first argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}
@ -143,14 +143,14 @@ private:
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
if (auto * y_vec = checkAndGetColumn<ColumnUInt16>(&y))
vector_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
else if (auto * y_vec = checkAndGetColumn<ColumnUInt32>(&y))
vector_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
else if (auto * y_const = checkAndGetColumnConst<ColumnUInt16>(&y))
vector_constant<Transform>(x, y_const->getValue<UInt16>(), timezone_x, timezone_y, result);
else if (auto * y_const = checkAndGetColumnConst<ColumnUInt32>(&y))
vector_constant<Transform>(x, y_const->getValue<UInt32>(), timezone_x, timezone_y, result);
if (auto * y_vec_16 = checkAndGetColumn<ColumnUInt16>(&y))
vector_vector<Transform>(x, *y_vec_16, timezone_x, timezone_y, result);
else if (auto * y_vec_32 = checkAndGetColumn<ColumnUInt32>(&y))
vector_vector<Transform>(x, *y_vec_32, timezone_x, timezone_y, result);
else if (auto * y_const_16 = checkAndGetColumnConst<ColumnUInt16>(&y))
vector_constant<Transform>(x, y_const_16->getValue<UInt16>(), timezone_x, timezone_y, result);
else if (auto * y_const_32 = checkAndGetColumnConst<ColumnUInt32>(&y))
vector_constant<Transform>(x, y_const_32->getValue<UInt32>(), timezone_x, timezone_y, result);
else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}
@ -161,10 +161,10 @@ private:
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
if (auto * y_vec = checkAndGetColumn<ColumnUInt16>(&y))
constant_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
else if (auto * y_vec = checkAndGetColumn<ColumnUInt32>(&y))
constant_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
if (auto * y_vec_16 = checkAndGetColumn<ColumnUInt16>(&y))
constant_vector<Transform>(x, *y_vec_16, timezone_x, timezone_y, result);
else if (auto * y_vec_32 = checkAndGetColumn<ColumnUInt32>(&y))
constant_vector<Transform>(x, *y_vec_32, timezone_x, timezone_y, result);
else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -82,12 +82,12 @@ namespace
{
if (const ColumnVector<T> * src_data_concrete = checkAndGetColumn<ColumnVector<T>>(&src_data))
{
const PaddedPODArray<T> & src_data = src_data_concrete->getData();
const PaddedPODArray<T> & src_data_vec = src_data_concrete->getData();
PaddedPODArray<T> & res_data = static_cast<ColumnVector<T> &>(res_data_col).getData();
size_t size = src_offsets.size();
res_offsets.resize(size);
res_data.reserve(src_data.size());
res_data.reserve(src_data_vec.size());
if (nullable)
res_null_map->reserve(src_null_map->size());
@ -101,7 +101,7 @@ namespace
{
size_t size_to_write = src_offsets[i] - src_prev_offset;
res_data.resize(res_prev_offset + size_to_write);
memcpy(&res_data[res_prev_offset], &src_data[src_prev_offset], size_to_write * sizeof(T));
memcpy(&res_data[res_prev_offset], &src_data_vec[src_prev_offset], size_to_write * sizeof(T));
if (nullable)
{
@ -142,7 +142,7 @@ namespace
if (const ColumnFixedString * src_data_concrete = checkAndGetColumn<ColumnFixedString>(&src_data))
{
const size_t n = src_data_concrete->getN();
const ColumnFixedString::Chars & src_data = src_data_concrete->getChars();
const ColumnFixedString::Chars & src_data_vec = src_data_concrete->getChars();
auto concrete_res_data = typeid_cast<ColumnFixedString *>(&res_data_col);
if (!concrete_res_data)
@ -151,7 +151,7 @@ namespace
ColumnFixedString::Chars & res_data = concrete_res_data->getChars();
size_t size = src_offsets.size();
res_offsets.resize(size);
res_data.reserve(src_data.size());
res_data.reserve(src_data_vec.size());
if (nullable)
res_null_map->reserve(src_null_map->size());
@ -166,7 +166,7 @@ namespace
size_t size_to_write = src_offsets[i] - src_prev_offset;
size_t prev_res_data_size = res_data.size();
res_data.resize(prev_res_data_size + size_to_write * n);
memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset * n], size_to_write * n);
memcpy(&res_data[prev_res_data_size], &src_data_vec[src_prev_offset * n], size_to_write * n);
if (nullable)
{
@ -215,7 +215,7 @@ namespace
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
ColumnString::Offsets & res_string_offsets = concrete_res_string_offsets->getOffsets();
const ColumnString::Chars & src_data = src_data_concrete->getChars();
const ColumnString::Chars & src_data_vec = src_data_concrete->getChars();
auto concrete_res_data = typeid_cast<ColumnString *>(&res_data_col);
if (!concrete_res_data)
@ -225,7 +225,7 @@ namespace
size_t size = src_array_offsets.size();
res_array_offsets.resize(size);
res_string_offsets.reserve(src_string_offsets.size());
res_data.reserve(src_data.size());
res_data.reserve(src_data_vec.size());
if (nullable)
res_null_map->reserve(src_null_map->size());
@ -257,7 +257,7 @@ namespace
size_t res_data_old_size = res_data.size();
res_data.resize(res_data_old_size + bytes_to_copy);
memcpy(&res_data[res_data_old_size], &src_data[src_string_prev_offset], bytes_to_copy);
memcpy(&res_data[res_data_old_size], &src_data_vec[src_string_prev_offset], bytes_to_copy);
if (nullable)
{

View File

@ -50,10 +50,10 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
if (auto type = checkAndGetDataType<DataTypeEnum8>(block.getByPosition(arguments[0]).type.get()))
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, type->getValues().size());
else if (auto type = checkAndGetDataType<DataTypeEnum16>(block.getByPosition(arguments[0]).type.get()))
block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, type->getValues().size());
if (auto type8 = checkAndGetDataType<DataTypeEnum8>(block.getByPosition(arguments[0]).type.get()))
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, type8->getValues().size());
else if (auto type16 = checkAndGetDataType<DataTypeEnum16>(block.getByPosition(arguments[0]).type.get()))
block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, type16->getValues().size());
else
throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -2,7 +2,7 @@
#include <Functions/FunctionBinaryArithmetic.h>
#include <Functions/intDiv.h>
#if __SSE2__
#ifdef __SSE2__
#define LIBDIVIDE_USE_SSE2 1
#endif
@ -45,7 +45,7 @@ struct DivideIntegralByConstantImpl
const A * a_end = a_pos + size;
ResultType * c_pos = c.data();
#if __SSE2__
#ifdef __SSE2__
static constexpr size_t values_per_sse_register = 16 / sizeof(A);
const A * a_end_sse = a_pos + size / values_per_sse_register * values_per_sse_register;

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