Merge branch 'master' into taiyang-li-pr_add_auc

This commit is contained in:
Alexey Milovidov 2020-03-23 15:06:23 +03:00
commit 958b695618
175 changed files with 1045 additions and 940 deletions

View File

@ -57,6 +57,7 @@ Checks: '-*,
readability-simplify-subscript-expr,
readability-simplify-boolean-expr,
readability-inconsistent-declaration-parameter-name,
readability-identifier-naming,
bugprone-undelegated-constructor,
bugprone-argument-comment,
@ -162,3 +163,43 @@ Checks: '-*,
boost-use-to-string,
'
WarningsAsErrors: '*'
CheckOptions:
- key: readability-identifier-naming.ClassCase
value: CamelCase
- key: readability-identifier-naming.EnumCase
value: CamelCase
- key: readability-identifier-naming.LocalVariableCase
value: lower_case
- key: readability-identifier-naming.StaticConstantCase
value: aNy_CasE
- key: readability-identifier-naming.MemberCase
value: lower_case
- key: readability-identifier-naming.PrivateMemberPrefix
value: ''
- key: readability-identifier-naming.ProtectedMemberPrefix
value: ''
- key: readability-identifier-naming.PublicMemberCase
value: lower_case
- key: readability-identifier-naming.MethodCase
value: camelBack
- key: readability-identifier-naming.PrivateMethodPrefix
value: ''
- key: readability-identifier-naming.ProtectedMethodPrefix
value: ''
- key: readability-identifier-naming.ParameterPackCase
value: lower_case
- key: readability-identifier-naming.StructCase
value: CamelCase
- key: readability-identifier-naming.TemplateTemplateParameterCase
value: CamelCase
- key: readability-identifier-naming.TemplateUsingCase
value: lower_case
- key: readability-identifier-naming.TypeTemplateParameterCase
value: CamelCase
- key: readability-identifier-naming.TypedefCase
value: CamelCase
- key: readability-identifier-naming.UnionCase
value: CamelCase
- key: readability-identifier-naming.UsingCase
value: CamelCase

View File

@ -1,15 +1,18 @@
## ClickHouse release v20.3
### ClickHouse release v20.3.4.10, 2020-03-20
### Bug Fix
* This release also contains all bug fixes from 20.1.8.41
* Fix missing `rows_before_limit_at_least` for queries over http (with processors pipeline). This fixes [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730). [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
### ClickHouse release v20.3.3.6, 2020-03-17
### Bug Fix
* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again.
* This release also contains all bug fixes from 20.1.7.38
* Fix bug in a replication that doesn't allow replication to work if the user has executed mutations on the previous version. This fixes [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again.
* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. I lead to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). This issue was in all ClickHouse releases.
* Fixed the issue: timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)). This issue was in all ClickHouse releases.
* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. Fixes [#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ))
* Allow `ALTER ON CLUSTER` of `Distributed` tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). This issue was in all ClickHouse releases.
* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again.
### ClickHouse release v20.3.2.1, 2020-03-12
@ -252,6 +255,36 @@
## ClickHouse release v20.1
### ClickHouse release v20.1.8.41, 2020-03-20
### Bug Fix
* Fix possible permanent `Cannot schedule a task` error (due to unhandled exception in `ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread`). This fixes [#6833](https://github.com/ClickHouse/ClickHouse/issues/6833). [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat))
* Fix excessive memory consumption in `ALTER` queries (mutations). This fixes [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin))
* Fix bug in backquoting in external dictionaries DDL. This fixes [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin))
### ClickHouse release v20.1.7.38, 2020-03-18
#### Bug Fix
* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. I lead to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). This issue was in all ClickHouse releases.
* Allow `ALTER ON CLUSTER` of `Distributed` tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). This issue was in all ClickHouse releases.
* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. Fixes [#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ))
* Fixed the issue: timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)). This issue was in all ClickHouse releases.
* Now it's not possible to create or add columns with simple cyclic aliases like `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin))
* Fixed the issue when padding at the end of base64 encoded value can be malformed. Update base64 library. This fixes [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov))
* Fix data race at destruction of `Poco::HTTPServer`. It could happen when server is started and immediately shut down. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ))
* Fix possible crash/wrong number of rows in `LIMIT n WITH TIES` when there are a lot of rows equal to n'th row. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix))
* Fix possible mismatched checksums with column TTLs. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ))
* Fix crash when a user tries to `ALTER MODIFY SETTING` for old-formated `MergeTree` table engines family. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin))
* Now we will try finalize mutations more frequently. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin))
* Fix replication protocol incompatibility introduced in [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin))
* Fix not(has()) for the bloom_filter index of array types. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab))
* Fixed the behaviour of `match` and `extract` functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. This fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov))
### Build/Testing/Packaging Improvement
* Exception handling now works correctly on Windows Subsystem for Linux. See https://github.com/ClickHouse-Extras/libunwind/pull/3 This fixes [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv))
### ClickHouse release v20.1.6.30, 2020-03-05
#### Bug Fix

View File

@ -8,8 +8,8 @@
template <typename T>
static T shift10Impl(T x, int exponent)
{
static constexpr ssize_t MIN_EXPONENT = -323;
static constexpr ssize_t MAX_EXPONENT = 308;
static constexpr ssize_t min_exponent = -323;
static constexpr ssize_t max_exponent = 308;
static const long double powers10[] =
{
@ -47,12 +47,12 @@ static T shift10Impl(T x, int exponent)
1e291L,1e292L,1e293L,1e294L,1e295L,1e296L,1e297L,1e298L,1e299L,1e300L,1e301L,1e302L,1e303L,1e304L,1e305L,1e306L,1e307L,1e308L
};
if (unlikely(exponent < MIN_EXPONENT)) /// Note: there are some values below MIN_EXPONENT that is greater than zero.
if (unlikely(exponent < min_exponent)) /// Note: there are some values below MIN_EXPONENT that is greater than zero.
x *= 0; /// Multiplying to keep the sign of zero.
else if (unlikely(exponent > MAX_EXPONENT))
else if (unlikely(exponent > max_exponent))
x *= std::numeric_limits<T>::infinity(); /// Multiplying to keep the sign of infinity.
else
x *= powers10[exponent - MIN_EXPONENT];
x *= powers10[exponent - min_exponent];
return x;
}

View File

@ -5,12 +5,12 @@
int main(int, char **)
{
/** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */
static const time_t TIME = 66130;
static const time_t time = 66130;
const auto & date_lut = DateLUT::instance();
std::cerr << date_lut.toHour(TIME) << std::endl;
std::cerr << date_lut.toDayNum(TIME) << std::endl;
std::cerr << date_lut.toHour(time) << std::endl;
std::cerr << date_lut.toDayNum(time) << std::endl;
const auto * values = reinterpret_cast<const DateLUTImpl::Values *>(&date_lut);

View File

@ -119,7 +119,7 @@ Pool::~Pool()
}
Pool::Entry Pool::Get()
Pool::Entry Pool::get()
{
std::unique_lock<std::mutex> lock(mutex);

View File

@ -185,7 +185,7 @@ public:
~Pool();
/// Allocates connection.
Entry Get();
Entry get();
/// Allocates connection.
/// If database is not accessible, returns empty Entry object.

View File

@ -17,10 +17,10 @@ struct PoolFactory::Impl
std::mutex mutex;
};
PoolWithFailover PoolFactory::Get(const std::string & config_name, unsigned default_connections,
PoolWithFailover PoolFactory::get(const std::string & config_name, unsigned default_connections,
unsigned max_connections, size_t max_tries)
{
return Get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries);
return get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries);
}
/// Duplicate of code from StringUtils.h. Copied here for less dependencies.
@ -72,7 +72,7 @@ static std::string getPoolEntryName(const Poco::Util::AbstractConfiguration & co
return entry_name;
}
PoolWithFailover PoolFactory::Get(const Poco::Util::AbstractConfiguration & config,
PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries)
{

View File

@ -27,13 +27,13 @@ public:
PoolFactory(const PoolFactory &) = delete;
/** Allocates a PoolWithFailover to connect to MySQL. */
PoolWithFailover Get(const std::string & config_name,
PoolWithFailover get(const std::string & config_name,
unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
size_t max_tries = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
/** Allocates a PoolWithFailover to connect to MySQL. */
PoolWithFailover Get(const Poco::Util::AbstractConfiguration & config,
PoolWithFailover get(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name,
unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,

View File

@ -68,7 +68,7 @@ PoolWithFailover::PoolWithFailover(const PoolWithFailover & other)
}
}
PoolWithFailover::Entry PoolWithFailover::Get()
PoolWithFailover::Entry PoolWithFailover::get()
{
Poco::Util::Application & app = Poco::Util::Application::instance();
std::lock_guard<std::mutex> locker(mutex);
@ -89,7 +89,7 @@ PoolWithFailover::Entry PoolWithFailover::Get()
try
{
Entry entry = shareable ? pool->Get() : pool->tryGet();
Entry entry = shareable ? pool->get() : pool->tryGet();
if (!entry.isNull())
{
@ -121,7 +121,7 @@ PoolWithFailover::Entry PoolWithFailover::Get()
if (full_pool)
{
app.logger().error("All connections failed, trying to wait on a full pool " + (*full_pool)->getDescription());
return (*full_pool)->Get();
return (*full_pool)->get();
}
std::stringstream message;

View File

@ -105,6 +105,6 @@ namespace mysqlxx
PoolWithFailover(const PoolWithFailover & other);
/** Allocates a connection to use. */
Entry Get();
Entry get();
};
}

View File

@ -154,7 +154,7 @@ double Value::readFloatText(const char * buf, size_t length) const
void Value::throwException(const char * text) const
{
static constexpr size_t MYSQLXX_QUERY_PREVIEW_LENGTH = 1000;
static constexpr size_t preview_length = 1000;
std::stringstream info;
info << text;
@ -166,7 +166,7 @@ void Value::throwException(const char * text) const
}
if (res && res->getQuery())
info << ", query: " << res->getQuery()->str().substr(0, MYSQLXX_QUERY_PREVIEW_LENGTH);
info << ", query: " << res->getQuery()->str().substr(0, preview_length);
throw CannotParseValue(info.str());
}

View File

@ -1122,7 +1122,7 @@ private:
/// to avoid losing sync.
if (!cancelled)
{
auto cancelQuery = [&] {
auto cancel_query = [&] {
connection->sendCancel();
cancelled = true;
if (is_interactive)
@ -1134,7 +1134,7 @@ private:
if (interrupt_listener.check())
{
cancelQuery();
cancel_query();
}
else
{
@ -1145,7 +1145,7 @@ private:
<< " Waited for " << static_cast<size_t>(elapsed) << " seconds,"
<< " timeout is " << receive_timeout.totalSeconds() << " seconds." << std::endl;
cancelQuery();
cancel_query();
}
}
}

View File

@ -43,11 +43,11 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
void ClusterCopierApp::handleHelp(const std::string &, const std::string &)
{
Poco::Util::HelpFormatter helpFormatter(options());
helpFormatter.setCommand(commandName());
helpFormatter.setHeader("Copies tables from one cluster to another");
helpFormatter.setUsage("--config-file <config-file> --task-path <task-path>");
helpFormatter.format(std::cerr);
Poco::Util::HelpFormatter help_formatter(options());
help_formatter.setCommand(commandName());
help_formatter.setHeader("Copies tables from one cluster to another");
help_formatter.setUsage("--config-file <config-file> --task-path <task-path>");
help_formatter.format(std::cerr);
stopOptionsProcessing();
}

View File

@ -79,11 +79,11 @@ namespace
void ODBCBridge::handleHelp(const std::string &, const std::string &)
{
Poco::Util::HelpFormatter helpFormatter(options());
helpFormatter.setCommand(commandName());
helpFormatter.setHeader("HTTP-proxy for odbc requests");
helpFormatter.setUsage("--http-port <port>");
helpFormatter.format(std::cerr);
Poco::Util::HelpFormatter help_formatter(options());
help_formatter.setCommand(commandName());
help_formatter.setHeader("HTTP-proxy for odbc requests");
help_formatter.setUsage("--http-port <port>");
help_formatter.format(std::cerr);
stopOptionsProcessing();
}

View File

@ -546,7 +546,7 @@ void HTTPHandler::processQuery(
client_info.http_method = http_method;
client_info.http_user_agent = request.get("User-Agent", "");
auto appendCallback = [&context] (ProgressCallback callback)
auto append_callback = [&context] (ProgressCallback callback)
{
auto prev = context.getProgressCallback();
@ -561,13 +561,13 @@ void HTTPHandler::processQuery(
/// While still no data has been sent, we will report about query execution progress by sending HTTP headers.
if (settings.send_progress_in_http_headers)
appendCallback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); });
append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); });
if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close)
{
Poco::Net::StreamSocket & socket = dynamic_cast<Poco::Net::HTTPServerRequestImpl &>(request).socket();
appendCallback([&context, &socket](const Progress &)
append_callback([&context, &socket](const Progress &)
{
/// Assume that at the point this method is called no one is reading data from the socket any more.
/// True for read-only queries.

View File

@ -23,9 +23,9 @@ Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHand
<< ", Content Type: " << request.getContentType()
<< ", Transfer Encoding: " << request.getTransferEncoding());
for (auto & handlerFactory: child_handler_factories)
for (auto & handler_factory : child_handler_factories)
{
auto handler = handlerFactory->createRequestHandler(request);
auto handler = handler_factory->createRequestHandler(request);
if (handler != nullptr)
return handler;
}

View File

@ -252,8 +252,8 @@ void MySQLHandler::comFieldList(ReadBuffer & payload)
ComFieldList packet;
packet.readPayload(payload);
String database = connection_context.getCurrentDatabase();
StoragePtr tablePtr = DatabaseCatalog::instance().getTable({database, packet.table});
for (const NameAndTypePair & column: tablePtr->getColumns().getAll())
StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table});
for (const NameAndTypePair & column: table_ptr->getColumns().getAll())
{
ColumnDefinition column_definition(
database, packet.table, packet.table, column.name, column.name, CharacterSet::binary, 100, ColumnType::MYSQL_TYPE_STRING, 0, 0

View File

@ -56,26 +56,26 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_)
void MySQLHandlerFactory::readRSAKeys()
{
const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config();
String certificateFileProperty = "openSSL.server.certificateFile";
String privateKeyFileProperty = "openSSL.server.privateKeyFile";
String certificate_file_property = "openSSL.server.certificateFile";
String private_key_file_property = "openSSL.server.privateKeyFile";
if (!config.has(certificateFileProperty))
if (!config.has(certificate_file_property))
throw Exception("Certificate file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
if (!config.has(privateKeyFileProperty))
if (!config.has(private_key_file_property))
throw Exception("Private key file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
{
String certificateFile = config.getString(certificateFileProperty);
FILE * fp = fopen(certificateFile.data(), "r");
String certificate_file = config.getString(certificate_file_property);
FILE * fp = fopen(certificate_file.data(), "r");
if (fp == nullptr)
throw Exception("Cannot open certificate file: " + certificateFile + ".", ErrorCodes::CANNOT_OPEN_FILE);
throw Exception("Cannot open certificate file: " + certificate_file + ".", ErrorCodes::CANNOT_OPEN_FILE);
SCOPE_EXIT(fclose(fp));
X509 * x509 = PEM_read_X509(fp, nullptr, nullptr, nullptr);
SCOPE_EXIT(X509_free(x509));
if (x509 == nullptr)
throw Exception("Failed to read PEM certificate from " + certificateFile + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
throw Exception("Failed to read PEM certificate from " + certificate_file + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
EVP_PKEY * p = X509_get_pubkey(x509);
if (p == nullptr)
@ -88,16 +88,16 @@ void MySQLHandlerFactory::readRSAKeys()
}
{
String privateKeyFile = config.getString(privateKeyFileProperty);
String private_key_file = config.getString(private_key_file_property);
FILE * fp = fopen(privateKeyFile.data(), "r");
FILE * fp = fopen(private_key_file.data(), "r");
if (fp == nullptr)
throw Exception ("Cannot open private key file " + privateKeyFile + ".", ErrorCodes::CANNOT_OPEN_FILE);
throw Exception ("Cannot open private key file " + private_key_file + ".", ErrorCodes::CANNOT_OPEN_FILE);
SCOPE_EXIT(fclose(fp));
private_key.reset(PEM_read_RSAPrivateKey(fp, nullptr, nullptr, nullptr));
if (!private_key)
throw Exception("Failed to read RSA private key from " + privateKeyFile + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
throw Exception("Failed to read RSA private key from " + private_key_file + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
}
}

View File

@ -162,12 +162,12 @@ int Server::run()
{
if (config().hasOption("help"))
{
Poco::Util::HelpFormatter helpFormatter(Server::options());
Poco::Util::HelpFormatter help_formatter(Server::options());
std::stringstream header;
header << commandName() << " [OPTION] [-- [ARG]...]\n";
header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010";
helpFormatter.setHeader(header.str());
helpFormatter.format(std::cout);
help_formatter.setHeader(header.str());
help_formatter.format(std::cout);
return 0;
}
if (config().hasOption("version"))

View File

@ -101,7 +101,7 @@ boost::shared_ptr<const EnabledQuota::Intervals> QuotaCache::QuotaInfo::rebuildI
new_intervals->quota_key = key;
auto & intervals = new_intervals->intervals;
intervals.reserve(quota->all_limits.size());
constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE;
static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE;
for (const auto & limits : quota->all_limits)
{
intervals.emplace_back();

View File

@ -20,32 +20,32 @@ namespace ErrorCodes
namespace
{
template <typename T, typename Tlimit_num_elems>
template <typename T, typename LimitNumberOfElements>
struct MovingSum
{
using DataType = MovingSumData<T>;
using Function = MovingImpl<T, Tlimit_num_elems, DataType>;
using Function = MovingImpl<T, LimitNumberOfElements, DataType>;
};
template <typename T, typename Tlimit_num_elems>
template <typename T, typename LimitNumberOfElements>
struct MovingAvg
{
using DataType = MovingAvgData<T>;
using Function = MovingImpl<T, Tlimit_num_elems, DataType>;
using Function = MovingImpl<T, LimitNumberOfElements, DataType>;
};
template <typename T, typename Tlimit_num_elems> using MovingSumTemplate = typename MovingSum<T, Tlimit_num_elems>::Function;
template <typename T, typename Tlimit_num_elems> using MovingAvgTemplate = typename MovingAvg<T, Tlimit_num_elems>::Function;
template <typename T, typename LimitNumberOfElements> using MovingSumTemplate = typename MovingSum<T, LimitNumberOfElements>::Function;
template <typename T, typename LimitNumberOfElements> using MovingAvgTemplate = typename MovingAvg<T, LimitNumberOfElements>::Function;
template <template <typename, typename> class Function, typename has_limit, typename ... TArgs>
template <template <typename, typename> class Function, typename HasLimit, typename ... TArgs>
inline AggregateFunctionPtr createAggregateFunctionMovingImpl(const std::string & name, const DataTypePtr & argument_type, TArgs ... args)
{
AggregateFunctionPtr res;
if (isDecimal(argument_type))
res.reset(createWithDecimalType<Function, has_limit>(*argument_type, argument_type, std::forward<TArgs>(args)...));
res.reset(createWithDecimalType<Function, HasLimit>(*argument_type, argument_type, std::forward<TArgs>(args)...));
else
res.reset(createWithNumericType<Function, has_limit>(*argument_type, argument_type, std::forward<TArgs>(args)...));
res.reset(createWithNumericType<Function, HasLimit>(*argument_type, argument_type, std::forward<TArgs>(args)...));
if (!res)
throw Exception("Illegal type " + argument_type->getName() + " of argument for aggregate function " + name,

View File

@ -21,48 +21,48 @@ namespace
{
/// Substitute return type for Date and DateTime
template <typename has_limit>
class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, has_limit>
template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>
{
public:
explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, has_limit>(argument_type, max_elems_) {}
: AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>(argument_type, max_elems_) {}
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDate>()); }
};
template <typename has_limit>
class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, has_limit>
template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>
{
public:
explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, has_limit>(argument_type, max_elems_) {}
: AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>(argument_type, max_elems_) {}
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>()); }
};
template <typename has_limit, typename ... TArgs>
template <typename HasLimit, typename ... TArgs>
static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate<has_limit>(argument_type, std::forward<TArgs>(args)...);
else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime<has_limit>(argument_type, std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate<HasLimit>(argument_type, std::forward<TArgs>(args)...);
else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime<HasLimit>(argument_type, std::forward<TArgs>(args)...);
else
{
/// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric
if (argument_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
return new AggregateFunctionGroupUniqArrayGeneric<true, has_limit>(argument_type, std::forward<TArgs>(args)...);
return new AggregateFunctionGroupUniqArrayGeneric<true, HasLimit>(argument_type, std::forward<TArgs>(args)...);
else
return new AggregateFunctionGroupUniqArrayGeneric<false, has_limit>(argument_type, std::forward<TArgs>(args)...);
return new AggregateFunctionGroupUniqArrayGeneric<false, HasLimit>(argument_type, std::forward<TArgs>(args)...);
}
}
template <typename has_limit, typename ... TArgs>
template <typename HasLimit, typename ... TArgs>
inline AggregateFunctionPtr createAggregateFunctionGroupUniqArrayImpl(const std::string & name, const DataTypePtr & argument_type, TArgs ... args)
{
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionGroupUniqArray, has_limit, const DataTypePtr &, TArgs...>(*argument_type, argument_type, std::forward<TArgs>(args)...));
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionGroupUniqArray, HasLimit, const DataTypePtr &, TArgs...>(*argument_type, argument_type, std::forward<TArgs>(args)...));
if (!res)
res = AggregateFunctionPtr(createWithExtraTypes<has_limit>(argument_type, std::forward<TArgs>(args)...));
res = AggregateFunctionPtr(createWithExtraTypes<HasLimit>(argument_type, std::forward<TArgs>(args)...));
if (!res)
throw Exception("Illegal type " + argument_type->getName() +

View File

@ -130,7 +130,7 @@ LinearModelData::LinearModelData(
gradient_batch.resize(param_num_ + 1, Float64{0.0});
}
void LinearModelData::update_state()
void LinearModelData::updateState()
{
if (batch_size == 0)
return;
@ -197,7 +197,7 @@ void LinearModelData::merge(const DB::LinearModelData & rhs)
if (iter_num == 0 && rhs.iter_num == 0)
return;
update_state();
updateState();
/// can't update rhs state because it's constant
/// squared mean is more stable (in sence of quality of prediction) when two states with quietly different number of learning steps are merged
@ -219,13 +219,13 @@ void LinearModelData::add(const IColumn ** columns, size_t row_num)
Float64 target = (*columns[0]).getFloat64(row_num);
/// Here we have columns + 1 as first column corresponds to target value, and others - to features
weights_updater->add_to_batch(
weights_updater->addToBatch(
gradient_batch, *gradient_computer, weights, bias, l2_reg_coef, target, columns + 1, row_num);
++batch_size;
if (batch_size == batch_capacity)
{
update_state();
updateState();
}
}
@ -300,7 +300,7 @@ void Adam::update(UInt64 batch_size, std::vector<Float64> & weights, Float64 & b
beta2_powered_ *= beta2_;
}
void Adam::add_to_batch(
void Adam::addToBatch(
std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer,
const std::vector<Float64> & weights,
@ -358,7 +358,7 @@ void Nesterov::update(UInt64 batch_size, std::vector<Float64> & weights, Float64
bias += accumulated_gradient[weights.size()];
}
void Nesterov::add_to_batch(
void Nesterov::addToBatch(
std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer,
const std::vector<Float64> & weights,
@ -432,7 +432,7 @@ void StochasticGradientDescent::update(
bias += (learning_rate * batch_gradient[weights.size()]) / batch_size;
}
void IWeightsUpdater::add_to_batch(
void IWeightsUpdater::addToBatch(
std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer,
const std::vector<Float64> & weights,

View File

@ -112,7 +112,7 @@ public:
virtual ~IWeightsUpdater() = default;
/// Calls GradientComputer to update current mini-batch
virtual void add_to_batch(
virtual void addToBatch(
std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer,
const std::vector<Float64> & weights,
@ -176,7 +176,7 @@ public:
Nesterov(Float64 alpha) : alpha_(alpha) {}
void add_to_batch(
void addToBatch(
std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer,
const std::vector<Float64> & weights,
@ -209,7 +209,7 @@ public:
beta2_powered_ = beta2_;
}
void add_to_batch(
void addToBatch(
std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer,
const std::vector<Float64> & weights,
@ -289,7 +289,7 @@ private:
/** The function is called when we want to flush current batch and update our weights
*/
void update_state();
void updateState();
};

View File

@ -91,13 +91,13 @@ MutableColumnPtr ColumnAggregateFunction::predictValues(Block & block, const Col
MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn();
res->reserve(data.size());
auto ML_function = func.get();
if (ML_function)
auto machine_learning_function = func.get();
if (machine_learning_function)
{
if (data.size() == 1)
{
/// Case for const column. Predict using single model.
ML_function->predictValues(data[0], *res, block, 0, block.rows(), arguments, context);
machine_learning_function->predictValues(data[0], *res, block, 0, block.rows(), arguments, context);
}
else
{
@ -105,7 +105,7 @@ MutableColumnPtr ColumnAggregateFunction::predictValues(Block & block, const Col
size_t row_num = 0;
for (auto val : data)
{
ML_function->predictValues(val, *res, block, row_num, 1, arguments, context);
machine_learning_function->predictValues(val, *res, block, row_num, 1, arguments, context);
++row_num;
}
}

View File

@ -152,16 +152,15 @@ void ColumnArray::insertData(const char * pos, size_t length)
{
/** Similarly - only for arrays of fixed length values.
*/
IColumn * data_ = &getData();
if (!data_->isFixedAndContiguous())
if (!data->isFixedAndContiguous())
throw Exception("Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
size_t field_size = data_->sizeOfValueIfFixed();
size_t field_size = data->sizeOfValueIfFixed();
const char * end = pos + length;
size_t elems = 0;
for (; pos + field_size <= end; pos += field_size, ++elems)
data_->insertData(pos, field_size);
data->insertData(pos, field_size);
if (pos != end)
throw Exception("Incorrect length argument for method ColumnArray::insertData", ErrorCodes::BAD_ARGUMENTS);
@ -278,12 +277,12 @@ int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_dir
namespace
{
template <bool positive>
struct less
struct Less
{
const ColumnArray & parent;
int nan_direction_hint;
less(const ColumnArray & parent_, int nan_direction_hint_)
Less(const ColumnArray & parent_, int nan_direction_hint_)
: parent(parent_), nan_direction_hint(nan_direction_hint_) {}
bool operator()(size_t lhs, size_t rhs) const
@ -300,7 +299,7 @@ namespace
void ColumnArray::reserve(size_t n)
{
getOffsets().reserve(n);
getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1.
getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1.
}
@ -689,16 +688,16 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h
if (limit)
{
if (reverse)
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this, nan_direction_hint));
std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less<false>(*this, nan_direction_hint));
else
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this, nan_direction_hint));
std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less<true>(*this, nan_direction_hint));
}
else
{
if (reverse)
std::sort(res.begin(), res.end(), less<false>(*this, nan_direction_hint));
std::sort(res.begin(), res.end(), Less<false>(*this, nan_direction_hint));
else
std::sort(res.begin(), res.end(), less<true>(*this, nan_direction_hint));
std::sort(res.begin(), res.end(), Less<true>(*this, nan_direction_hint));
}
}
@ -738,13 +737,13 @@ ColumnPtr ColumnArray::replicateNumber(const Offsets & replicate_offsets) const
if (0 == col_size)
return res;
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
ColumnArray & res_arr = typeid_cast<ColumnArray &>(*res);
const typename ColumnVector<T>::Container & src_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
const Offsets & src_offsets = getOffsets();
typename ColumnVector<T>::Container & res_data = typeid_cast<ColumnVector<T> &>(res_.getData()).getData();
Offsets & res_offsets = res_.getOffsets();
typename ColumnVector<T>::Container & res_data = typeid_cast<ColumnVector<T> &>(res_arr.getData()).getData();
Offsets & res_offsets = res_arr.getOffsets();
res_data.reserve(data->size() / col_size * replicate_offsets.back());
res_offsets.reserve(replicate_offsets.back());
@ -789,16 +788,16 @@ ColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets) const
if (0 == col_size)
return res;
ColumnArray & res_ = assert_cast<ColumnArray &>(*res);
ColumnArray & res_arr = assert_cast<ColumnArray &>(*res);
const ColumnString & src_string = typeid_cast<const ColumnString &>(*data);
const ColumnString::Chars & src_chars = src_string.getChars();
const Offsets & src_string_offsets = src_string.getOffsets();
const Offsets & src_offsets = getOffsets();
ColumnString::Chars & res_chars = typeid_cast<ColumnString &>(res_.getData()).getChars();
Offsets & res_string_offsets = typeid_cast<ColumnString &>(res_.getData()).getOffsets();
Offsets & res_offsets = res_.getOffsets();
ColumnString::Chars & res_chars = typeid_cast<ColumnString &>(res_arr.getData()).getChars();
Offsets & res_string_offsets = typeid_cast<ColumnString &>(res_arr.getData()).getOffsets();
Offsets & res_offsets = res_arr.getOffsets();
res_chars.reserve(src_chars.size() / col_size * replicate_offsets.back());
res_string_offsets.reserve(src_string_offsets.size() / col_size * replicate_offsets.back());

View File

@ -124,8 +124,8 @@ std::vector<MutableColumnPtr> ColumnFunction::scatter(IColumn::ColumnIndex num_c
for (IColumn::ColumnIndex part = 0; part < num_columns; ++part)
{
auto & capture = captures[part];
size_t size__ = capture.empty() ? counts[part] : capture.front().column->size();
columns.emplace_back(ColumnFunction::create(size__, function, std::move(capture)));
size_t capture_size = capture.empty() ? counts[part] : capture.front().column->size();
columns.emplace_back(ColumnFunction::create(capture_size, function, std::move(capture)));
}
return columns;

View File

@ -401,25 +401,25 @@ void ColumnLowCardinality::Index::callForType(Callback && callback, size_t size_
size_t ColumnLowCardinality::Index::getSizeOfIndexType(const IColumn & column, size_t hint)
{
auto checkFor = [&](auto type) { return typeid_cast<const ColumnVector<decltype(type)> *>(&column) != nullptr; };
auto tryGetSizeFor = [&](auto type) -> size_t { return checkFor(type) ? sizeof(decltype(type)) : 0; };
auto check_for = [&](auto type) { return typeid_cast<const ColumnVector<decltype(type)> *>(&column) != nullptr; };
auto try_get_size_for = [&](auto type) -> size_t { return check_for(type) ? sizeof(decltype(type)) : 0; };
if (hint)
{
size_t size = 0;
callForType([&](auto type) { size = tryGetSizeFor(type); }, hint);
callForType([&](auto type) { size = try_get_size_for(type); }, hint);
if (size)
return size;
}
if (auto size = tryGetSizeFor(UInt8()))
if (auto size = try_get_size_for(UInt8()))
return size;
if (auto size = tryGetSizeFor(UInt16()))
if (auto size = try_get_size_for(UInt16()))
return size;
if (auto size = tryGetSizeFor(UInt32()))
if (auto size = try_get_size_for(UInt32()))
return size;
if (auto size = tryGetSizeFor(UInt64()))
if (auto size = try_get_size_for(UInt64()))
return size;
throw Exception("Unexpected indexes type for ColumnLowCardinality. Expected UInt, got " + column.getName(),
@ -515,13 +515,13 @@ UInt64 ColumnLowCardinality::Index::getMaxPositionForCurrentType() const
size_t ColumnLowCardinality::Index::getPositionAt(size_t row) const
{
size_t pos;
auto getPosition = [&](auto type)
auto get_position = [&](auto type)
{
using CurIndexType = decltype(type);
pos = getPositionsData<CurIndexType>()[row];
};
callForType(std::move(getPosition), size_of_type);
callForType(std::move(get_position), size_of_type);
return pos;
}
@ -536,7 +536,7 @@ void ColumnLowCardinality::Index::insertPosition(UInt64 position)
void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, UInt64 offset, UInt64 limit)
{
auto insertForType = [&](auto type)
auto insert_for_type = [&](auto type)
{
using ColumnType = decltype(type);
const auto * column_ptr = typeid_cast<const ColumnVector<ColumnType> *>(&column);
@ -570,10 +570,10 @@ void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, U
return true;
};
if (!insertForType(UInt8()) &&
!insertForType(UInt16()) &&
!insertForType(UInt32()) &&
!insertForType(UInt64()))
if (!insert_for_type(UInt8()) &&
!insert_for_type(UInt16()) &&
!insert_for_type(UInt32()) &&
!insert_for_type(UInt64()))
throw Exception("Invalid column for ColumnLowCardinality index. Expected UInt, got " + column.getName(),
ErrorCodes::ILLEGAL_COLUMN);

View File

@ -223,24 +223,24 @@ Dwarf::Path::Path(std::string_view baseDir, std::string_view subDir, std::string
size_t Dwarf::Path::size() const
{
size_t size = 0;
bool needsSlash = false;
bool needs_slash = false;
if (!baseDir_.empty())
{
size += baseDir_.size();
needsSlash = baseDir_.back() != '/';
needs_slash = baseDir_.back() != '/';
}
if (!subDir_.empty())
{
size += needsSlash;
size += needs_slash;
size += subDir_.size();
needsSlash = subDir_.back() != '/';
needs_slash = subDir_.back() != '/';
}
if (!file_.empty())
{
size += needsSlash;
size += needs_slash;
size += file_.size();
}
@ -249,38 +249,38 @@ size_t Dwarf::Path::size() const
size_t Dwarf::Path::toBuffer(char * buf, size_t bufSize) const
{
size_t totalSize = 0;
bool needsSlash = false;
size_t total_size = 0;
bool needs_slash = false;
auto append = [&](std::string_view sp)
{
if (bufSize >= 2)
{
size_t toCopy = std::min(sp.size(), bufSize - 1);
memcpy(buf, sp.data(), toCopy);
buf += toCopy;
bufSize -= toCopy;
size_t to_copy = std::min(sp.size(), bufSize - 1);
memcpy(buf, sp.data(), to_copy);
buf += to_copy;
bufSize -= to_copy;
}
totalSize += sp.size();
total_size += sp.size();
};
if (!baseDir_.empty())
{
append(baseDir_);
needsSlash = baseDir_.back() != '/';
needs_slash = baseDir_.back() != '/';
}
if (!subDir_.empty())
{
if (needsSlash)
if (needs_slash)
{
append("/");
}
append(subDir_);
needsSlash = subDir_.back() != '/';
needs_slash = subDir_.back() != '/';
}
if (!file_.empty())
{
if (needsSlash)
if (needs_slash)
{
append("/");
}
@ -291,14 +291,14 @@ size_t Dwarf::Path::toBuffer(char * buf, size_t bufSize) const
*buf = '\0';
}
SAFE_CHECK(totalSize == size(), "Size mismatch");
return totalSize;
SAFE_CHECK(total_size == size(), "Size mismatch");
return total_size;
}
void Dwarf::Path::toString(std::string & dest) const
{
size_t initialSize = dest.size();
dest.reserve(initialSize + size());
size_t initial_size = dest.size();
dest.reserve(initial_size + size());
if (!baseDir_.empty())
{
dest.append(baseDir_.begin(), baseDir_.end());
@ -319,7 +319,7 @@ void Dwarf::Path::toString(std::string & dest) const
}
dest.append(file_.begin(), file_.end());
}
SAFE_CHECK(dest.size() == initialSize + size(), "Size mismatch");
SAFE_CHECK(dest.size() == initial_size + size(), "Size mismatch");
}
// Next chunk in section
@ -332,9 +332,9 @@ bool Dwarf::Section::next(std::string_view & chunk)
// Initial length is a uint32_t value for a 32-bit section, and
// a 96-bit value (0xffffffff followed by the 64-bit length) for a 64-bit
// section.
auto initialLength = read<uint32_t>(chunk);
is64Bit_ = (initialLength == uint32_t(-1));
auto length = is64Bit_ ? read<uint64_t>(chunk) : initialLength;
auto initial_length = read<uint32_t>(chunk);
is64Bit_ = (initial_length == uint32_t(-1));
auto length = is64Bit_ ? read<uint64_t>(chunk) : initial_length;
SAFE_CHECK(length <= chunk.size(), "invalid DWARF section");
chunk = std::string_view(chunk.data(), length);
data_ = std::string_view(chunk.end(), data_.end() - chunk.end());
@ -387,7 +387,7 @@ bool Dwarf::readAbbreviation(std::string_view & section, DIEAbbreviation & abbr)
abbr.hasChildren = (read<uint8_t>(section) != DW_CHILDREN_no);
// attributes
const char * attributeBegin = section.data();
const char * attribute_begin = section.data();
for (;;)
{
SAFE_CHECK(!section.empty(), "invalid attribute section");
@ -396,7 +396,7 @@ bool Dwarf::readAbbreviation(std::string_view & section, DIEAbbreviation & abbr)
break;
}
abbr.attributes = std::string_view(attributeBegin, section.data() - attributeBegin);
abbr.attributes = std::string_view(attribute_begin, section.data() - attribute_begin);
return true;
}
@ -483,18 +483,18 @@ std::string_view Dwarf::getStringFromStringSection(uint64_t offset) const
*/
bool Dwarf::findDebugInfoOffset(uintptr_t address, std::string_view aranges, uint64_t & offset)
{
Section arangesSection(aranges);
Section aranges_section(aranges);
std::string_view chunk;
while (arangesSection.next(chunk))
while (aranges_section.next(chunk))
{
auto version = read<uint16_t>(chunk);
SAFE_CHECK(version == 2, "invalid aranges version");
offset = readOffset(chunk, arangesSection.is64Bit());
auto addressSize = read<uint8_t>(chunk);
SAFE_CHECK(addressSize == sizeof(uintptr_t), "invalid address size");
auto segmentSize = read<uint8_t>(chunk);
SAFE_CHECK(segmentSize == 0, "segmented architecture not supported");
offset = readOffset(chunk, aranges_section.is64Bit());
auto address_size = read<uint8_t>(chunk);
SAFE_CHECK(address_size == sizeof(uintptr_t), "invalid address size");
auto segment_size = read<uint8_t>(chunk);
SAFE_CHECK(segment_size == 0, "segmented architecture not supported");
// Padded to a multiple of 2 addresses.
// Strangely enough, this is the only place in the DWARF spec that requires
@ -537,15 +537,15 @@ bool Dwarf::findLocation(uintptr_t address, std::string_view & infoEntry, Locati
// 3. debug_abbrev_offset (4B or 8B): offset into the .debug_abbrev section
// 4. address_size (1B)
Section debugInfoSection(infoEntry);
Section debug_info_section(infoEntry);
std::string_view chunk;
SAFE_CHECK(debugInfoSection.next(chunk), "invalid debug info");
SAFE_CHECK(debug_info_section.next(chunk), "invalid debug info");
auto version = read<uint16_t>(chunk);
SAFE_CHECK(version >= 2 && version <= 4, "invalid info version");
uint64_t abbrevOffset = readOffset(chunk, debugInfoSection.is64Bit());
auto addressSize = read<uint8_t>(chunk);
SAFE_CHECK(addressSize == sizeof(uintptr_t), "invalid address size");
uint64_t abbrev_offset = readOffset(chunk, debug_info_section.is64Bit());
auto address_size = read<uint8_t>(chunk);
SAFE_CHECK(address_size == sizeof(uintptr_t), "invalid address size");
// We survived so far. The first (and only) DIE should be DW_TAG_compile_unit
// NOTE: - binutils <= 2.25 does not issue DW_TAG_partial_unit.
@ -553,16 +553,16 @@ bool Dwarf::findLocation(uintptr_t address, std::string_view & infoEntry, Locati
// TODO(tudorb): Handle DW_TAG_partial_unit?
auto code = readULEB(chunk);
SAFE_CHECK(code != 0, "invalid code");
auto abbr = getAbbreviation(code, abbrevOffset);
auto abbr = getAbbreviation(code, abbrev_offset);
SAFE_CHECK(abbr.tag == DW_TAG_compile_unit, "expecting compile unit entry");
// Skip children entries, remove_prefix to the next compilation unit entry.
infoEntry.remove_prefix(chunk.end() - infoEntry.begin());
// Read attributes, extracting the few we care about
bool foundLineOffset = false;
uint64_t lineOffset = 0;
std::string_view compilationDirectory;
std::string_view mainFileName;
bool found_line_offset = false;
uint64_t line_offset = 0;
std::string_view compilation_directory;
std::string_view main_file_name;
DIEAbbreviation::Attribute attr;
std::string_view attributes = abbr.attributes;
@ -573,43 +573,43 @@ bool Dwarf::findLocation(uintptr_t address, std::string_view & infoEntry, Locati
{
break;
}
auto val = readAttributeValue(chunk, attr.form, debugInfoSection.is64Bit());
auto val = readAttributeValue(chunk, attr.form, debug_info_section.is64Bit());
switch (attr.name)
{
case DW_AT_stmt_list:
// Offset in .debug_line for the line number VM program for this
// compilation unit
lineOffset = std::get<uint64_t>(val);
foundLineOffset = true;
line_offset = std::get<uint64_t>(val);
found_line_offset = true;
break;
case DW_AT_comp_dir:
// Compilation directory
compilationDirectory = std::get<std::string_view>(val);
compilation_directory = std::get<std::string_view>(val);
break;
case DW_AT_name:
// File name of main file being compiled
mainFileName = std::get<std::string_view>(val);
main_file_name = std::get<std::string_view>(val);
break;
}
}
if (!mainFileName.empty())
if (!main_file_name.empty())
{
locationInfo.hasMainFile = true;
locationInfo.mainFile = Path(compilationDirectory, "", mainFileName);
locationInfo.mainFile = Path(compilation_directory, "", main_file_name);
}
if (!foundLineOffset)
if (!found_line_offset)
{
return false;
}
std::string_view lineSection(line_);
lineSection.remove_prefix(lineOffset);
LineNumberVM lineVM(lineSection, compilationDirectory);
std::string_view line_section(line_);
line_section.remove_prefix(line_offset);
LineNumberVM line_vm(line_section, compilation_directory);
// Execute line number VM program to find file and line
locationInfo.hasFileAndLine = lineVM.findAddress(address, locationInfo.file, locationInfo.line);
locationInfo.hasFileAndLine = line_vm.findAddress(address, locationInfo.file, locationInfo.line);
return locationInfo.hasFileAndLine;
}
@ -635,9 +635,9 @@ bool Dwarf::findAddress(uintptr_t address, LocationInfo & locationInfo, Location
if (findDebugInfoOffset(address, aranges_, offset))
{
// Read compilation unit header from .debug_info
std::string_view infoEntry(info_);
infoEntry.remove_prefix(offset);
findLocation(address, infoEntry, locationInfo);
std::string_view info_entry(info_);
info_entry.remove_prefix(offset);
findLocation(address, info_entry, locationInfo);
return locationInfo.hasFileAndLine;
}
else if (mode == LocationInfoMode::FAST)
@ -657,9 +657,9 @@ bool Dwarf::findAddress(uintptr_t address, LocationInfo & locationInfo, Location
// Slow path (linear scan): Iterate over all .debug_info entries
// and look for the address in each compilation unit.
std::string_view infoEntry(info_);
while (!infoEntry.empty() && !locationInfo.hasFileAndLine)
findLocation(address, infoEntry, locationInfo);
std::string_view info_entry(info_);
while (!info_entry.empty() && !locationInfo.hasFileAndLine)
findLocation(address, info_entry, locationInfo);
return locationInfo.hasFileAndLine;
}
@ -693,16 +693,16 @@ void Dwarf::LineNumberVM::init()
{
version_ = read<uint16_t>(data_);
SAFE_CHECK(version_ >= 2 && version_ <= 4, "invalid version in line number VM");
uint64_t headerLength = readOffset(data_, is64Bit_);
SAFE_CHECK(headerLength <= data_.size(), "invalid line number VM header length");
std::string_view header(data_.data(), headerLength);
uint64_t header_length = readOffset(data_, is64Bit_);
SAFE_CHECK(header_length <= data_.size(), "invalid line number VM header length");
std::string_view header(data_.data(), header_length);
data_ = std::string_view(header.end(), data_.end() - header.end());
minLength_ = read<uint8_t>(header);
if (version_ == 4)
{ // Version 2 and 3 records don't have this
uint8_t maxOpsPerInstruction = read<uint8_t>(header);
SAFE_CHECK(maxOpsPerInstruction == 1, "VLIW not supported");
uint8_t max_ops_per_instruction = read<uint8_t>(header);
SAFE_CHECK(max_ops_per_instruction == 1, "VLIW not supported");
}
defaultIsStmt_ = read<uint8_t>(header);
lineBase_ = read<int8_t>(header); // yes, signed
@ -752,10 +752,10 @@ Dwarf::LineNumberVM::FileName Dwarf::LineNumberVM::getFileName(uint64_t index) c
FileName fn;
if (index <= fileNameCount_)
{
std::string_view fileNames = fileNames_;
std::string_view file_names = fileNames_;
for (; index; --index)
{
if (!readFileName(fileNames, fn))
if (!readFileName(file_names, fn))
{
abort();
}
@ -783,11 +783,11 @@ std::string_view Dwarf::LineNumberVM::getIncludeDirectory(uint64_t index) const
SAFE_CHECK(index <= includeDirectoryCount_, "invalid include directory");
std::string_view includeDirectories = includeDirectories_;
std::string_view include_directories = includeDirectories_;
std::string_view dir;
for (; index; --index)
{
dir = readNullTerminated(includeDirectories);
dir = readNullTerminated(include_directories);
if (dir.empty())
{
abort(); // BUG
@ -825,8 +825,8 @@ bool Dwarf::LineNumberVM::nextDefineFile(std::string_view & program, FileName &
if (opcode != 0)
{ // standard opcode
// Skip, slurp the appropriate number of LEB arguments
uint8_t argCount = standardOpcodeLengths_[opcode - 1];
while (argCount--)
uint8_t arg_count = standardOpcodeLengths_[opcode - 1];
while (arg_count--)
{
readULEB(program);
}
@ -858,11 +858,11 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro
if (opcode >= opcodeBase_)
{ // special opcode
uint8_t adjustedOpcode = opcode - opcodeBase_;
uint8_t opAdvance = adjustedOpcode / lineRange_;
uint8_t adjusted_opcode = opcode - opcodeBase_;
uint8_t op_advance = adjusted_opcode / lineRange_;
address_ += minLength_ * opAdvance;
line_ += lineBase_ + adjustedOpcode % lineRange_;
address_ += minLength_ * op_advance;
line_ += lineBase_ + adjusted_opcode % lineRange_;
basicBlock_ = false;
prologueEnd_ = false;
@ -932,8 +932,8 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro
// Unrecognized standard opcode, slurp the appropriate number of LEB
// arguments.
uint8_t argCount = standardOpcodeLengths_[opcode - 1];
while (argCount--)
uint8_t arg_count = standardOpcodeLengths_[opcode - 1];
while (arg_count--)
{
readULEB(program);
}
@ -944,10 +944,10 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro
auto length = readULEB(program);
// the opcode itself should be included in the length, so length >= 1
SAFE_CHECK(length != 0, "invalid extended opcode length");
auto extendedOpcode = read<uint8_t>(program);
auto extended_opcode = read<uint8_t>(program);
--length;
switch (extendedOpcode)
switch (extended_opcode)
{
case DW_LNE_end_sequence:
return END;
@ -987,15 +987,15 @@ bool Dwarf::LineNumberVM::findAddress(uintptr_t target, Path & file, uint64_t &
State state = START;
reset();
uint64_t prevFile = 0;
uint64_t prevLine = 0;
uint64_t prev_file = 0;
uint64_t prev_line = 0;
while (!program.empty())
{
bool seqEnd = !next(program);
bool seq_end = !next(program);
if (state == START)
{
if (!seqEnd)
if (!seq_end)
{
state = address_ <= target ? LOW_SEQ : HIGH_SEQ;
}
@ -1008,20 +1008,20 @@ bool Dwarf::LineNumberVM::findAddress(uintptr_t target, Path & file, uint64_t &
// Found it! Note that ">" is indeed correct (not ">="), as each
// sequence is guaranteed to have one entry past-the-end (emitted by
// DW_LNE_end_sequence)
if (prevFile == 0)
if (prev_file == 0)
{
return false;
}
auto fn = getFileName(prevFile);
auto fn = getFileName(prev_file);
file = Path(compilationDirectory_, getIncludeDirectory(fn.directoryIndex), fn.relativeName);
line = prevLine;
line = prev_line;
return true;
}
prevFile = file_;
prevLine = line_;
prev_file = file_;
prev_line = line_;
}
if (seqEnd)
if (seq_end)
{
state = START;
reset();

View File

@ -74,7 +74,7 @@ public:
/** The `Entry` object protects the resource from being used by another thread.
* The following methods are forbidden for `rvalue`, so you can not write a similar to
*
* auto q = pool.Get()->query("SELECT .."); // Oops, after this line Entry was destroyed
* auto q = pool.get()->query("SELECT .."); // Oops, after this line Entry was destroyed
* q.execute (); // Someone else can use this Connection
*/
Object * operator->() && = delete;

View File

@ -64,7 +64,7 @@ public:
private:
/// A separate method which binds the lock holder to the owned lock
/// N.B. It is very important that this method produces no allocations
bool bind_with(RWLock && parent_, GroupsContainer::iterator it_group_) noexcept
bool bindWith(RWLock && parent_, GroupsContainer::iterator it_group_) noexcept
{
if (bound)
return false;
@ -170,7 +170,7 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
/// N.B. Type is Read here, query_id is not empty and it_query is a valid iterator
all_read_locks.add(query_id); /// SM1: may throw on insertion (nothing to roll back)
++it_query->second; /// SM2: nothrow
lock_holder->bind_with(shared_from_this(), current_owner_group); /// SM3: nothrow
lock_holder->bindWith(shared_from_this(), current_owner_group); /// SM3: nothrow
finalize_metrics();
return lock_holder;
@ -244,7 +244,7 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
}
}
lock_holder->bind_with(shared_from_this(), it_group); /// SM: nothrow
lock_holder->bindWith(shared_from_this(), it_group); /// SM: nothrow
finalize_metrics();
return lock_holder;

View File

@ -212,11 +212,11 @@ void ThreadFuzzer::setup()
if (sigaction(SIGPROF, &sa, nullptr))
throwFromErrno("Failed to setup signal handler for thread fuzzer", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
static constexpr UInt32 TIMER_PRECISION = 1000000;
static constexpr UInt32 timer_precision = 1000000;
struct timeval interval;
interval.tv_sec = cpu_time_period_us / TIMER_PRECISION;
interval.tv_usec = cpu_time_period_us % TIMER_PRECISION;
interval.tv_sec = cpu_time_period_us / timer_precision;
interval.tv_usec = cpu_time_period_us % timer_precision;
struct itimerval timer = {.it_interval = interval, .it_value = interval};

View File

@ -758,17 +758,17 @@ struct ZooKeeperMultiResponse final : MultiResponse, ZooKeeperResponse
{
ZooKeeper::OpNum op_num;
bool done;
int32_t error_;
int32_t error_read;
Coordination::read(op_num, in);
Coordination::read(done, in);
Coordination::read(error_, in);
Coordination::read(error_read, in);
if (!done)
throw Exception("Too many results received for multi transaction", ZMARSHALLINGERROR);
if (op_num != -1)
throw Exception("Unexpected op_num received at the end of results for multi transaction", ZMARSHALLINGERROR);
if (error_ != -1)
if (error_read != -1)
throw Exception("Unexpected error value received at the end of results for multi transaction", ZMARSHALLINGERROR);
}
}

View File

@ -32,7 +32,7 @@ struct SmallLock
{
std::atomic<int> locked {false};
bool try_lock()
bool tryLock()
{
int expected = 0;
return locked.compare_exchange_strong(expected, 1, std::memory_order_acquire);

View File

@ -61,7 +61,7 @@ UInt32 getCompressedDataSize(UInt8 data_bytes_size, UInt32 uncompressed_size)
return (items_count * max_item_size_bits + 8) / 8;
}
struct binary_value_info
struct BinaryValueInfo
{
UInt8 leading_zero_bits;
UInt8 data_bits;
@ -69,7 +69,7 @@ struct binary_value_info
};
template <typename T>
binary_value_info getLeadingAndTrailingBits(const T & value)
BinaryValueInfo getLeadingAndTrailingBits(const T & value)
{
constexpr UInt8 bit_size = sizeof(T) * 8;
@ -77,7 +77,7 @@ binary_value_info getLeadingAndTrailingBits(const T & value)
const UInt8 tz = getTrailingZeroBits(value);
const UInt8 data_size = value == 0 ? 0 : static_cast<UInt8>(bit_size - lz - tz);
return binary_value_info{lz, data_size, tz};
return BinaryValueInfo{lz, data_size, tz};
}
template <typename T>
@ -99,7 +99,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
T prev_value{};
// That would cause first XORed value to be written in-full.
binary_value_info prev_xored_info{0, 0, 0};
BinaryValueInfo prev_xored_info{0, 0, 0};
if (source < source_end)
{
@ -118,7 +118,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
source += sizeof(curr_value);
const auto xored_data = curr_value ^ prev_value;
const binary_value_info curr_xored_info = getLeadingAndTrailingBits(xored_data);
const BinaryValueInfo curr_xored_info = getLeadingAndTrailingBits(xored_data);
if (xored_data == 0)
{
@ -177,14 +177,14 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
BitReader reader(source, source_size - sizeof(items_count) - sizeof(prev_value));
binary_value_info prev_xored_info{0, 0, 0};
BinaryValueInfo prev_xored_info{0, 0, 0};
// since data is tightly packed, up to 1 bit per value, and last byte is padded with zeroes,
// we have to keep track of items to avoid reading more that there is.
for (UInt32 items_read = 1; items_read < items_count && !reader.eof(); ++items_read)
{
T curr_value = prev_value;
binary_value_info curr_xored_info = prev_xored_info;
BinaryValueInfo curr_xored_info = prev_xored_info;
T xored_data{};
if (reader.readBit() == 1)

View File

@ -171,7 +171,7 @@ public:
return current_value;
}
size_t ItemsLeft() const
size_t itemsLeft() const
{
return reinterpret_cast<const char *>(data_end) - reinterpret_cast<const char *>(data);
}
@ -184,7 +184,7 @@ public:
explicit operator bool() const
{
return ItemsLeft() > 0;
return itemsLeft() > 0;
}
private:
@ -230,7 +230,7 @@ template <typename T, typename ContainerLeft, typename ContainerRight>
auto l = AsSequenceOf<T>(left);
auto r = AsSequenceOf<T>(right);
const auto MAX_MISMATCHING_ITEMS = 5;
static constexpr auto MAX_MISMATCHING_ITEMS = 5;
int mismatching_items = 0;
size_t i = 0;
@ -607,7 +607,7 @@ TEST_P(CodecTestPerformance, TranscodingWithDataType)
}
}
auto computeMeanAndStdDev = [](const auto & values)
auto compute_mean_and_stddev = [](const auto & values)
{
double mean{};
@ -648,7 +648,7 @@ TEST_P(CodecTestPerformance, TranscodingWithDataType)
for (const auto & k : {"encoding", "decoding"})
{
const auto & values = results[k];
const auto & [mean, std_dev] = computeMeanAndStdDev(values);
const auto & [mean, std_dev] = compute_mean_and_stddev(values);
// Ensure that Coefficient of variation is reasonably low, otherwise these numbers are meaningless
EXPECT_GT(0.05, std_dev / mean);
std::cerr << "\t" << std::fixed << std::setprecision(1) << mean / 1000.0;
@ -1128,7 +1128,7 @@ template <typename ValueType>
auto DDCompatibilityTestSequence()
{
// Generates sequences with double delta in given range.
auto ddGenerator = [prev_delta = static_cast<Int64>(0), prev = static_cast<Int64>(0)](auto dd) mutable
auto dd_generator = [prev_delta = static_cast<Int64>(0), prev = static_cast<Int64>(0)](auto dd) mutable
{
const auto curr = dd + prev + prev_delta;
prev = curr;
@ -1150,7 +1150,7 @@ auto DDCompatibilityTestSequence()
// - 4 is to allow DD value to settle before transitioning through important point,
// since DD depends on 2 previous values of data, + 2 is arbitrary.
ret.append(generateSeq<ValueType>(G(ddGenerator), p - 4, p + 2));
ret.append(generateSeq<ValueType>(G(dd_generator), p - 4, p + 2));
}
return ret;

View File

@ -191,7 +191,7 @@ private:
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
A a = c0_const->template getValue<A>();
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
constant_vector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
constantVector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
@ -200,7 +200,7 @@ private:
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
B b = c1_const->template getValue<B>();
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
vector_constant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
vectorConstant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
@ -209,7 +209,7 @@ private:
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
{
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
vectorVector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
@ -260,7 +260,7 @@ private:
}
template <bool scale_left, bool scale_right>
static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, PaddedPODArray<UInt8> & c,
static void NO_INLINE vectorVector(const ArrayA & a, const ArrayB & b, PaddedPODArray<UInt8> & c,
CompareInt scale)
{
size_t size = a.size();
@ -279,7 +279,7 @@ private:
}
template <bool scale_left, bool scale_right>
static void NO_INLINE vector_constant(const ArrayA & a, B b, PaddedPODArray<UInt8> & c, CompareInt scale)
static void NO_INLINE vectorConstant(const ArrayA & a, B b, PaddedPODArray<UInt8> & c, CompareInt scale)
{
size_t size = a.size();
const A * a_pos = a.data();
@ -295,7 +295,7 @@ private:
}
template <bool scale_left, bool scale_right>
static void NO_INLINE constant_vector(A a, const ArrayB & b, PaddedPODArray<UInt8> & c, CompareInt scale)
static void NO_INLINE constantVector(A a, const ArrayB & b, PaddedPODArray<UInt8> & c, CompareInt scale)
{
size_t size = b.size();
const B * b_pos = b.data();

View File

@ -205,15 +205,15 @@ std::conditional_t<Kind == ArgumentKind::Optional, std::optional<T>, T>
getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name)
{
using NearestResultType = NearestFieldType<T>;
const auto fieldType = Field::TypeToEnum<NearestResultType>::value;
const auto field_type = Field::TypeToEnum<NearestResultType>::value;
const ASTLiteral * argument = nullptr;
auto exceptionMessage = [=](const String & message)
auto exception_message = [=](const String & message)
{
return std::string("Parameter #") + std::to_string(argument_index) + " '"
+ argument_name + "' for " + context_data_type_name
+ message
+ ", expected: " + Field::Types::toString(fieldType) + " literal.";
+ ", expected: " + Field::Types::toString(field_type) + " literal.";
};
if (!arguments || arguments->children.size() <= argument_index
@ -222,12 +222,12 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume
if constexpr (Kind == ArgumentKind::Optional)
return {};
else
throw Exception(exceptionMessage(" is missing"),
throw Exception(exception_message(" is missing"),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
if (argument->value.getType() != fieldType)
throw Exception(exceptionMessage(String(" has wrong type: ") + argument->value.getTypeName()),
if (argument->value.getType() != field_type)
throw Exception(exception_message(String(" has wrong type: ") + argument->value.getTypeName()),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return argument->value.get<NearestResultType>();

View File

@ -604,7 +604,7 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
auto * low_cardinality_state = checkAndGetLowCardinalityDeserializeState(state);
KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value);
auto readDictionary = [this, low_cardinality_state, keys_stream]()
auto read_dictionary = [this, low_cardinality_state, keys_stream]()
{
UInt64 num_keys;
readIntBinary(num_keys, *keys_stream);
@ -617,7 +617,7 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
low_cardinality_state->global_dictionary = std::move(column_unique);
};
auto readAdditionalKeys = [this, low_cardinality_state, indexes_stream]()
auto read_additional_keys = [this, low_cardinality_state, indexes_stream]()
{
UInt64 num_keys;
readIntBinary(num_keys, *indexes_stream);
@ -636,7 +636,7 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
}
};
auto readIndexes = [this, low_cardinality_state, indexes_stream, &low_cardinality_column](UInt64 num_rows)
auto read_indexes = [this, low_cardinality_state, indexes_stream, &low_cardinality_column](UInt64 num_rows)
{
auto indexes_type = low_cardinality_state->index_type.getDataType();
MutableColumnPtr indexes_column = indexes_type->createColumn();
@ -715,12 +715,12 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
!global_dictionary || index_type.need_update_dictionary || low_cardinality_state->need_update_dictionary;
if (index_type.need_global_dictionary && need_update_dictionary)
{
readDictionary();
read_dictionary();
low_cardinality_state->need_update_dictionary = false;
}
if (low_cardinality_state->index_type.has_additional_keys)
readAdditionalKeys();
read_additional_keys();
else
low_cardinality_state->additional_keys = nullptr;
@ -728,7 +728,7 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
}
size_t num_rows_to_read = std::min<UInt64>(limit, low_cardinality_state->num_pending_rows);
readIndexes(num_rows_to_read);
read_indexes(num_rows_to_read);
limit -= num_rows_to_read;
low_cardinality_state->num_pending_rows -= num_rows_to_read;
}

View File

@ -46,8 +46,7 @@ String getExceptionMessagePrefix(const DataTypes & types)
DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_nothing, bool force_support_conversion)
{
auto getNothingOrThrow = [throw_if_result_is_nothing, & types](const std::string & reason)
auto get_nothing_or_throw = [throw_if_result_is_nothing, & types](const std::string & reason)
{
if (throw_if_result_is_nothing)
throw Exception(getExceptionMessagePrefix(types) + reason, ErrorCodes::NO_COMMON_TYPE);
@ -92,7 +91,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
{
for (const auto & type : types)
if (typeid_cast<const DataTypeNothing *>(type.get()))
return getNothingOrThrow(" because some of them are Nothing");
return get_nothing_or_throw(" because some of them are Nothing");
}
/// For Arrays
@ -117,7 +116,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (have_array)
{
if (!all_arrays)
return getNothingOrThrow(" because some of them are Array and some of them are not");
return get_nothing_or_throw(" because some of them are Array and some of them are not");
return std::make_shared<DataTypeArray>(getMostSubtype(nested_types, false, force_support_conversion));
}
@ -143,7 +142,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
nested_types[elem_idx].reserve(types.size());
}
else if (tuple_size != type_tuple->getElements().size())
return getNothingOrThrow(" because Tuples have different sizes");
return get_nothing_or_throw(" because Tuples have different sizes");
have_tuple = true;
@ -157,7 +156,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (have_tuple)
{
if (!all_tuples)
return getNothingOrThrow(" because some of them are Tuple and some of them are not");
return get_nothing_or_throw(" because some of them are Tuple and some of them are not");
DataTypes common_tuple_types(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
@ -218,7 +217,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (!fixed_string_type)
fixed_string_type = type;
else if (!type->equals(*fixed_string_type))
return getNothingOrThrow(" because some of them are FixedStrings with different length");
return get_nothing_or_throw(" because some of them are FixedStrings with different length");
}
else if (isString(type))
have_string = true;
@ -229,7 +228,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (have_string)
{
if (!all_strings)
return getNothingOrThrow(" because some of them are String/FixedString and some of them are not");
return get_nothing_or_throw(" because some of them are String/FixedString and some of them are not");
return fixed_string_type ? fixed_string_type : std::make_shared<DataTypeString>();
}
@ -251,7 +250,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (have_date_or_datetime)
{
if (!all_date_or_datetime)
return getNothingOrThrow(" because some of them are Date/DateTime and some of them are not");
return get_nothing_or_throw(" because some of them are Date/DateTime and some of them are not");
return std::make_shared<DataTypeDate>();
}
@ -300,7 +299,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (min_bits_of_signed_integer || min_bits_of_unsigned_integer || min_mantissa_bits_of_floating)
{
if (!all_numbers)
return getNothingOrThrow(" because some of them are numbers and some of them are not");
return get_nothing_or_throw(" because some of them are numbers and some of them are not");
/// If the result must be floating.
if (!min_bits_of_signed_integer && !min_bits_of_unsigned_integer)
@ -349,7 +348,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
}
/// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases).
return getNothingOrThrow("");
return get_nothing_or_throw("");
}
}

View File

@ -260,7 +260,7 @@ std::map<String, UInt64> DatabaseMySQL::fetchTablesWithModificationTime() const
" WHERE TABLE_SCHEMA = " << quote << database_name_in_mysql;
std::map<String, UInt64> tables_with_modification_time;
MySQLBlockInputStream result(mysql_pool.Get(), query.str(), tables_status_sample_block, DEFAULT_BLOCK_SIZE);
MySQLBlockInputStream result(mysql_pool.get(), query.str(), tables_status_sample_block, DEFAULT_BLOCK_SIZE);
while (Block block = result.read())
{
@ -305,7 +305,7 @@ std::map<String, NamesAndTypesList> DatabaseMySQL::fetchTablesColumnsList(const
<< " AND TABLE_NAME IN " << toQueryStringWithQuote(tables_name) << " ORDER BY ORDINAL_POSITION";
const auto & external_table_functions_use_nulls = global_context.getSettings().external_table_functions_use_nulls;
MySQLBlockInputStream result(mysql_pool.Get(), query.str(), tables_columns_sample_block, DEFAULT_BLOCK_SIZE);
MySQLBlockInputStream result(mysql_pool.get(), query.str(), tables_columns_sample_block, DEFAULT_BLOCK_SIZE);
while (Block block = result.read())
{
size_t rows = block.rows();

View File

@ -198,7 +198,7 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
AtomicStopwatch watch;
std::atomic<size_t> tables_processed{0};
auto startupOneTable = [&](const StoragePtr & table)
auto startup_one_table = [&](const StoragePtr & table)
{
table->startup();
logAboutProgress(log, ++tables_processed, total_tables, watch);
@ -207,7 +207,7 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
try
{
for (const auto & table : tables)
thread_pool.scheduleOrThrowOnError([&]() { startupOneTable(table.second); });
thread_pool.scheduleOrThrowOnError([&]() { startup_one_table(table.second); });
}
catch (...)
{

View File

@ -209,7 +209,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re
void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -218,7 +218,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
{
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
};
factory.registerSource("clickhouse", createTableSource);
factory.registerSource("clickhouse", create_table_source);
}
}

View File

@ -58,8 +58,8 @@ DictionaryPtr DictionaryFactory::create(
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const
{
auto configurationFromAST = getDictionaryConfigurationFromAST(ast);
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context, true);
auto configuration = getDictionaryConfigurationFromAST(ast);
return DictionaryFactory::create(name, *configuration, "dictionary", context, true);
}
bool DictionaryFactory::isComplex(const std::string & layout_type) const

View File

@ -216,7 +216,7 @@ std::string ExecutableDictionarySource::toString() const
void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -236,7 +236,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
dict_struct, config, config_prefix + ".executable",
sample_block, context);
};
factory.registerSource("executable", createTableSource);
factory.registerSource("executable", create_table_source);
}
}

View File

@ -70,7 +70,7 @@ Poco::Timestamp FileDictionarySource::getLastModification() const
void registerDictionarySourceFile(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -86,7 +86,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context, check_config);
};
factory.registerSource("file", createTableSource);
factory.registerSource("file", create_table_source);
}
}

View File

@ -192,7 +192,7 @@ std::string HTTPDictionarySource::toString() const
void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -206,7 +206,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
dict_struct, config, config_prefix + ".http",
sample_block, context, check_config);
};
factory.registerSource("http", createTableSource);
factory.registerSource("http", create_table_source);
}
}

View File

@ -151,9 +151,9 @@ LibraryDictionarySource::LibraryDictionarySource(
#endif
);
settings = std::make_shared<CStringsHolder>(getLibSettings(config, config_prefix + lib_config_settings));
if (auto libNew = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings), decltype(&ClickHouseLibrary::log))>(
if (auto lib_new = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings), decltype(&ClickHouseLibrary::log))>(
"ClickHouseDictionary_v3_libNew"))
lib_data = libNew(&settings->strings, ClickHouseLibrary::log);
lib_data = lib_new(&settings->strings, ClickHouseLibrary::log);
}
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
@ -166,18 +166,18 @@ LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource &
, description{other.description}
, settings{other.settings}
{
if (auto libClone = library->tryGet<decltype(lib_data) (*)(decltype(other.lib_data))>("ClickHouseDictionary_v3_libClone"))
lib_data = libClone(other.lib_data);
if (auto lib_clone = library->tryGet<decltype(lib_data) (*)(decltype(other.lib_data))>("ClickHouseDictionary_v3_libClone"))
lib_data = lib_clone(other.lib_data);
else if (
auto libNew = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings), decltype(&ClickHouseLibrary::log))>(
auto lib_new = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings), decltype(&ClickHouseLibrary::log))>(
"ClickHouseDictionary_v3_libNew"))
lib_data = libNew(&settings->strings, ClickHouseLibrary::log);
lib_data = lib_new(&settings->strings, ClickHouseLibrary::log);
}
LibraryDictionarySource::~LibraryDictionarySource()
{
if (auto libDelete = library->tryGet<void (*)(decltype(lib_data))>("ClickHouseDictionary_v3_libDelete"))
libDelete(lib_data);
if (auto lib_delete = library->tryGet<void (*)(decltype(lib_data))>("ClickHouseDictionary_v3_libDelete"))
lib_delete(lib_data);
}
BlockInputStreamPtr LibraryDictionarySource::loadAll()
@ -196,10 +196,10 @@ BlockInputStreamPtr LibraryDictionarySource::loadAll()
void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw.
auto func_loadAll
auto func_load_all
= library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&columns))>("ClickHouseDictionary_v3_loadAll");
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v3_dataNew")(lib_data);
auto data = func_loadAll(data_ptr, &settings->strings, &columns);
auto data = func_load_all(data_ptr, &settings->strings, &columns);
auto block = dataToBlock(description.sample_block, data);
SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v3_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block);
@ -222,11 +222,11 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> &
void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw.
auto func_loadIds
auto func_load_ids
= library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&columns_pass), decltype(&ids_data))>(
"ClickHouseDictionary_v3_loadIds");
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v3_dataNew")(lib_data);
auto data = func_loadIds(data_ptr, &settings->strings, &columns_pass, &ids_data);
auto data = func_load_ids(data_ptr, &settings->strings, &columns_pass, &ids_data);
auto block = dataToBlock(description.sample_block, data);
SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v3_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block);
@ -256,10 +256,10 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column
void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw.
auto func_loadKeys = library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&request_cols))>(
auto func_load_keys = library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&request_cols))>(
"ClickHouseDictionary_v3_loadKeys");
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v3_dataNew")(lib_data);
auto data = func_loadKeys(data_ptr, &settings->strings, &request_cols);
auto data = func_load_keys(data_ptr, &settings->strings, &request_cols);
auto block = dataToBlock(description.sample_block, data);
SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v3_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block);
@ -267,17 +267,17 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column
bool LibraryDictionarySource::isModified() const
{
if (auto func_isModified
if (auto func_is_modified
= library->tryGet<bool (*)(decltype(lib_data), decltype(&settings->strings))>("ClickHouseDictionary_v3_isModified"))
return func_isModified(lib_data, &settings->strings);
return func_is_modified(lib_data, &settings->strings);
return true;
}
bool LibraryDictionarySource::supportsSelectiveLoad() const
{
if (auto func_supportsSelectiveLoad
if (auto func_supports_selective_load
= library->tryGet<bool (*)(decltype(lib_data), decltype(&settings->strings))>("ClickHouseDictionary_v3_supportsSelectiveLoad"))
return func_supportsSelectiveLoad(lib_data, &settings->strings);
return func_supports_selective_load(lib_data, &settings->strings);
return true;
}
@ -293,7 +293,7 @@ std::string LibraryDictionarySource::toString() const
void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -302,7 +302,7 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
{
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
};
factory.registerSource("library", createTableSource);
factory.registerSource("library", create_table_source);
}
}

View File

@ -12,7 +12,7 @@ namespace ErrorCodes
void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -29,7 +29,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("mongodb", createTableSource);
factory.registerSource("mongodb", create_table_source);
}
}
@ -319,16 +319,16 @@ BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_column
break;
case AttributeUnderlyingType::utString:
String _str(get<String>((*key_columns[attr.first])[row_idx]));
String loaded_str(get<String>((*key_columns[attr.first])[row_idx]));
/// Convert string to ObjectID
if (attr.second.is_object_id)
{
Poco::MongoDB::ObjectId::Ptr _id(new Poco::MongoDB::ObjectId(_str));
key.add(attr.second.name, _id);
Poco::MongoDB::ObjectId::Ptr loaded_id(new Poco::MongoDB::ObjectId(loaded_str));
key.add(attr.second.name, loaded_id);
}
else
{
key.add(attr.second.name, _str);
key.add(attr.second.name, loaded_str);
}
break;
}

View File

@ -14,7 +14,7 @@ namespace ErrorCodes
void registerDictionarySourceMysql(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -31,7 +31,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("mysql", createTableSource);
factory.registerSource("mysql", create_table_source);
}
}
@ -67,7 +67,7 @@ MySQLDictionarySource::MySQLDictionarySource(
, update_field{config.getString(config_prefix + ".update_field", "")}
, dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)}
, sample_block{sample_block_}
, pool{mysqlxx::PoolFactory::instance().Get(config, config_prefix)}
, pool{mysqlxx::PoolFactory::instance().get(config, config_prefix)}
, query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}
, load_all_query{query_builder.composeLoadAllQuery()}
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
@ -115,7 +115,7 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
BlockInputStreamPtr MySQLDictionarySource::loadAll()
{
auto connection = pool.Get();
auto connection = pool.get();
last_modification = getLastModification(connection, false);
LOG_TRACE(log, load_all_query);
@ -124,7 +124,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadAll()
BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
{
auto connection = pool.Get();
auto connection = pool.get();
last_modification = getLastModification(connection, false);
std::string load_update_query = getUpdateFieldAndDate();
@ -137,7 +137,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & i
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
const auto query = query_builder.composeLoadIdsQuery(ids);
return std::make_shared<MySQLBlockInputStream>(pool.Get(), query, sample_block, max_block_size, close_connection);
return std::make_shared<MySQLBlockInputStream>(pool.get(), query, sample_block, max_block_size, close_connection);
}
BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
@ -145,7 +145,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns,
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
return std::make_shared<MySQLBlockInputStream>(pool.Get(), query, sample_block, max_block_size, close_connection);
return std::make_shared<MySQLBlockInputStream>(pool.get(), query, sample_block, max_block_size, close_connection);
}
bool MySQLDictionarySource::isModified() const
@ -161,7 +161,7 @@ bool MySQLDictionarySource::isModified() const
if (dont_check_update_time)
return true;
auto connection = pool.Get();
auto connection = pool.get();
return getLastModification(connection, true) > last_modification;
}
@ -221,7 +221,7 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry &
if (auto row = result.fetch())
{
++fetched_rows;
const auto UPDATE_TIME_IDX = 12;
static const auto UPDATE_TIME_IDX = 12;
const auto & update_time_value = row[UPDATE_TIME_IDX];
if (!update_time_value.isNull())
@ -259,7 +259,7 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request
Block invalidate_sample_block;
ColumnPtr column(ColumnString::create());
invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
MySQLBlockInputStream block_input_stream(pool.Get(), request, invalidate_sample_block, 1, close_connection);
MySQLBlockInputStream block_input_stream(pool.get(), request, invalidate_sample_block, 1, close_connection);
return readInvalidateQuery(block_input_stream);
}

View File

@ -124,7 +124,7 @@ namespace DB
for (const auto i : ext::range(0, size))
columns[i] = description.sample_block.getByPosition(i).column->cloneEmpty();
const auto insertValueByIdx = [this, &columns](size_t idx, const auto & value)
const auto insert_value_by_idx = [this, &columns](size_t idx, const auto & value)
{
if (description.types[idx].second)
{
@ -170,9 +170,9 @@ namespace DB
/// null string means 'no value for requested key'
if (!value.isNull())
{
insertValueByIdx(0, primary_key);
insertValueByIdx(1, secondary_key);
insertValueByIdx(2, value);
insert_value_by_idx(0, primary_key);
insert_value_by_idx(1, secondary_key);
insert_value_by_idx(2, value);
++num_rows;
}
}
@ -198,8 +198,8 @@ namespace DB
/// Null string means 'no value for requested key'
if (!value.isNull())
{
insertValueByIdx(0, key);
insertValueByIdx(1, value);
insert_value_by_idx(0, key);
insert_value_by_idx(1, value);
}
}
cursor += need_values;

View File

@ -12,7 +12,7 @@ namespace DB
void registerDictionarySourceRedis(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
Block & sample_block,
@ -29,7 +29,7 @@ namespace DB
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("redis", createTableSource);
factory.registerSource("redis", create_table_source);
}
}

View File

@ -688,7 +688,7 @@ static void trieTraverse(const btrie_t * trie, Getter && getter)
node = node->left;
}
auto getBit = [&high_bit](size_t size) { return size ? (high_bit >> (size - 1)) : 0; };
auto get_bit = [&high_bit](size_t size) { return size ? (high_bit >> (size - 1)) : 0; };
while (!stack.empty())
{
@ -703,13 +703,13 @@ static void trieTraverse(const btrie_t * trie, Getter && getter)
if (node && node->right)
{
stack.push(nullptr);
key |= getBit(stack.size());
key |= get_bit(stack.size());
stack.push(node->right);
while (stack.top()->left)
stack.push(stack.top()->left);
}
else
key &= ~getBit(stack.size());
key &= ~get_bit(stack.size());
}
}
@ -740,13 +740,13 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
{
using BlockInputStreamType = DictionaryBlockInputStream<TrieDictionary, UInt64>;
auto getKeys = [](const Columns & columns, const std::vector<DictionaryAttribute> & dict_attributes)
auto get_keys = [](const Columns & columns, const std::vector<DictionaryAttribute> & dict_attributes)
{
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> & dict_attributes)
auto get_view = [](const Columns & columns, const std::vector<DictionaryAttribute> & dict_attributes)
{
auto column = ColumnString::create();
const auto & ip_column = assert_cast<const ColumnFixedString &>(*columns.front());
@ -765,7 +765,7 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
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));
shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view));
}

View File

@ -236,7 +236,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
Poco::Data::ODBC::Connector::registerConnector();
#endif
auto createTableSource = [=](const DictionaryStructure & dict_struct,
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -256,12 +256,12 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("odbc", createTableSource);
factory.registerSource("odbc", create_table_source);
}
void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & /* dict_struct */,
auto create_table_source = [=](const DictionaryStructure & /* dict_struct */,
const Poco::Util::AbstractConfiguration & /* config */,
const std::string & /* config_prefix */,
Block & /* sample_block */,
@ -272,7 +272,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));
// return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge);
};
factory.registerSource("jdbc", createTableSource);
factory.registerSource("jdbc", create_table_source);
}

View File

@ -115,7 +115,7 @@ void buildLayoutConfiguration(
void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTDictionaryRange * range, const NamesToTypeNames & all_attrs)
{
// appends <key><name>value</name></key> to root
auto appendElem = [&doc, &root](const std::string & key, const std::string & name, const std::string & type)
auto append_element = [&doc, &root](const std::string & key, const std::string & name, const std::string & type)
{
AutoPtr<Element> element(doc->createElement(key));
AutoPtr<Element> name_node(doc->createElement("name"));
@ -131,8 +131,8 @@ void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const
root->appendChild(element);
};
appendElem("range_min", range->min_attr_name, all_attrs.at(range->min_attr_name));
appendElem("range_max", range->max_attr_name, all_attrs.at(range->max_attr_name));
append_element("range_min", range->min_attr_name, all_attrs.at(range->min_attr_name));
append_element("range_max", range->max_attr_name, all_attrs.at(range->max_attr_name));
}

View File

@ -61,7 +61,7 @@ namespace
struct Metadata
{
/// Metadata file version.
const UInt32 VERSION = 1;
static constexpr UInt32 VERSION = 1;
using PathAndSize = std::pair<String, size_t>;

View File

@ -176,7 +176,7 @@ Volume::Volume(
<< formatReadableSizeWithBinarySuffix(sizes[i]) << ") for containing part the size of max_data_part_size ("
<< formatReadableSizeWithBinarySuffix(max_data_part_size) << ")");
}
constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
static constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE)
LOG_WARNING(
logger,

View File

@ -50,14 +50,10 @@ template <typename T>
class DiskTest : public testing::Test
{
public:
void SetUp() override { disk_ = createDisk<T>(); }
void SetUp() override { disk = createDisk<T>(); }
void TearDown() override { destroyDisk<T>(disk); }
void TearDown() override { destroyDisk<T>(disk_); }
const DB::DiskPtr & getDisk() { return disk_; }
private:
DB::DiskPtr disk_;
DB::DiskPtr disk;
};
@ -67,49 +63,43 @@ TYPED_TEST_SUITE(DiskTest, DiskImplementations);
TYPED_TEST(DiskTest, createDirectories)
{
const auto & disk = this->getDisk();
this->disk->createDirectories("test_dir1/");
EXPECT_TRUE(this->disk->isDirectory("test_dir1/"));
disk->createDirectories("test_dir1/");
EXPECT_TRUE(disk->isDirectory("test_dir1/"));
disk->createDirectories("test_dir2/nested_dir/");
EXPECT_TRUE(disk->isDirectory("test_dir2/nested_dir/"));
this->disk->createDirectories("test_dir2/nested_dir/");
EXPECT_TRUE(this->disk->isDirectory("test_dir2/nested_dir/"));
}
TYPED_TEST(DiskTest, writeFile)
{
const auto & disk = this->getDisk();
{
std::unique_ptr<DB::WriteBuffer> out = disk->writeFile("test_file");
std::unique_ptr<DB::WriteBuffer> out = this->disk->writeFile("test_file");
writeString("test data", *out);
}
DB::String data;
{
std::unique_ptr<DB::ReadBuffer> in = disk->readFile("test_file");
std::unique_ptr<DB::ReadBuffer> in = this->disk->readFile("test_file");
readString(data, *in);
}
EXPECT_EQ("test data", data);
EXPECT_EQ(data.size(), disk->getFileSize("test_file"));
EXPECT_EQ(data.size(), this->disk->getFileSize("test_file"));
}
TYPED_TEST(DiskTest, readFile)
{
const auto & disk = this->getDisk();
{
std::unique_ptr<DB::WriteBuffer> out = disk->writeFile("test_file");
std::unique_ptr<DB::WriteBuffer> out = this->disk->writeFile("test_file");
writeString("test data", *out);
}
// Test SEEK_SET
{
String buf(4, '0');
std::unique_ptr<DB::SeekableReadBuffer> in = disk->readFile("test_file");
std::unique_ptr<DB::SeekableReadBuffer> in = this->disk->readFile("test_file");
in->seek(5, SEEK_SET);
@ -119,7 +109,7 @@ TYPED_TEST(DiskTest, readFile)
// Test SEEK_CUR
{
std::unique_ptr<DB::SeekableReadBuffer> in = disk->readFile("test_file");
std::unique_ptr<DB::SeekableReadBuffer> in = this->disk->readFile("test_file");
String buf(4, '0');
in->readStrict(buf.data(), 4);
@ -136,12 +126,10 @@ TYPED_TEST(DiskTest, readFile)
TYPED_TEST(DiskTest, iterateDirectory)
{
const auto & disk = this->getDisk();
disk->createDirectories("test_dir/nested_dir/");
this->disk->createDirectories("test_dir/nested_dir/");
{
auto iter = disk->iterateDirectory("");
auto iter = this->disk->iterateDirectory("");
EXPECT_TRUE(iter->isValid());
EXPECT_EQ("test_dir/", iter->path());
iter->next();
@ -149,7 +137,7 @@ TYPED_TEST(DiskTest, iterateDirectory)
}
{
auto iter = disk->iterateDirectory("test_dir/");
auto iter = this->disk->iterateDirectory("test_dir/");
EXPECT_TRUE(iter->isValid());
EXPECT_EQ("test_dir/nested_dir/", iter->path());
iter->next();

View File

@ -28,7 +28,7 @@ struct CRCImpl
{
using ReturnType = T;
static T make_crc(const unsigned char *buf, size_t size)
static T makeCRC(const unsigned char *buf, size_t size)
{
static CRCBase<ReturnType> base(polynomial);
@ -56,7 +56,7 @@ struct CRC32ZLIBImpl
using ReturnType = UInt32;
static constexpr auto name = "CRC32";
static UInt32 make_crc(const unsigned char *buf, size_t size)
static UInt32 makeCRC(const unsigned char *buf, size_t size)
{
return crc32_z(0L, buf, size);
}
@ -85,20 +85,20 @@ struct CRCFunctionWrapper
ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
res[i] = do_crc(data, prev_offset, offsets[i] - prev_offset - 1);
res[i] = doCRC(data, prev_offset, offsets[i] - prev_offset - 1);
prev_offset = offsets[i];
}
}
static void vector_fixed_to_constant(const ColumnString::Chars & data, size_t n, ReturnType & res) { res = do_crc(data, 0, n); }
static void vectorFixedToConstant(const ColumnString::Chars & data, size_t n, ReturnType & res) { res = doCRC(data, 0, n); }
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<ReturnType> & res)
static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray<ReturnType> & res)
{
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
{
res[i] = do_crc(data, i * n, n);
res[i] = doCRC(data, i * n, n);
}
}
@ -108,10 +108,10 @@ struct CRCFunctionWrapper
}
private:
static ReturnType do_crc(const ColumnString::Chars & buf, size_t offset, size_t size)
static ReturnType doCRC(const ColumnString::Chars & buf, size_t offset, size_t size)
{
const unsigned char * p = reinterpret_cast<const unsigned char *>(&buf[0]) + offset;
return Impl::make_crc(p, size);
return Impl::makeCRC(p, size);
}
};

View File

@ -32,12 +32,12 @@ struct EmptyImpl
}
/// Only make sense if is_fixed_to_constant.
static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/)
static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/)
{
throw Exception("Logical error: 'vector_fixed_to_constant method' is called", ErrorCodes::LOGICAL_ERROR);
throw Exception("Logical error: 'vectorFixedToConstant method' is called", ErrorCodes::LOGICAL_ERROR);
}
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
{
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)

View File

@ -62,25 +62,25 @@ struct BinaryOperationImplBase
using ResultType = ResultType_;
static const constexpr bool allow_fixed_string = false;
static void NO_INLINE vector_vector(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size)
static void NO_INLINE vectorVector(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size)
{
for (size_t i = 0; i < size; ++i)
c[i] = Op::template apply<ResultType>(a[i], b[i]);
}
static void NO_INLINE vector_constant(const A * __restrict a, B b, ResultType * __restrict c, size_t size)
static void NO_INLINE vectorConstant(const A * __restrict a, B b, ResultType * __restrict c, size_t size)
{
for (size_t i = 0; i < size; ++i)
c[i] = Op::template apply<ResultType>(a[i], b);
}
static void NO_INLINE constant_vector(A a, const B * __restrict b, ResultType * __restrict c, size_t size)
static void NO_INLINE constantVector(A a, const B * __restrict b, ResultType * __restrict c, size_t size)
{
for (size_t i = 0; i < size; ++i)
c[i] = Op::template apply<ResultType>(a, b[i]);
}
static ResultType constant_constant(A a, B b)
static ResultType constantConstant(A a, B b)
{
return Op::template apply<ResultType>(a, b);
}
@ -89,7 +89,7 @@ struct BinaryOperationImplBase
template <typename Op>
struct FixedStringOperationImpl
{
static void NO_INLINE vector_vector(const UInt8 * __restrict a, const UInt8 * __restrict b, UInt8 * __restrict c, size_t size)
static void NO_INLINE vectorVector(const UInt8 * __restrict a, const UInt8 * __restrict b, UInt8 * __restrict c, size_t size)
{
for (size_t i = 0; i < size; ++i)
c[i] = Op::template apply<UInt8>(a[i], b[i]);
@ -146,12 +146,12 @@ struct FixedStringOperationImpl
}
}
static void vector_constant(const UInt8 * __restrict a, const UInt8 * __restrict b, UInt8 * __restrict c, size_t size, size_t N)
static void vectorConstant(const UInt8 * __restrict a, const UInt8 * __restrict b, UInt8 * __restrict c, size_t size, size_t N)
{
vector_constant_impl<false>(a, b, c, size, N);
}
static void constant_vector(const UInt8 * __restrict a, const UInt8 * __restrict b, UInt8 * __restrict c, size_t size, size_t N)
static void constantVector(const UInt8 * __restrict a, const UInt8 * __restrict b, UInt8 * __restrict c, size_t size, size_t N)
{
vector_constant_impl<true>(b, a, c, size, N);
}
@ -206,39 +206,39 @@ struct DecimalBinaryOperation
using ArrayC = typename ColumnDecimal<ResultType>::Container;
using SelfNoOverflow = DecimalBinaryOperation<A, B, Operation, ResultType_, false>;
static void vector_vector(const ArrayA & a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow)
static void vectorVector(const ArrayA & a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow)
{
if (check_overflow)
vector_vector(a, b, c, scale_a, scale_b);
vectorVector(a, b, c, scale_a, scale_b);
else
SelfNoOverflow::vector_vector(a, b, c, scale_a, scale_b);
SelfNoOverflow::vectorVector(a, b, c, scale_a, scale_b);
}
static void vector_constant(const ArrayA & a, B b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow)
static void vectorConstant(const ArrayA & a, B b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow)
{
if (check_overflow)
vector_constant(a, b, c, scale_a, scale_b);
vectorConstant(a, b, c, scale_a, scale_b);
else
SelfNoOverflow::vector_constant(a, b, c, scale_a, scale_b);
SelfNoOverflow::vectorConstant(a, b, c, scale_a, scale_b);
}
static void constant_vector(A a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow)
static void constantVector(A a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow)
{
if (check_overflow)
constant_vector(a, b, c, scale_a, scale_b);
constantVector(a, b, c, scale_a, scale_b);
else
SelfNoOverflow::constant_vector(a, b, c, scale_a, scale_b);
SelfNoOverflow::constantVector(a, b, c, scale_a, scale_b);
}
static ResultType constant_constant(A a, B b, ResultType scale_a, ResultType scale_b, bool check_overflow)
static ResultType constantConstant(A a, B b, ResultType scale_a, ResultType scale_b, bool check_overflow)
{
if (check_overflow)
return constant_constant(a, b, scale_a, scale_b);
return constantConstant(a, b, scale_a, scale_b);
else
return SelfNoOverflow::constant_constant(a, b, scale_a, scale_b);
return SelfNoOverflow::constantConstant(a, b, scale_a, scale_b);
}
static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, ArrayC & c,
static void NO_INLINE vectorVector(const ArrayA & a, const ArrayB & b, ArrayC & c,
ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]])
{
size_t size = a.size();
@ -269,7 +269,7 @@ struct DecimalBinaryOperation
c[i] = apply(a[i], b[i]);
}
static void NO_INLINE vector_constant(const ArrayA & a, B b, ArrayC & c,
static void NO_INLINE vectorConstant(const ArrayA & a, B b, ArrayC & c,
ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]])
{
size_t size = a.size();
@ -300,7 +300,7 @@ struct DecimalBinaryOperation
c[i] = apply(a[i], b);
}
static void NO_INLINE constant_vector(A a, const ArrayB & b, ArrayC & c,
static void NO_INLINE constantVector(A a, const ArrayB & b, ArrayC & c,
ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]])
{
size_t size = b.size();
@ -331,7 +331,7 @@ struct DecimalBinaryOperation
c[i] = apply(a, b[i]);
}
static ResultType constant_constant(A a, B b, ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]])
static ResultType constantConstant(A a, B b, ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]])
{
if constexpr (is_plus_minus_compare)
{
@ -863,7 +863,7 @@ public:
auto col_res = ColumnFixedString::create(col_left->getN());
auto & out_chars = col_res->getChars();
out_chars.resize(col_left->getN());
OpImpl::vector_vector(col_left->getChars().data(),
OpImpl::vectorVector(col_left->getChars().data(),
col_right->getChars().data(),
out_chars.data(),
out_chars.size());
@ -893,7 +893,7 @@ public:
if (!is_left_column_const && !is_right_column_const)
{
OpImpl::vector_vector(
OpImpl::vectorVector(
col_left->getChars().data(),
col_right->getChars().data(),
out_chars.data(),
@ -901,7 +901,7 @@ public:
}
else if (is_left_column_const)
{
OpImpl::constant_vector(
OpImpl::constantVector(
col_left->getChars().data(),
col_right->getChars().data(),
out_chars.data(),
@ -910,7 +910,7 @@ public:
}
else
{
OpImpl::vector_constant(
OpImpl::vectorConstant(
col_left->getChars().data(),
col_right->getChars().data(),
out_chars.data(),
@ -965,7 +965,7 @@ public:
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>(),
auto res = OpImpl::constantConstant(col_left->template getValue<T0>(), col_right->template getValue<T1>(),
scale_a, scale_b, check_decimal_overflow);
block.getByPosition(result).column =
ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
@ -974,7 +974,7 @@ public:
}
else
{
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>());
auto res = OpImpl::constantConstant(col_left->template getValue<T0>(), col_right->template getValue<T1>());
block.getByPosition(result).column = ResultDataType().createColumnConst(col_left->size(), toField(res));
}
return true;
@ -1006,11 +1006,11 @@ public:
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res,
OpImpl::constantVector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res,
scale_a, scale_b, check_decimal_overflow);
}
else
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData().data(), vec_res.data(), vec_res.size());
OpImpl::constantVector(col_left_const->template getValue<T0>(), col_right->getData().data(), vec_res.data(), vec_res.size());
}
else
return false;
@ -1027,12 +1027,12 @@ public:
scale_a = right.getScaleMultiplier();
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
{
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b,
OpImpl::vectorVector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b,
check_decimal_overflow);
}
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
{
OpImpl::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res,
OpImpl::vectorConstant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res,
scale_a, scale_b, check_decimal_overflow);
}
else
@ -1041,9 +1041,9 @@ public:
else
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
OpImpl::vector_vector(col_left->getData().data(), col_right->getData().data(), vec_res.data(), vec_res.size());
OpImpl::vectorVector(col_left->getData().data(), col_right->getData().data(), vec_res.data(), vec_res.size());
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
OpImpl::vector_constant(col_left->getData().data(), col_right_const->template getValue<T1>(), vec_res.data(), vec_res.size());
OpImpl::vectorConstant(col_left->getData().data(), col_right_const->template getValue<T1>(), vec_res.data(), vec_res.size());
else
return false;
}

View File

@ -249,7 +249,7 @@ struct Adder
{}
template <typename FromVectorType, typename ToVectorType>
void NO_INLINE vector_constant(const FromVectorType & vec_from, ToVectorType & vec_to, Int64 delta, const DateLUTImpl & time_zone) const
void NO_INLINE vectorConstant(const FromVectorType & vec_from, ToVectorType & vec_to, Int64 delta, const DateLUTImpl & time_zone) const
{
size_t size = vec_from.size();
vec_to.resize(size);
@ -259,7 +259,7 @@ struct Adder
}
template <typename FromVectorType, typename ToVectorType>
void vector_vector(const FromVectorType & vec_from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
void vectorVector(const FromVectorType & vec_from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
{
size_t size = vec_from.size();
vec_to.resize(size);
@ -268,11 +268,11 @@ struct Adder
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>(
&delta, [&](const auto & column){ vector_vector(vec_from, vec_to, column, time_zone, size); return true; });
&delta, [&](const auto & column){ vectorVector(vec_from, vec_to, column, time_zone, size); return true; });
}
template <typename FromType, typename ToVectorType>
void constant_vector(const FromType & from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
void constantVector(const FromType & from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
{
size_t size = delta.size();
vec_to.resize(size);
@ -281,19 +281,19 @@ struct Adder
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>(
&delta, [&](const auto & column){ constant_vector(from, vec_to, column, time_zone, size); return true; });
&delta, [&](const auto & column){ constantVector(from, vec_to, column, time_zone, size); return true; });
}
private:
template <typename FromVectorType, typename ToVectorType, typename DeltaColumnType>
void NO_INLINE vector_vector(const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const
void NO_INLINE vectorVector(const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const
{
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], delta.getData()[i], time_zone);
}
template <typename FromType, typename ToVectorType, typename DeltaColumnType>
void NO_INLINE constant_vector(const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const
void NO_INLINE constantVector(const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, size_t size) const
{
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(from, delta.getData()[i], time_zone);
@ -324,13 +324,13 @@ struct DateTimeAddIntervalImpl
const IColumn & delta_column = *block.getByPosition(arguments[1]).column;
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_column))
op.vector_constant(sources->getData(), col_to->getData(), delta_const_column->getField().get<Int64>(), time_zone);
op.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getField().get<Int64>(), time_zone);
else
op.vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone);
op.vectorVector(sources->getData(), col_to->getData(), delta_column, time_zone);
}
else if (const auto * sources_const = checkAndGetColumnConst<FromColumnType>(source_col.get()))
{
op.constant_vector(sources_const->template getValue<FromValueType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
op.constantVector(sources_const->template getValue<FromValueType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
}
else
{

View File

@ -137,13 +137,13 @@ void validateArgumentsImpl(const IFunction & func,
const auto & arg = arguments[i + argument_offset];
const auto descriptor = descriptors[i];
if (int errorCode = descriptor.isValid(arg.type, arg.column); errorCode != 0)
if (int error_code = descriptor.isValid(arg.type, arg.column); error_code != 0)
throw Exception("Illegal type of argument #" + std::to_string(i)
+ (descriptor.argument_name ? " '" + std::string(descriptor.argument_name) + "'" : String{})
+ " of function " + func.getName()
+ (descriptor.expected_type_description ? String(", expected ") + descriptor.expected_type_description : String{})
+ (arg.type ? ", got " + arg.type->getName() : String{}),
errorCode);
error_code);
}
}
@ -167,7 +167,7 @@ void validateFunctionArgumentTypes(const IFunction & func,
{
if (arguments.size() < mandatory_args.size() || arguments.size() > mandatory_args.size() + optional_args.size())
{
auto joinArgumentTypes = [](const auto & args, const String sep = ", ")
auto join_argument_types = [](const auto & args, const String sep = ", ")
{
String result;
for (const auto & a : args)
@ -194,11 +194,11 @@ void validateFunctionArgumentTypes(const IFunction & func,
throw Exception("Incorrect number of arguments for function " + func.getName()
+ " provided " + std::to_string(arguments.size())
+ (!arguments.empty() ? " (" + joinArgumentTypes(arguments) + ")" : String{})
+ (!arguments.empty() ? " (" + join_argument_types(arguments) + ")" : String{})
+ ", expected " + std::to_string(mandatory_args.size())
+ (!optional_args.empty() ? " to " + std::to_string(mandatory_args.size() + optional_args.size()) : "")
+ " (" + joinArgumentTypes(mandatory_args)
+ (!optional_args.empty() ? ", [" + joinArgumentTypes(optional_args) + "]" : "")
+ " (" + join_argument_types(mandatory_args)
+ (!optional_args.empty() ? ", [" + join_argument_types(optional_args) + "]" : "")
+ ")",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}

View File

@ -67,7 +67,7 @@ public:
if (Impl::is_fixed_to_constant)
{
ResultType res = 0;
Impl::vector_fixed_to_constant(col_fixed->getChars(), col_fixed->getN(), res);
Impl::vectorFixedToConstant(col_fixed->getChars(), col_fixed->getN(), res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_fixed->size(), toField(res));
}
@ -77,7 +77,7 @@ public:
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
vec_res.resize(col_fixed->size());
Impl::vector_fixed_to_vector(col_fixed->getChars(), col_fixed->getN(), vec_res);
Impl::vectorFixedToVector(col_fixed->getChars(), col_fixed->getN(), vec_res);
block.getByPosition(result).column = std::move(col_res);
}

View File

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

View File

@ -78,7 +78,7 @@ template <typename A, typename B, typename Op>
struct NumComparisonImpl
{
/// If you don't specify NO_INLINE, the compiler will inline this function, but we don't need this as this function contains tight loop inside.
static void NO_INLINE vector_vector(const PaddedPODArray<A> & a, const PaddedPODArray<B> & b, PaddedPODArray<UInt8> & c)
static void NO_INLINE vectorVector(const PaddedPODArray<A> & a, const PaddedPODArray<B> & b, PaddedPODArray<UInt8> & c)
{
/** GCC 4.8.2 vectorizes a loop only if it is written in this form.
* In this case, if you loop through the array index (the code will look simpler),
@ -100,7 +100,7 @@ struct NumComparisonImpl
}
}
static void NO_INLINE vector_constant(const PaddedPODArray<A> & a, B b, PaddedPODArray<UInt8> & c)
static void NO_INLINE vectorConstant(const PaddedPODArray<A> & a, B b, PaddedPODArray<UInt8> & c)
{
size_t size = a.size();
const A * __restrict a_pos = a.data();
@ -115,12 +115,12 @@ struct NumComparisonImpl
}
}
static void constant_vector(A a, const PaddedPODArray<B> & b, PaddedPODArray<UInt8> & c)
static void constantVector(A a, const PaddedPODArray<B> & b, PaddedPODArray<UInt8> & c)
{
NumComparisonImpl<B, A, typename Op::SymmetricOp>::vector_constant(b, a, c);
NumComparisonImpl<B, A, typename Op::SymmetricOp>::vectorConstant(b, a, c);
}
static void constant_constant(A a, B b, UInt8 & c)
static void constantConstant(A a, B b, UInt8 & c)
{
c = Op::apply(a, b);
}
@ -168,7 +168,7 @@ struct StringComparisonImpl
}
}
static void NO_INLINE string_vector_constant(
static void NO_INLINE string_vectorConstant(
const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets,
const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c)
@ -243,7 +243,7 @@ struct StringComparisonImpl
}
}
static void NO_INLINE fixed_string_vector_constant(
static void NO_INLINE fixed_string_vectorConstant(
const ColumnString::Chars & a_data, ColumnString::Offset a_n,
const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c)
@ -271,7 +271,7 @@ struct StringComparisonImpl
const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets,
PaddedPODArray<UInt8> & c)
{
StringComparisonImpl<typename Op::SymmetricOp>::string_vector_constant(b_data, b_offsets, a_data, a_size, c);
StringComparisonImpl<typename Op::SymmetricOp>::string_vectorConstant(b_data, b_offsets, a_data, a_size, c);
}
static void constant_fixed_string_vector(
@ -279,10 +279,10 @@ struct StringComparisonImpl
const ColumnString::Chars & b_data, ColumnString::Offset b_n,
PaddedPODArray<UInt8> & c)
{
StringComparisonImpl<typename Op::SymmetricOp>::fixed_string_vector_constant(b_data, b_n, a_data, a_size, c);
StringComparisonImpl<typename Op::SymmetricOp>::fixed_string_vectorConstant(b_data, b_n, a_data, a_size, c);
}
static void constant_constant(
static void constantConstant(
const ColumnString::Chars & a_data, ColumnString::Offset a_size,
const ColumnString::Chars & b_data, ColumnString::Offset b_size,
UInt8 & c)
@ -339,7 +339,7 @@ struct StringEqualsImpl
}
}
static void NO_INLINE string_vector_constant(
static void NO_INLINE string_vectorConstant(
const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets,
const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c)
@ -405,7 +405,7 @@ struct StringEqualsImpl
}
}
static void NO_INLINE fixed_string_vector_constant(
static void NO_INLINE fixed_string_vectorConstant(
const ColumnString::Chars & a_data, ColumnString::Offset a_n,
const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c)
@ -435,7 +435,7 @@ struct StringEqualsImpl
const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets,
PaddedPODArray<UInt8> & c)
{
string_vector_constant(b_data, b_offsets, a_data, a_size, c);
string_vectorConstant(b_data, b_offsets, a_data, a_size, c);
}
static void constant_fixed_string_vector(
@ -443,10 +443,10 @@ struct StringEqualsImpl
const ColumnString::Chars & b_data, ColumnString::Offset b_n,
PaddedPODArray<UInt8> & c)
{
fixed_string_vector_constant(b_data, b_n, a_data, a_size, c);
fixed_string_vectorConstant(b_data, b_n, a_data, a_size, c);
}
static void constant_constant(
static void constantConstant(
const ColumnString::Chars & a_data, ColumnString::Offset a_size,
const ColumnString::Chars & b_data, ColumnString::Offset b_size,
UInt8 & c)
@ -467,25 +467,25 @@ struct StringComparisonImpl<NotEqualsOp<A, B>> : StringEqualsImpl<false> {};
template <typename Op>
struct GenericComparisonImpl
{
static void NO_INLINE vector_vector(const IColumn & a, const IColumn & b, PaddedPODArray<UInt8> & c)
static void NO_INLINE vectorVector(const IColumn & a, const IColumn & b, PaddedPODArray<UInt8> & c)
{
for (size_t i = 0, size = a.size(); i < size; ++i)
c[i] = Op::apply(a.compareAt(i, i, b, 1), 0);
}
static void NO_INLINE vector_constant(const IColumn & a, const IColumn & b, PaddedPODArray<UInt8> & c)
static void NO_INLINE vectorConstant(const IColumn & a, const IColumn & b, PaddedPODArray<UInt8> & c)
{
auto b_materialized = b.cloneResized(1)->convertToFullColumnIfConst();
for (size_t i = 0, size = a.size(); i < size; ++i)
c[i] = Op::apply(a.compareAt(i, 0, *b_materialized, 1), 0);
}
static void constant_vector(const IColumn & a, const IColumn & b, PaddedPODArray<UInt8> & c)
static void constantVector(const IColumn & a, const IColumn & b, PaddedPODArray<UInt8> & c)
{
GenericComparisonImpl<typename Op::SymmetricOp>::vector_constant(b, a, c);
GenericComparisonImpl<typename Op::SymmetricOp>::vectorConstant(b, a, c);
}
static void constant_constant(const IColumn & a, const IColumn & b, UInt8 & c)
static void constantConstant(const IColumn & a, const IColumn & b, UInt8 & c)
{
c = Op::apply(a.compareAt(0, 0, b, 1), 0);
}
@ -582,7 +582,7 @@ private:
ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col_left->getData().size());
NumComparisonImpl<T0, T1, Op<T0, T1>>::vector_vector(col_left->getData(), col_right->getData(), vec_res);
NumComparisonImpl<T0, T1, Op<T0, T1>>::vectorVector(col_left->getData(), col_right->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
@ -593,7 +593,7 @@ private:
ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col_left->size());
NumComparisonImpl<T0, T1, Op<T0, T1>>::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
NumComparisonImpl<T0, T1, Op<T0, T1>>::vectorConstant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
@ -611,7 +611,7 @@ private:
ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col_left->size());
NumComparisonImpl<T0, T1, Op<T0, T1>>::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
NumComparisonImpl<T0, T1, Op<T0, T1>>::constantVector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
@ -619,7 +619,7 @@ private:
else if (auto col_right_const = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
{
UInt8 res = 0;
NumComparisonImpl<T0, T1, Op<T0, T1>>::constant_constant(col_left->template getValue<T0>(), col_right_const->template getValue<T1>(), res);
NumComparisonImpl<T0, T1, Op<T0, T1>>::constantConstant(col_left->template getValue<T0>(), col_right_const->template getValue<T1>(), res);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
return true;
@ -759,7 +759,7 @@ private:
if (c0_const && c1_const)
{
UInt8 res = 0;
StringImpl::constant_constant(*c0_const_chars, c0_const_size, *c1_const_chars, c1_const_size, res);
StringImpl::constantConstant(*c0_const_chars, c0_const_size, *c1_const_chars, c1_const_size, res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(c0_const->size(), toField(res));
return true;
}
@ -780,7 +780,7 @@ private:
c1_fixed_string->getChars(), c1_fixed_string->getN(),
c_res->getData());
else if (c0_string && c1_const)
StringImpl::string_vector_constant(
StringImpl::string_vectorConstant(
c0_string->getChars(), c0_string->getOffsets(),
*c1_const_chars, c1_const_size,
c_res->getData());
@ -795,7 +795,7 @@ private:
c1_fixed_string->getChars(), c1_fixed_string->getN(),
c_res->getData());
else if (c0_fixed_string && c1_const)
StringImpl::fixed_string_vector_constant(
StringImpl::fixed_string_vectorConstant(
c0_fixed_string->getChars(), c0_fixed_string->getN(),
*c1_const_chars, c1_const_size,
c_res->getData());
@ -1081,7 +1081,7 @@ private:
if (c0_const && c1_const)
{
UInt8 res = 0;
GenericComparisonImpl<Op<int, int>>::constant_constant(*c0, *c1, res);
GenericComparisonImpl<Op<int, int>>::constantConstant(*c0, *c1, res);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(c0->size(), toField(res));
}
else
@ -1091,11 +1091,11 @@ private:
vec_res.resize(c0->size());
if (c0_const)
GenericComparisonImpl<Op<int, int>>::constant_vector(*c0, *c1, vec_res);
GenericComparisonImpl<Op<int, int>>::constantVector(*c0, *c1, vec_res);
else if (c1_const)
GenericComparisonImpl<Op<int, int>>::vector_constant(*c0, *c1, vec_res);
GenericComparisonImpl<Op<int, int>>::vectorConstant(*c0, *c1, vec_res);
else
GenericComparisonImpl<Op<int, int>>::vector_vector(*c0, *c1, vec_res);
GenericComparisonImpl<Op<int, int>>::vectorVector(*c0, *c1, vec_res);
block.getByPosition(result).column = std::move(c_res);
}

View File

@ -301,13 +301,13 @@ struct OperationApplier
return;
}
const OperationApplierImpl<Op, N> operationApplierImpl(in);
const OperationApplierImpl<Op, N> operation_applier_impl(in);
size_t i = 0;
for (auto & res : result_data)
if constexpr (CarryResult)
res = Op::apply(res, operationApplierImpl.apply(i++));
res = Op::apply(res, operation_applier_impl.apply(i++));
else
res = operationApplierImpl.apply(i++);
res = operation_applier_impl.apply(i++);
in.erase(in.end() - N, in.end());
}

View File

@ -96,7 +96,7 @@ struct MatchImpl
using ResultType = UInt8;
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
{
if (offsets.empty())
@ -246,14 +246,14 @@ struct MatchImpl
}
template <typename... Args>
static void vector_vector(Args &&...)
static void vectorVector(Args &&...)
{
throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
/// Search different needles in single haystack.
template <typename... Args>
static void constant_vector(Args &&...)
static void constantVector(Args &&...)
{
throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
@ -269,22 +269,22 @@ struct MultiMatchAnyImpl
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto ReturnType()
static auto getReturnType()
{
return std::make_shared<DataTypeNumber<ResultType>>();
}
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res,
PaddedPODArray<UInt64> & offsets)
{
vector_constant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
vectorConstant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
}
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
@ -351,7 +351,7 @@ struct MultiMatchAnyImpl
memset(accum.data(), 0, accum.size());
for (size_t j = 0; j < needles.size(); ++j)
{
MatchImpl<false, false>::vector_constant(haystack_data, haystack_offsets, needles[j].toString(), accum);
MatchImpl<false, false>::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), accum);
for (size_t i = 0; i < res.size(); ++i)
{
if constexpr (FindAny)
@ -372,22 +372,22 @@ struct MultiMatchAllIndicesImpl
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = true;
static auto ReturnType()
static auto getReturnType()
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
PaddedPODArray<Type> & res,
PaddedPODArray<UInt64> & offsets)
{
vector_constant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
vectorConstant(haystack_data, haystack_offsets, needles, res, offsets, std::nullopt);
}
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
@ -652,7 +652,7 @@ struct ReplaceRegexpImpl
}
}
static void vector_fixed(
static void vectorFixed(
const ColumnString::Chars & data,
size_t n,
const std::string & needle,
@ -763,7 +763,7 @@ struct ReplaceStringImpl
/// Note: this function converts fixed-length strings to variable-length strings
/// and each variable-length string should ends with zero byte.
static void vector_fixed(
static void vectorFixed(
const ColumnString::Chars & data,
size_t n,
const std::string & needle,
@ -910,7 +910,7 @@ public:
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_src.get()))
{
auto col_res = ColumnString::create();
Impl::vector_fixed(col_fixed->getChars(), col_fixed->getN(), needle, replacement, col_res->getChars(), col_res->getOffsets());
Impl::vectorFixed(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

@ -63,7 +63,7 @@ public:
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return Impl::ReturnType();
return Impl::getReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
@ -121,7 +121,7 @@ public:
/// The blame for resizing output is for the callee.
if (col_haystack_vector)
Impl::vector_constant(
Impl::vectorConstant(
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res, edit_distance);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -155,7 +155,7 @@ struct PositionImpl
using ResultType = UInt64;
/// Find one substring in many strings.
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
{
const UInt8 * begin = data.data();
@ -192,7 +192,7 @@ struct PositionImpl
}
/// Search for substring in string.
static void constant_constant(std::string data, std::string needle, UInt64 & res)
static void constantConstant(std::string data, std::string needle, UInt64 & res)
{
Impl::toLowerIfNeed(data);
Impl::toLowerIfNeed(needle);
@ -205,7 +205,7 @@ struct PositionImpl
}
/// Search each time for a different single substring inside each time different string.
static void vector_vector(
static void vectorVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
@ -255,7 +255,7 @@ struct PositionImpl
}
/// Find many substrings in single string.
static void constant_vector(
static void constantVector(
const String & haystack,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
@ -303,7 +303,7 @@ struct MultiSearchAllPositionsImpl
{
using ResultType = UInt64;
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
@ -344,9 +344,9 @@ struct MultiSearchImpl
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto ReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
@ -381,9 +381,9 @@ struct MultiSearchFirstPositionImpl
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto ReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
@ -428,9 +428,9 @@ struct MultiSearchFirstIndexImpl
/// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false;
static auto ReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles,
@ -467,7 +467,7 @@ struct HasTokenImpl
static constexpr bool use_default_implementation_for_constants = true;
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
{
if (offsets.empty())
@ -508,14 +508,14 @@ struct HasTokenImpl
}
template <typename... Args>
static void vector_vector(Args &&...)
static void vectorVector(Args &&...)
{
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
/// Search different needles in single haystack.
template <typename... Args>
static void constant_vector(Args &&...)
static void constantVector(Args &&...)
{
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -119,7 +119,7 @@ public:
if (col_haystack_const && col_needle_const)
{
ResultType res{};
Impl::constant_constant(col_haystack_const->getValue<String>(), col_needle_const->getValue<String>(), res);
Impl::constantConstant(col_haystack_const->getValue<String>(), col_needle_const->getValue<String>(), res);
block.getByPosition(result).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
return;
@ -135,17 +135,17 @@ public:
const ColumnString * col_needle_vector = checkAndGetColumn<ColumnString>(&*column_needle);
if (col_haystack_vector && col_needle_vector)
Impl::vector_vector(
Impl::vectorVector(
col_haystack_vector->getChars(),
col_haystack_vector->getOffsets(),
col_needle_vector->getChars(),
col_needle_vector->getOffsets(),
vec_res);
else if (col_haystack_vector && col_needle_const)
Impl::vector_constant(
Impl::vectorConstant(
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue<String>(), vec_res);
else if (col_haystack_const && col_needle_vector)
Impl::constant_vector(
Impl::constantVector(
col_haystack_const->getValue<String>(), col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
else
throw Exception(
@ -277,7 +277,7 @@ public:
vec_res.resize(column_haystack_size * refs.size());
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
@ -325,7 +325,7 @@ public:
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return Impl::ReturnType();
return Impl::getReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
@ -366,7 +366,7 @@ public:
/// The blame for resizing output is for the callee.
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -54,12 +54,12 @@ struct NgramDistanceImpl
*/
using NgramStats = UInt16[map_size];
static ALWAYS_INLINE UInt16 ASCIIHash(const CodePoint * code_points)
static ALWAYS_INLINE UInt16 calculateASCIIHash(const CodePoint * code_points)
{
return intHashCRC32(unalignedLoad<UInt32>(code_points)) & 0xFFFFu;
}
static ALWAYS_INLINE UInt16 UTF8Hash(const CodePoint * code_points)
static ALWAYS_INLINE UInt16 calculateUTF8Hash(const CodePoint * code_points)
{
UInt64 combined = (static_cast<UInt64>(code_points[0]) << 32) | code_points[1];
#ifdef __SSE4_2__
@ -249,12 +249,12 @@ struct NgramDistanceImpl
static inline auto dispatchSearcher(Callback callback, Args &&... args)
{
if constexpr (!UTF8)
return callback(std::forward<Args>(args)..., readASCIICodePoints, ASCIIHash);
return callback(std::forward<Args>(args)..., readASCIICodePoints, calculateASCIIHash);
else
return callback(std::forward<Args>(args)..., readUTF8CodePoints, UTF8Hash);
return callback(std::forward<Args>(args)..., readUTF8CodePoints, calculateUTF8Hash);
}
static void constant_constant(std::string data, std::string needle, Float32 & res)
static void constantConstant(std::string data, std::string needle, Float32 & res)
{
NgramStats common_stats = {};
@ -284,7 +284,7 @@ struct NgramDistanceImpl
}
}
static void vector_vector(
static void vectorVector(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data,
@ -358,7 +358,7 @@ struct NgramDistanceImpl
}
}
static void constant_vector(
static void constantVector(
std::string haystack,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
@ -367,7 +367,7 @@ struct NgramDistanceImpl
/// For symmetric version it is better to use vector_constant
if constexpr (symmetric)
{
vector_constant(needle_data, needle_offsets, std::move(haystack), res);
vectorConstant(needle_data, needle_offsets, std::move(haystack), res);
}
else
{
@ -423,7 +423,7 @@ struct NgramDistanceImpl
}
}
static void vector_constant(
static void vectorConstant(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
std::string needle,

View File

@ -72,7 +72,7 @@ public:
+ std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
Impl::constant_constant(col_haystack_const->getValue<String>(), needle, res);
Impl::constantConstant(col_haystack_const->getValue<String>(), needle, res);
block.getByPosition(result).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
return;
@ -96,11 +96,11 @@ public:
+ std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res);
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res);
}
else if (col_haystack_vector && col_needle_vector)
{
Impl::vector_vector(
Impl::vectorVector(
col_haystack_vector->getChars(),
col_haystack_vector->getOffsets(),
col_needle_vector->getChars(),
@ -117,7 +117,7 @@ public:
+ std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
Impl::constant_vector(haystack, col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
Impl::constantVector(haystack, col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
}
else
{

View File

@ -81,7 +81,7 @@ struct ExtractParamImpl
static constexpr bool use_default_implementation_for_constants = true;
/// It is assumed that `res` is the correct size and initialized with zeros.
static void vector_constant(const ColumnString::Chars & data, const ColumnString::Offsets & offsets,
static void vectorConstant(const ColumnString::Chars & data, const ColumnString::Offsets & offsets,
std::string needle,
PaddedPODArray<ResultType> & res)
{
@ -121,12 +121,12 @@ struct ExtractParamImpl
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
}
template <typename... Args> static void vector_vector(Args &&...)
template <typename... Args> static void vectorVector(Args &&...)
{
throw Exception("Functions 'visitParamHas' and 'visitParamExtract*' doesn't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename... Args> static void constant_vector(Args &&...)
template <typename... Args> static void constantVector(Args &&...)
{
throw Exception("Functions 'visitParamHas' and 'visitParamExtract*' doesn't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -17,7 +17,7 @@ struct LowerUpperImpl
array(data.data(), data.data() + data.size(), res_data.data());
}
static void vector_fixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data)
static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data)
{
res_data.resize(data.size());
array(data.data(), data.data() + data.size(), res_data.data());

View File

@ -96,7 +96,7 @@ struct LowerUpperUTF8Impl
array(data.data(), data.data() + data.size(), res_data.data());
}
static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Functions lowerUTF8 and upperUTF8 cannot work with FixedString argument", ErrorCodes::BAD_ARGUMENTS);
}

View File

@ -98,7 +98,7 @@ struct ExtractSubstringImpl
res_data.assign(start, length);
}
static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN);
}
@ -153,7 +153,7 @@ struct CutSubstringImpl
res_data.append(start + length, data.data() + data.size());
}
static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -93,7 +93,7 @@ struct DecodeURLComponentImpl
res_data.resize(res_offset);
}
[[noreturn]] static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
[[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -64,7 +64,7 @@ struct ExtractFirstSignificantSubdomain
end_of_level_domain = end;
}
if (tldLookup::is_valid(last_3_periods[1] + 1, end_of_level_domain - last_3_periods[1] - 1) != nullptr)
if (tldLookup::isValid(last_3_periods[1] + 1, end_of_level_domain - last_3_periods[1] - 1) != nullptr)
{
res_data += last_3_periods[2] + 1 - begin;
res_size = last_3_periods[1] - last_3_periods[2] - 1;

View File

@ -33,16 +33,16 @@
#define MAX_HASH_VALUE 95690
/* maximum key range = 95616, duplicates = 0 */
class tldLookupHash
class TopLevelDomainLookupHash
{
private:
static inline unsigned int hash(const char * str, size_t len);
public:
static const char * is_valid(const char * str, size_t len);
static const char * isValid(const char * str, size_t len);
};
inline unsigned int tldLookupHash::hash(const char * str, size_t len)
inline unsigned int TopLevelDomainLookupHash::hash(const char * str, size_t len)
{
static const unsigned int asso_values[] = {
95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691, 95691,
@ -105,7 +105,7 @@ inline unsigned int tldLookupHash::hash(const char * str, size_t len)
return hval + asso_values[static_cast<unsigned char>(str[len - 1])];
}
const char * tldLookupHash::is_valid(const char * str, size_t len)
const char * TopLevelDomainLookupHash::isValid(const char * str, size_t len)
{
static const char * const wordlist[]
= {"",

View File

@ -1,15 +1,15 @@
#pragma once
// Definition of the class generated by gperf, present on gperf/tldLookup.gperf
class tldLookupHash
class TopLevelDomainLookupHash
{
private:
static inline unsigned int hash (const char *str, size_t len);
static inline unsigned int hash(const char *str, size_t len);
public:
static const char *is_valid (const char *str, size_t len);
static const char * isValid(const char *str, size_t len);
};
namespace DB
{
using tldLookup = tldLookupHash;
using tldLookup = TopLevelDomainLookupHash;
}

View File

@ -349,7 +349,7 @@ struct ArrayIndexStringImpl
}
}
static void vector_vector(
static void vectorVector(
const ColumnString::Chars & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets,
const ColumnString::Chars & item_values, const ColumnString::Offsets & item_offsets,
PaddedPODArray<typename IndexConv::ResultType> & result,
@ -731,7 +731,7 @@ private:
}
else if (const auto item_arg_vector = checkAndGetColumn<ColumnString>(item_arg))
{
ArrayIndexStringImpl<IndexConv>::vector_vector(col_nested->getChars(), col_array->getOffsets(),
ArrayIndexStringImpl<IndexConv>::vectorVector(col_nested->getChars(), col_array->getOffsets(),
col_nested->getOffsets(), item_arg_vector->getChars(), item_arg_vector->getOffsets(),
col_res->getData(), null_map_data, null_map_item);
}

View File

@ -20,12 +20,12 @@ struct LengthImpl
res[i] = offsets[i] - 1 - offsets[i - 1];
}
static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t n, UInt64 & res)
static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t n, UInt64 & res)
{
res = n;
}
static void vector_fixed_to_vector(const ColumnString::Chars & /*data*/, size_t /*n*/, PaddedPODArray<UInt64> & /*res*/)
static void vectorFixedToVector(const ColumnString::Chars & /*data*/, size_t /*n*/, PaddedPODArray<UInt64> & /*res*/)
{
}

View File

@ -119,7 +119,7 @@ private:
auto c_res = ColumnString::create();
std::vector<const ColumnString::Chars *> data(arguments.size());
std::vector<const ColumnString::Offsets *> offsets(arguments.size());
std::vector<size_t> fixed_string_N(arguments.size());
std::vector<size_t> fixed_string_sizes(arguments.size());
std::vector<String> constant_strings(arguments.size());
bool has_column_string = false;
bool has_column_fixed_string = false;
@ -136,7 +136,7 @@ private:
{
has_column_fixed_string = true;
data[i] = &fixed_col->getChars();
fixed_string_N[i] = fixed_col->getN();
fixed_string_sizes[i] = fixed_col->getN();
}
else if (const ColumnConst * const_col = checkAndGetColumnConstStringOrFixedString(column.get()))
{
@ -159,7 +159,7 @@ private:
std::move(pattern),
data,
offsets,
fixed_string_N,
fixed_string_sizes,
constant_strings,
c_res->getChars(),
c_res->getOffsets(),

View File

@ -145,13 +145,13 @@ private:
ColumnInt64::Container & result)
{
if (auto * y_vec_16 = checkAndGetColumn<ColumnUInt16>(&y))
vector_vector<Transform>(x, *y_vec_16, timezone_x, timezone_y, result);
vectorVector<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);
vectorVector<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);
vectorConstant<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);
vectorConstant<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);
}
@ -163,15 +163,15 @@ private:
ColumnInt64::Container & result)
{
if (auto * y_vec_16 = checkAndGetColumn<ColumnUInt16>(&y))
constant_vector<Transform>(x, *y_vec_16, timezone_x, timezone_y, result);
constantVector<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);
constantVector<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);
}
template <typename Transform, typename T1, typename T2>
void vector_vector(
void vectorVector(
const ColumnVector<T1> & x, const ColumnVector<T2> & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
@ -183,7 +183,7 @@ private:
}
template <typename Transform, typename T1, typename T2>
void vector_constant(
void vectorConstant(
const ColumnVector<T1> & x, T2 y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
@ -194,7 +194,7 @@ private:
}
template <typename Transform, typename T1, typename T2>
void constant_vector(
void constantVector(
T1 x, const ColumnVector<T2> & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)

View File

@ -155,7 +155,7 @@ private:
writeNumber2(target, day);
}
static void ISO8601Date(char * target, Time source, const DateLUTImpl & timezone)
static void ISO8601Date(char * target, Time source, const DateLUTImpl & timezone) // NOLINT
{
writeNumber4(target, ToYearImpl::execute(source, timezone));
writeNumber2(target + 5, ToMonthImpl::execute(source, timezone));
@ -183,7 +183,7 @@ private:
*target += (day == 7 ? 0 : day);
}
static void ISO8601Week(char * target, Time source, const DateLUTImpl & timezone)
static void ISO8601Week(char * target, Time source, const DateLUTImpl & timezone) // NOLINT
{
writeNumber2(target, ToISOWeekImpl::execute(source, timezone));
}
@ -214,7 +214,7 @@ private:
writeNumber2(target, ToMinuteImpl::execute(source, timezone));
}
static void AMPM(char * target, Time source, const DateLUTImpl & timezone)
static void AMPM(char * target, Time source, const DateLUTImpl & timezone) // NOLINT
{
auto hour = ToHourImpl::execute(source, timezone);
if (hour >= 12)
@ -232,7 +232,7 @@ private:
writeNumber2(target, ToSecondImpl::execute(source, timezone));
}
static void ISO8601Time(char * target, Time source, const DateLUTImpl & timezone)
static void ISO8601Time(char * target, Time source, const DateLUTImpl & timezone) // NOLINT
{
writeNumber2(target, ToHourImpl::execute(source, timezone));
writeNumber2(target + 3, ToMinuteImpl::execute(source, timezone));
@ -394,7 +394,7 @@ public:
const char * end = pos + pattern.size();
/// Add shift to previous action; or if there were none, add noop action with shift.
auto addShift = [&](size_t amount)
auto add_shift = [&](size_t amount)
{
if (instructions.empty())
instructions.emplace_back(&Action<T>::noop);
@ -402,12 +402,12 @@ public:
};
/// If the argument was DateTime, add instruction for printing. If it was date, just shift (the buffer is pre-filled with default values).
auto addInstructionOrShift = [&](typename Action<T>::Func func [[maybe_unused]], size_t shift)
auto add_instruction_or_shift = [&](typename Action<T>::Func func [[maybe_unused]], size_t shift)
{
if constexpr (std::is_same_v<T, UInt32>)
instructions.emplace_back(func, shift);
else
addShift(shift);
add_shift(shift);
};
while (true)
@ -419,7 +419,7 @@ public:
if (pos < percent_pos)
{
result.append(pos, percent_pos);
addShift(percent_pos - pos);
add_shift(percent_pos - pos);
}
pos = percent_pos + 1;
@ -505,58 +505,58 @@ public:
// Minute (00-59)
case 'M':
addInstructionOrShift(&Action<T>::minute, 2);
add_instruction_or_shift(&Action<T>::minute, 2);
result.append("00");
break;
// AM or PM
case 'p':
addInstructionOrShift(&Action<T>::AMPM, 2);
add_instruction_or_shift(&Action<T>::AMPM, 2);
result.append("AM");
break;
// 24-hour HH:MM time, equivalent to %H:%M 14:55
case 'R':
addInstructionOrShift(&Action<T>::hhmm24, 5);
add_instruction_or_shift(&Action<T>::hhmm24, 5);
result.append("00:00");
break;
// Seconds
case 'S':
addInstructionOrShift(&Action<T>::second, 2);
add_instruction_or_shift(&Action<T>::second, 2);
result.append("00");
break;
// ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S 14:55:02
case 'T':
addInstructionOrShift(&Action<T>::ISO8601Time, 8);
add_instruction_or_shift(&Action<T>::ISO8601Time, 8);
result.append("00:00:00");
break;
// Hour in 24h format (00-23)
case 'H':
addInstructionOrShift(&Action<T>::hour24, 2);
add_instruction_or_shift(&Action<T>::hour24, 2);
result.append("00");
break;
// Hour in 12h format (01-12)
case 'I':
addInstructionOrShift(&Action<T>::hour12, 2);
add_instruction_or_shift(&Action<T>::hour12, 2);
result.append("12");
break;
/// Escaped literal characters.
case '%':
result += '%';
addShift(1);
add_shift(1);
break;
case 't':
result += '\t';
addShift(1);
add_shift(1);
break;
case 'n':
result += '\n';
addShift(1);
add_shift(1);
break;
// Unimplemented
@ -575,7 +575,7 @@ public:
else
{
result.append(pos, end);
addShift(end + 1 - pos); /// including zero terminator
add_shift(end + 1 - pos); /// including zero terminator
break;
}
}

View File

@ -78,7 +78,7 @@ public:
std::vector<const ColumnString::Chars *> data(arguments.size() - 1);
std::vector<const ColumnString::Offsets *> offsets(arguments.size() - 1);
std::vector<size_t> fixed_string_N(arguments.size() - 1);
std::vector<size_t> fixed_string_sizes(arguments.size() - 1);
std::vector<String> constant_strings(arguments.size() - 1);
bool has_column_string = false;
@ -96,7 +96,7 @@ public:
{
has_column_fixed_string = true;
data[i - 1] = &fixed_col->getChars();
fixed_string_N[i - 1] = fixed_col->getN();
fixed_string_sizes[i - 1] = fixed_col->getN();
}
else if (const ColumnConst * const_col = checkAndGetColumnConstStringOrFixedString(column.get()))
{
@ -113,7 +113,7 @@ public:
std::move(pattern),
data,
offsets,
fixed_string_N,
fixed_string_sizes,
constant_strings,
col_res->getChars(),
col_res->getOffsets(),

View File

@ -53,7 +53,7 @@ struct NumIfImpl
using ArrayB = PaddedPODArray<B>;
using ColVecResult = ColumnVector<ResultType>;
static void vector_vector(const ArrayCond & cond, const ArrayA & a, const ArrayB & b, Block & block, size_t result, UInt32)
static void vectorVector(const ArrayCond & cond, const ArrayA & a, const ArrayB & b, Block & block, size_t result, UInt32)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size);
@ -64,7 +64,7 @@ struct NumIfImpl
block.getByPosition(result).column = std::move(col_res);
}
static void vector_constant(const ArrayCond & cond, const ArrayA & a, B b, Block & block, size_t result, UInt32)
static void vectorConstant(const ArrayCond & cond, const ArrayA & a, B b, Block & block, size_t result, UInt32)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size);
@ -75,7 +75,7 @@ struct NumIfImpl
block.getByPosition(result).column = std::move(col_res);
}
static void constant_vector(const ArrayCond & cond, A a, const ArrayB & b, Block & block, size_t result, UInt32)
static void constantVector(const ArrayCond & cond, A a, const ArrayB & b, Block & block, size_t result, UInt32)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size);
@ -86,7 +86,7 @@ struct NumIfImpl
block.getByPosition(result).column = std::move(col_res);
}
static void constant_constant(const ArrayCond & cond, A a, B b, Block & block, size_t result, UInt32)
static void constantConstant(const ArrayCond & cond, A a, B b, Block & block, size_t result, UInt32)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size);
@ -107,7 +107,7 @@ struct NumIfImpl<Decimal<A>, Decimal<B>, Decimal<R>>
using ArrayB = DecimalPaddedPODArray<Decimal<B>>;
using ColVecResult = ColumnDecimal<ResultType>;
static void vector_vector(const ArrayCond & cond, const ArrayA & a, const ArrayB & b, Block & block, size_t result, UInt32 scale)
static void vectorVector(const ArrayCond & cond, const ArrayA & a, const ArrayB & b, Block & block, size_t result, UInt32 scale)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size, scale);
@ -118,7 +118,7 @@ struct NumIfImpl<Decimal<A>, Decimal<B>, Decimal<R>>
block.getByPosition(result).column = std::move(col_res);
}
static void vector_constant(const ArrayCond & cond, const ArrayA & a, B b, Block & block, size_t result, UInt32 scale)
static void vectorConstant(const ArrayCond & cond, const ArrayA & a, B b, Block & block, size_t result, UInt32 scale)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size, scale);
@ -129,7 +129,7 @@ struct NumIfImpl<Decimal<A>, Decimal<B>, Decimal<R>>
block.getByPosition(result).column = std::move(col_res);
}
static void constant_vector(const ArrayCond & cond, A a, const ArrayB & b, Block & block, size_t result, UInt32 scale)
static void constantVector(const ArrayCond & cond, A a, const ArrayB & b, Block & block, size_t result, UInt32 scale)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size, scale);
@ -140,7 +140,7 @@ struct NumIfImpl<Decimal<A>, Decimal<B>, Decimal<R>>
block.getByPosition(result).column = std::move(col_res);
}
static void constant_constant(const ArrayCond & cond, A a, B b, Block & block, size_t result, UInt32 scale)
static void constantConstant(const ArrayCond & cond, A a, B b, Block & block, size_t result, UInt32 scale)
{
size_t size = cond.size();
auto col_res = ColVecResult::create(size, scale);
@ -156,15 +156,15 @@ template <typename A, typename B>
struct NumIfImpl<A, B, NumberTraits::Error>
{
private:
[[noreturn]] static void throw_error()
[[noreturn]] static void throwError()
{
throw Exception("Internal logic error: invalid types of arguments 2 and 3 of if", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
public:
template <typename... Args> static void vector_vector(Args &&...) { throw_error(); }
template <typename... Args> static void vector_constant(Args &&...) { throw_error(); }
template <typename... Args> static void constant_vector(Args &&...) { throw_error(); }
template <typename... Args> static void constant_constant(Args &&...) { throw_error(); }
template <typename... Args> static void vectorVector(Args &&...) { throwError(); }
template <typename... Args> static void vectorConstant(Args &&...) { throwError(); }
template <typename... Args> static void constantVector(Args &&...) { throwError(); }
template <typename... Args> static void constantConstant(Args &&...) { throwError(); }
};
@ -209,13 +209,13 @@ private:
if (auto col_right_vec = checkAndGetColumn<ColVecT1>(col_right_untyped))
{
NumIfImpl<T0, T1, ResultType>::vector_vector(
NumIfImpl<T0, T1, ResultType>::vectorVector(
cond_col->getData(), col_left->getData(), col_right_vec->getData(), block, result, scale);
return true;
}
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_untyped))
{
NumIfImpl<T0, T1, ResultType>::vector_constant(
NumIfImpl<T0, T1, ResultType>::vectorConstant(
cond_col->getData(), col_left->getData(), col_right_const->template getValue<T1>(), block, result, scale);
return true;
}
@ -238,13 +238,13 @@ private:
if (auto col_right_vec = checkAndGetColumn<ColVecT1>(col_right_untyped))
{
NumIfImpl<T0, T1, ResultType>::constant_vector(
NumIfImpl<T0, T1, ResultType>::constantVector(
cond_col->getData(), col_left->template getValue<T0>(), col_right_vec->getData(), block, result, scale);
return true;
}
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_untyped))
{
NumIfImpl<T0, T1, ResultType>::constant_constant(
NumIfImpl<T0, T1, ResultType>::constantConstant(
cond_col->getData(), col_left->template getValue<T0>(), col_right_const->template getValue<T1>(), block, result, scale);
return true;
}

View File

@ -24,7 +24,7 @@ struct DivideIntegralByConstantImpl
using ResultType = typename DivideIntegralImpl<A, B>::ResultType;
static const constexpr bool allow_fixed_string = false;
static NO_INLINE void vector_constant(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size)
static NO_INLINE void vectorConstant(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size)
{
if (unlikely(b == 0))
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);

View File

@ -304,9 +304,9 @@ SOFTWARE.
}
}
static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/) {}
static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/) {}
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
{
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)

View File

@ -34,11 +34,11 @@ struct LengthUTF8Impl
}
}
static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt64 & /*res*/)
static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt64 & /*res*/)
{
}
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt64> & res)
static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt64> & res)
{
size_t size = data.size() / n;

View File

@ -25,7 +25,7 @@ struct ModuloByConstantImpl
using ResultType = typename ModuloImpl<A, B>::ResultType;
static const constexpr bool allow_fixed_string = false;
static NO_INLINE void vector_constant(const A * __restrict src, B b, ResultType * __restrict dst, size_t size)
static NO_INLINE void vectorConstant(const A * __restrict src, B b, ResultType * __restrict dst, size_t size)
{
if (unlikely(b == 0))
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);

View File

@ -121,28 +121,28 @@ public:
throw Exception("Too few arguments", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION);
}
auto getMsgPrefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
auto get_message_prefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
{
auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (array == nullptr && i != 1)
throw Exception(getMsgPrefix(i) + " must be array of tuples.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(get_message_prefix(i) + " must be array of tuples.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto * tuple = checkAndGetDataType<DataTypeTuple>(array ? array->getNestedType().get() : arguments[i].get());
if (tuple == nullptr)
throw Exception(getMsgPrefix(i) + " must contains tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(get_message_prefix(i) + " must contains tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypes & elements = tuple->getElements();
if (elements.size() != 2)
throw Exception(getMsgPrefix(i) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS);
throw Exception(get_message_prefix(i) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS);
for (auto j : ext::range(0, elements.size()))
{
if (!isNativeNumber(elements[j]))
{
throw Exception(getMsgPrefix(i) + " must contains numeric tuple at position " + toString(j + 1),
throw Exception(get_message_prefix(i) + " must contains numeric tuple at position " + toString(j + 1),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
@ -207,7 +207,7 @@ private:
{
Polygon<Type> polygon;
auto getMsgPrefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
auto get_message_prefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
{
@ -216,7 +216,7 @@ private:
auto tuple_col = array_col ? checkAndGetColumn<ColumnTuple>(&array_col->getData()) : nullptr;
if (!tuple_col)
throw Exception(getMsgPrefix(i) + " must be constant array of tuples.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception(get_message_prefix(i) + " must be constant array of tuples.", ErrorCodes::ILLEGAL_COLUMN);
const auto & tuple_columns = tuple_col->getColumns();
const auto & column_x = tuple_columns[0];
@ -230,7 +230,7 @@ private:
auto size = column_x->size();
if (size == 0)
throw Exception(getMsgPrefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception(get_message_prefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN);
for (auto j : ext::range(0, size))
{
@ -244,13 +244,12 @@ private:
container.push_back(container.front());
}
auto callImpl = use_object_pool
auto call_impl = use_object_pool
? FunctionPointInPolygonDetail::callPointInPolygonImplWithPool<Polygon<Type>, PointInPolygonImpl<Type>>
: FunctionPointInPolygonDetail::callPointInPolygonImpl<Polygon<Type>, PointInPolygonImpl<Type>>;
return callImpl(x, y, polygon);
return call_impl(x, y, polygon);
}
};

View File

@ -40,7 +40,7 @@ struct ReverseImpl
}
}
static void vector_fixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data)
static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data)
{
res_data.resize(data.size());
size_t size = data.size() / n;
@ -100,7 +100,7 @@ public:
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
{
auto col_res = ColumnFixedString::create(col_fixed->getN());
ReverseImpl::vector_fixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars());
ReverseImpl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars());
block.getByPosition(result).column = std::move(col_res);
}
else

View File

@ -61,7 +61,7 @@ struct ReverseUTF8Impl
}
}
[[noreturn]] static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
[[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -34,7 +34,7 @@ namespace ErrorCodes
template <typename DurationType>
struct TimeSlotsImpl
{
static void vector_vector(
static void vectorVector(
const PaddedPODArray<UInt32> & starts, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
@ -56,7 +56,7 @@ struct TimeSlotsImpl
}
}
static void vector_constant(
static void vectorConstant(
const PaddedPODArray<UInt32> & starts, DurationType duration, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
@ -78,7 +78,7 @@ struct TimeSlotsImpl
}
}
static void constant_vector(
static void constantVector(
UInt32 start, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
@ -168,17 +168,17 @@ public:
if (starts && durations)
{
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::vectorVector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (starts && const_durations)
{
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getValue<UInt32>(), time_slot_size, res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::vectorConstant(starts->getData(), const_durations->getValue<UInt32>(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (const_starts && durations)
{
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getValue<UInt32>(), durations->getData(), time_slot_size, res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::constantVector(const_starts->getValue<UInt32>(), durations->getData(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else

View File

@ -124,7 +124,7 @@ struct ToValidUTF8Impl
write_buffer.finalize();
}
[[noreturn]] static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
[[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Column of type FixedString is not supported by toValidUTF8 function", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -33,7 +33,7 @@ struct TrimModeBoth
static constexpr bool trim_right = true;
};
template <typename mode>
template <typename Mode>
class FunctionTrimImpl
{
public:
@ -67,7 +67,7 @@ public:
}
}
static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
{
throw Exception("Functions trimLeft, trimRight and trimBoth cannot work with FixedString argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -78,14 +78,14 @@ private:
const char * char_data = reinterpret_cast<const char *>(data);
const char * char_end = char_data + size;
if constexpr (mode::trim_left)
if constexpr (Mode::trim_left)
{ // NOLINT
const char * found = find_first_not_symbols<' '>(char_data, char_end);
size_t num_chars = found - char_data;
char_data += num_chars;
}
if constexpr (mode::trim_right)
if constexpr (Mode::trim_right)
{ // NOLINT
const char * found = find_last_not_symbols_or_null<' '>(char_data, char_end);
if (found)

View File

@ -164,10 +164,10 @@ namespace
/// We store exception messages in session data.
/// Poco HTTPSession also stores exception, but it can be removed at any time.
const auto & sessionData = session->sessionData();
if (!sessionData.empty())
const auto & session_data = session->sessionData();
if (!session_data.empty())
{
auto msg = Poco::AnyCast<std::string>(sessionData);
auto msg = Poco::AnyCast<std::string>(session_data);
if (!msg.empty())
{
LOG_TRACE((&Logger::get("HTTPCommon")), "Failed communicating with " << host << " with error '" << msg << "' will try to reconnect session");

View File

@ -786,14 +786,14 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
static constexpr auto DATE_TIME_BROKEN_DOWN_LENGTH = 19;
static constexpr auto UNIX_TIMESTAMP_MAX_LENGTH = 10;
static constexpr auto date_time_broken_down_length = 19;
static constexpr auto unix_timestamp_max_length = 10;
char s[DATE_TIME_BROKEN_DOWN_LENGTH];
char s[date_time_broken_down_length];
char * s_pos = s;
/// A piece similar to unix timestamp.
while (s_pos < s + UNIX_TIMESTAMP_MAX_LENGTH && !buf.eof() && isNumericASCII(*buf.position()))
while (s_pos < s + unix_timestamp_max_length && !buf.eof() && isNumericASCII(*buf.position()))
{
*s_pos = *buf.position();
++s_pos;
@ -803,7 +803,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
/// 2015-01-01 01:02:03
if (s_pos == s + 4 && !buf.eof() && (*buf.position() < '0' || *buf.position() > '9'))
{
const size_t remaining_size = DATE_TIME_BROKEN_DOWN_LENGTH - (s_pos - s);
const size_t remaining_size = date_time_broken_down_length - (s_pos - s);
size_t size = buf.read(s_pos, remaining_size);
if (remaining_size != size)
{

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