Clang Tidy, part 7 (#9799)

* Attempt to enable identifier naming check

* Attempt to enable identifier naming check, continue

* Progress on identifier names

* Adopt identifier names check

* Fixed error

* Merge with master

* Fixed build

* Fixed build

* Fixed build

* Fixed build

* Fixed build

* Fixed error

* Fixed error

* Fixed error
This commit is contained in:
alexey-milovidov 2020-03-23 05:12:31 +03:00 committed by GitHub
parent 296fade014
commit c7863e1ac6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
167 changed files with 948 additions and 923 deletions

View File

@ -57,6 +57,7 @@ Checks: '-*,
readability-simplify-subscript-expr, readability-simplify-subscript-expr,
readability-simplify-boolean-expr, readability-simplify-boolean-expr,
readability-inconsistent-declaration-parameter-name, readability-inconsistent-declaration-parameter-name,
readability-identifier-naming,
bugprone-undelegated-constructor, bugprone-undelegated-constructor,
bugprone-argument-comment, bugprone-argument-comment,
@ -162,3 +163,43 @@ Checks: '-*,
boost-use-to-string, boost-use-to-string,
' '
WarningsAsErrors: '*' 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

@ -8,8 +8,8 @@
template <typename T> template <typename T>
static T shift10Impl(T x, int exponent) static T shift10Impl(T x, int exponent)
{ {
static constexpr ssize_t MIN_EXPONENT = -323; static constexpr ssize_t min_exponent = -323;
static constexpr ssize_t MAX_EXPONENT = 308; static constexpr ssize_t max_exponent = 308;
static const long double powers10[] = 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 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. 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. x *= std::numeric_limits<T>::infinity(); /// Multiplying to keep the sign of infinity.
else else
x *= powers10[exponent - MIN_EXPONENT]; x *= powers10[exponent - min_exponent];
return x; return x;
} }

View File

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

View File

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

View File

@ -17,10 +17,10 @@ struct PoolFactory::Impl
std::mutex mutex; 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) 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. /// 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; 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) 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; PoolFactory(const PoolFactory &) = delete;
/** Allocates a PoolWithFailover to connect to MySQL. */ /** 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 default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
size_t max_tries = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); size_t max_tries = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
/** Allocates a PoolWithFailover to connect to MySQL. */ /** 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, const std::string & config_name,
unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_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(); Poco::Util::Application & app = Poco::Util::Application::instance();
std::lock_guard<std::mutex> locker(mutex); std::lock_guard<std::mutex> locker(mutex);
@ -89,7 +89,7 @@ PoolWithFailover::Entry PoolWithFailover::Get()
try try
{ {
Entry entry = shareable ? pool->Get() : pool->tryGet(); Entry entry = shareable ? pool->get() : pool->tryGet();
if (!entry.isNull()) if (!entry.isNull())
{ {
@ -121,7 +121,7 @@ PoolWithFailover::Entry PoolWithFailover::Get()
if (full_pool) if (full_pool)
{ {
app.logger().error("All connections failed, trying to wait on a full pool " + (*full_pool)->getDescription()); 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; std::stringstream message;

View File

@ -105,6 +105,6 @@ namespace mysqlxx
PoolWithFailover(const PoolWithFailover & other); PoolWithFailover(const PoolWithFailover & other);
/** Allocates a connection to use. */ /** 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 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; std::stringstream info;
info << text; info << text;
@ -166,7 +166,7 @@ void Value::throwException(const char * text) const
} }
if (res && res->getQuery()) 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()); throw CannotParseValue(info.str());
} }

View File

@ -1122,7 +1122,7 @@ private:
/// to avoid losing sync. /// to avoid losing sync.
if (!cancelled) if (!cancelled)
{ {
auto cancelQuery = [&] { auto cancel_query = [&] {
connection->sendCancel(); connection->sendCancel();
cancelled = true; cancelled = true;
if (is_interactive) if (is_interactive)
@ -1134,7 +1134,7 @@ private:
if (interrupt_listener.check()) if (interrupt_listener.check())
{ {
cancelQuery(); cancel_query();
} }
else else
{ {
@ -1145,7 +1145,7 @@ private:
<< " Waited for " << static_cast<size_t>(elapsed) << " seconds," << " Waited for " << static_cast<size_t>(elapsed) << " seconds,"
<< " timeout is " << receive_timeout.totalSeconds() << " seconds." << std::endl; << " 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 &) void ClusterCopierApp::handleHelp(const std::string &, const std::string &)
{ {
Poco::Util::HelpFormatter helpFormatter(options()); Poco::Util::HelpFormatter help_formatter(options());
helpFormatter.setCommand(commandName()); help_formatter.setCommand(commandName());
helpFormatter.setHeader("Copies tables from one cluster to another"); help_formatter.setHeader("Copies tables from one cluster to another");
helpFormatter.setUsage("--config-file <config-file> --task-path <task-path>"); help_formatter.setUsage("--config-file <config-file> --task-path <task-path>");
helpFormatter.format(std::cerr); help_formatter.format(std::cerr);
stopOptionsProcessing(); stopOptionsProcessing();
} }

View File

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

View File

@ -546,7 +546,7 @@ void HTTPHandler::processQuery(
client_info.http_method = http_method; client_info.http_method = http_method;
client_info.http_user_agent = request.get("User-Agent", ""); client_info.http_user_agent = request.get("User-Agent", "");
auto appendCallback = [&context] (ProgressCallback callback) auto append_callback = [&context] (ProgressCallback callback)
{ {
auto prev = context.getProgressCallback(); 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. /// 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) 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) if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close)
{ {
Poco::Net::StreamSocket & socket = dynamic_cast<Poco::Net::HTTPServerRequestImpl &>(request).socket(); 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. /// Assume that at the point this method is called no one is reading data from the socket any more.
/// True for read-only queries. /// True for read-only queries.

View File

@ -23,9 +23,9 @@ Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHand
<< ", Content Type: " << request.getContentType() << ", Content Type: " << request.getContentType()
<< ", Transfer Encoding: " << request.getTransferEncoding()); << ", 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) if (handler != nullptr)
return handler; return handler;
} }

View File

@ -252,8 +252,8 @@ void MySQLHandler::comFieldList(ReadBuffer & payload)
ComFieldList packet; ComFieldList packet;
packet.readPayload(payload); packet.readPayload(payload);
String database = connection_context.getCurrentDatabase(); String database = connection_context.getCurrentDatabase();
StoragePtr tablePtr = DatabaseCatalog::instance().getTable({database, packet.table}); StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table});
for (const NameAndTypePair & column: tablePtr->getColumns().getAll()) for (const NameAndTypePair & column: table_ptr->getColumns().getAll())
{ {
ColumnDefinition column_definition( ColumnDefinition column_definition(
database, packet.table, packet.table, column.name, column.name, CharacterSet::binary, 100, ColumnType::MYSQL_TYPE_STRING, 0, 0 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() void MySQLHandlerFactory::readRSAKeys()
{ {
const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config(); const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config();
String certificateFileProperty = "openSSL.server.certificateFile"; String certificate_file_property = "openSSL.server.certificateFile";
String privateKeyFileProperty = "openSSL.server.privateKeyFile"; 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); 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); throw Exception("Private key file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
{ {
String certificateFile = config.getString(certificateFileProperty); String certificate_file = config.getString(certificate_file_property);
FILE * fp = fopen(certificateFile.data(), "r"); FILE * fp = fopen(certificate_file.data(), "r");
if (fp == nullptr) 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)); SCOPE_EXIT(fclose(fp));
X509 * x509 = PEM_read_X509(fp, nullptr, nullptr, nullptr); X509 * x509 = PEM_read_X509(fp, nullptr, nullptr, nullptr);
SCOPE_EXIT(X509_free(x509)); SCOPE_EXIT(X509_free(x509));
if (x509 == nullptr) 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); EVP_PKEY * p = X509_get_pubkey(x509);
if (p == nullptr) 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) 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)); SCOPE_EXIT(fclose(fp));
private_key.reset(PEM_read_RSAPrivateKey(fp, nullptr, nullptr, nullptr)); private_key.reset(PEM_read_RSAPrivateKey(fp, nullptr, nullptr, nullptr));
if (!private_key) 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")) if (config().hasOption("help"))
{ {
Poco::Util::HelpFormatter helpFormatter(Server::options()); Poco::Util::HelpFormatter help_formatter(Server::options());
std::stringstream header; std::stringstream header;
header << commandName() << " [OPTION] [-- [ARG]...]\n"; header << commandName() << " [OPTION] [-- [ARG]...]\n";
header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010"; header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010";
helpFormatter.setHeader(header.str()); help_formatter.setHeader(header.str());
helpFormatter.format(std::cout); help_formatter.format(std::cout);
return 0; return 0;
} }
if (config().hasOption("version")) if (config().hasOption("version"))

View File

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

View File

@ -20,32 +20,32 @@ namespace ErrorCodes
namespace namespace
{ {
template <typename T, typename Tlimit_num_elems> template <typename T, typename LimitNumberOfElements>
struct MovingSum struct MovingSum
{ {
using DataType = MovingSumData<T>; 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 struct MovingAvg
{ {
using DataType = MovingAvgData<T>; 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 LimitNumberOfElements> using MovingSumTemplate = typename MovingSum<T, LimitNumberOfElements>::Function;
template <typename T, typename Tlimit_num_elems> using MovingAvgTemplate = typename MovingAvg<T, Tlimit_num_elems>::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) inline AggregateFunctionPtr createAggregateFunctionMovingImpl(const std::string & name, const DataTypePtr & argument_type, TArgs ... args)
{ {
AggregateFunctionPtr res; AggregateFunctionPtr res;
if (isDecimal(argument_type)) 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 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) if (!res)
throw Exception("Illegal type " + argument_type->getName() + " of argument for aggregate function " + name, 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 /// Substitute return type for Date and DateTime
template <typename has_limit> template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, has_limit> class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>
{ {
public: public:
explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, UInt64 max_elems_ = std::numeric_limits<UInt64>::max()) 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>()); } DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDate>()); }
}; };
template <typename has_limit> template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, has_limit> class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>
{ {
public: public:
explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, UInt64 max_elems_ = std::numeric_limits<UInt64>::max()) 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>()); } 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) static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args)
{ {
WhichDataType which(argument_type); WhichDataType which(argument_type);
if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate<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<has_limit>(argument_type, std::forward<TArgs>(args)...); else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime<HasLimit>(argument_type, std::forward<TArgs>(args)...);
else else
{ {
/// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric /// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric
if (argument_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) 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 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) 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) 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) if (!res)
throw Exception("Illegal type " + argument_type->getName() + throw Exception("Illegal type " + argument_type->getName() +

View File

@ -130,7 +130,7 @@ LinearModelData::LinearModelData(
gradient_batch.resize(param_num_ + 1, Float64{0.0}); gradient_batch.resize(param_num_ + 1, Float64{0.0});
} }
void LinearModelData::update_state() void LinearModelData::updateState()
{ {
if (batch_size == 0) if (batch_size == 0)
return; return;
@ -197,7 +197,7 @@ void LinearModelData::merge(const DB::LinearModelData & rhs)
if (iter_num == 0 && rhs.iter_num == 0) if (iter_num == 0 && rhs.iter_num == 0)
return; return;
update_state(); updateState();
/// can't update rhs state because it's constant /// 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 /// 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); Float64 target = (*columns[0]).getFloat64(row_num);
/// Here we have columns + 1 as first column corresponds to target value, and others - to features /// 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); gradient_batch, *gradient_computer, weights, bias, l2_reg_coef, target, columns + 1, row_num);
++batch_size; ++batch_size;
if (batch_size == batch_capacity) 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_; beta2_powered_ *= beta2_;
} }
void Adam::add_to_batch( void Adam::addToBatch(
std::vector<Float64> & batch_gradient, std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer, IGradientComputer & gradient_computer,
const std::vector<Float64> & weights, 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()]; bias += accumulated_gradient[weights.size()];
} }
void Nesterov::add_to_batch( void Nesterov::addToBatch(
std::vector<Float64> & batch_gradient, std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer, IGradientComputer & gradient_computer,
const std::vector<Float64> & weights, const std::vector<Float64> & weights,
@ -432,7 +432,7 @@ void StochasticGradientDescent::update(
bias += (learning_rate * batch_gradient[weights.size()]) / batch_size; bias += (learning_rate * batch_gradient[weights.size()]) / batch_size;
} }
void IWeightsUpdater::add_to_batch( void IWeightsUpdater::addToBatch(
std::vector<Float64> & batch_gradient, std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer, IGradientComputer & gradient_computer,
const std::vector<Float64> & weights, const std::vector<Float64> & weights,

View File

@ -112,7 +112,7 @@ public:
virtual ~IWeightsUpdater() = default; virtual ~IWeightsUpdater() = default;
/// Calls GradientComputer to update current mini-batch /// Calls GradientComputer to update current mini-batch
virtual void add_to_batch( virtual void addToBatch(
std::vector<Float64> & batch_gradient, std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer, IGradientComputer & gradient_computer,
const std::vector<Float64> & weights, const std::vector<Float64> & weights,
@ -176,7 +176,7 @@ public:
Nesterov(Float64 alpha) : alpha_(alpha) {} Nesterov(Float64 alpha) : alpha_(alpha) {}
void add_to_batch( void addToBatch(
std::vector<Float64> & batch_gradient, std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer, IGradientComputer & gradient_computer,
const std::vector<Float64> & weights, const std::vector<Float64> & weights,
@ -209,7 +209,7 @@ public:
beta2_powered_ = beta2_; beta2_powered_ = beta2_;
} }
void add_to_batch( void addToBatch(
std::vector<Float64> & batch_gradient, std::vector<Float64> & batch_gradient,
IGradientComputer & gradient_computer, IGradientComputer & gradient_computer,
const std::vector<Float64> & weights, 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 /** 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(); MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn();
res->reserve(data.size()); res->reserve(data.size());
auto ML_function = func.get(); auto machine_learning_function = func.get();
if (ML_function) if (machine_learning_function)
{ {
if (data.size() == 1) if (data.size() == 1)
{ {
/// Case for const column. Predict using single model. /// 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 else
{ {
@ -105,7 +105,7 @@ MutableColumnPtr ColumnAggregateFunction::predictValues(Block & block, const Col
size_t row_num = 0; size_t row_num = 0;
for (auto val : data) 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; ++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. /** 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); 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; const char * end = pos + length;
size_t elems = 0; size_t elems = 0;
for (; pos + field_size <= end; pos += field_size, ++elems) for (; pos + field_size <= end; pos += field_size, ++elems)
data_->insertData(pos, field_size); data->insertData(pos, field_size);
if (pos != end) if (pos != end)
throw Exception("Incorrect length argument for method ColumnArray::insertData", ErrorCodes::BAD_ARGUMENTS); 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 namespace
{ {
template <bool positive> template <bool positive>
struct less struct Less
{ {
const ColumnArray & parent; const ColumnArray & parent;
int nan_direction_hint; 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_) {} : parent(parent_), nan_direction_hint(nan_direction_hint_) {}
bool operator()(size_t lhs, size_t rhs) const bool operator()(size_t lhs, size_t rhs) const
@ -689,16 +688,16 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h
if (limit) if (limit)
{ {
if (reverse) 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 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 else
{ {
if (reverse) 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 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) if (0 == col_size)
return res; 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 typename ColumnVector<T>::Container & src_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
const Offsets & src_offsets = getOffsets(); const Offsets & src_offsets = getOffsets();
typename ColumnVector<T>::Container & res_data = typeid_cast<ColumnVector<T> &>(res_.getData()).getData(); typename ColumnVector<T>::Container & res_data = typeid_cast<ColumnVector<T> &>(res_arr.getData()).getData();
Offsets & res_offsets = res_.getOffsets(); Offsets & res_offsets = res_arr.getOffsets();
res_data.reserve(data->size() / col_size * replicate_offsets.back()); res_data.reserve(data->size() / col_size * replicate_offsets.back());
res_offsets.reserve(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) if (0 == col_size)
return res; 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 & src_string = typeid_cast<const ColumnString &>(*data);
const ColumnString::Chars & src_chars = src_string.getChars(); const ColumnString::Chars & src_chars = src_string.getChars();
const Offsets & src_string_offsets = src_string.getOffsets(); const Offsets & src_string_offsets = src_string.getOffsets();
const Offsets & src_offsets = getOffsets(); const Offsets & src_offsets = getOffsets();
ColumnString::Chars & res_chars = typeid_cast<ColumnString &>(res_.getData()).getChars(); ColumnString::Chars & res_chars = typeid_cast<ColumnString &>(res_arr.getData()).getChars();
Offsets & res_string_offsets = typeid_cast<ColumnString &>(res_.getData()).getOffsets(); Offsets & res_string_offsets = typeid_cast<ColumnString &>(res_arr.getData()).getOffsets();
Offsets & res_offsets = res_.getOffsets(); Offsets & res_offsets = res_arr.getOffsets();
res_chars.reserve(src_chars.size() / col_size * replicate_offsets.back()); res_chars.reserve(src_chars.size() / col_size * replicate_offsets.back());
res_string_offsets.reserve(src_string_offsets.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) for (IColumn::ColumnIndex part = 0; part < num_columns; ++part)
{ {
auto & capture = captures[part]; auto & capture = captures[part];
size_t size__ = capture.empty() ? counts[part] : capture.front().column->size(); size_t capture_size = capture.empty() ? counts[part] : capture.front().column->size();
columns.emplace_back(ColumnFunction::create(size__, function, std::move(capture))); columns.emplace_back(ColumnFunction::create(capture_size, function, std::move(capture)));
} }
return columns; 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) 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 check_for = [&](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 try_get_size_for = [&](auto type) -> size_t { return check_for(type) ? sizeof(decltype(type)) : 0; };
if (hint) if (hint)
{ {
size_t size = 0; size_t size = 0;
callForType([&](auto type) { size = tryGetSizeFor(type); }, hint); callForType([&](auto type) { size = try_get_size_for(type); }, hint);
if (size) if (size)
return size; return size;
} }
if (auto size = tryGetSizeFor(UInt8())) if (auto size = try_get_size_for(UInt8()))
return size; return size;
if (auto size = tryGetSizeFor(UInt16())) if (auto size = try_get_size_for(UInt16()))
return size; return size;
if (auto size = tryGetSizeFor(UInt32())) if (auto size = try_get_size_for(UInt32()))
return size; return size;
if (auto size = tryGetSizeFor(UInt64())) if (auto size = try_get_size_for(UInt64()))
return size; return size;
throw Exception("Unexpected indexes type for ColumnLowCardinality. Expected UInt, got " + column.getName(), 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 ColumnLowCardinality::Index::getPositionAt(size_t row) const
{ {
size_t pos; size_t pos;
auto getPosition = [&](auto type) auto get_position = [&](auto type)
{ {
using CurIndexType = decltype(type); using CurIndexType = decltype(type);
pos = getPositionsData<CurIndexType>()[row]; pos = getPositionsData<CurIndexType>()[row];
}; };
callForType(std::move(getPosition), size_of_type); callForType(std::move(get_position), size_of_type);
return pos; return pos;
} }
@ -536,7 +536,7 @@ void ColumnLowCardinality::Index::insertPosition(UInt64 position)
void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, UInt64 offset, UInt64 limit) 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); using ColumnType = decltype(type);
const auto * column_ptr = typeid_cast<const ColumnVector<ColumnType> *>(&column); const auto * column_ptr = typeid_cast<const ColumnVector<ColumnType> *>(&column);
@ -570,10 +570,10 @@ void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, U
return true; return true;
}; };
if (!insertForType(UInt8()) && if (!insert_for_type(UInt8()) &&
!insertForType(UInt16()) && !insert_for_type(UInt16()) &&
!insertForType(UInt32()) && !insert_for_type(UInt32()) &&
!insertForType(UInt64())) !insert_for_type(UInt64()))
throw Exception("Invalid column for ColumnLowCardinality index. Expected UInt, got " + column.getName(), throw Exception("Invalid column for ColumnLowCardinality index. Expected UInt, got " + column.getName(),
ErrorCodes::ILLEGAL_COLUMN); 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 Dwarf::Path::size() const
{ {
size_t size = 0; size_t size = 0;
bool needsSlash = false; bool needs_slash = false;
if (!baseDir_.empty()) if (!baseDir_.empty())
{ {
size += baseDir_.size(); size += baseDir_.size();
needsSlash = baseDir_.back() != '/'; needs_slash = baseDir_.back() != '/';
} }
if (!subDir_.empty()) if (!subDir_.empty())
{ {
size += needsSlash; size += needs_slash;
size += subDir_.size(); size += subDir_.size();
needsSlash = subDir_.back() != '/'; needs_slash = subDir_.back() != '/';
} }
if (!file_.empty()) if (!file_.empty())
{ {
size += needsSlash; size += needs_slash;
size += file_.size(); 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 Dwarf::Path::toBuffer(char * buf, size_t bufSize) const
{ {
size_t totalSize = 0; size_t total_size = 0;
bool needsSlash = false; bool needs_slash = false;
auto append = [&](std::string_view sp) auto append = [&](std::string_view sp)
{ {
if (bufSize >= 2) if (bufSize >= 2)
{ {
size_t toCopy = std::min(sp.size(), bufSize - 1); size_t to_copy = std::min(sp.size(), bufSize - 1);
memcpy(buf, sp.data(), toCopy); memcpy(buf, sp.data(), to_copy);
buf += toCopy; buf += to_copy;
bufSize -= toCopy; bufSize -= to_copy;
} }
totalSize += sp.size(); total_size += sp.size();
}; };
if (!baseDir_.empty()) if (!baseDir_.empty())
{ {
append(baseDir_); append(baseDir_);
needsSlash = baseDir_.back() != '/'; needs_slash = baseDir_.back() != '/';
} }
if (!subDir_.empty()) if (!subDir_.empty())
{ {
if (needsSlash) if (needs_slash)
{ {
append("/"); append("/");
} }
append(subDir_); append(subDir_);
needsSlash = subDir_.back() != '/'; needs_slash = subDir_.back() != '/';
} }
if (!file_.empty()) if (!file_.empty())
{ {
if (needsSlash) if (needs_slash)
{ {
append("/"); append("/");
} }
@ -291,14 +291,14 @@ size_t Dwarf::Path::toBuffer(char * buf, size_t bufSize) const
*buf = '\0'; *buf = '\0';
} }
SAFE_CHECK(totalSize == size(), "Size mismatch"); SAFE_CHECK(total_size == size(), "Size mismatch");
return totalSize; return total_size;
} }
void Dwarf::Path::toString(std::string & dest) const void Dwarf::Path::toString(std::string & dest) const
{ {
size_t initialSize = dest.size(); size_t initial_size = dest.size();
dest.reserve(initialSize + size()); dest.reserve(initial_size + size());
if (!baseDir_.empty()) if (!baseDir_.empty())
{ {
dest.append(baseDir_.begin(), baseDir_.end()); dest.append(baseDir_.begin(), baseDir_.end());
@ -319,7 +319,7 @@ void Dwarf::Path::toString(std::string & dest) const
} }
dest.append(file_.begin(), file_.end()); 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 // 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 // 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 // a 96-bit value (0xffffffff followed by the 64-bit length) for a 64-bit
// section. // section.
auto initialLength = read<uint32_t>(chunk); auto initial_length = read<uint32_t>(chunk);
is64Bit_ = (initialLength == uint32_t(-1)); is64Bit_ = (initial_length == uint32_t(-1));
auto length = is64Bit_ ? read<uint64_t>(chunk) : initialLength; auto length = is64Bit_ ? read<uint64_t>(chunk) : initial_length;
SAFE_CHECK(length <= chunk.size(), "invalid DWARF section"); SAFE_CHECK(length <= chunk.size(), "invalid DWARF section");
chunk = std::string_view(chunk.data(), length); chunk = std::string_view(chunk.data(), length);
data_ = std::string_view(chunk.end(), data_.end() - chunk.end()); 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); abbr.hasChildren = (read<uint8_t>(section) != DW_CHILDREN_no);
// attributes // attributes
const char * attributeBegin = section.data(); const char * attribute_begin = section.data();
for (;;) for (;;)
{ {
SAFE_CHECK(!section.empty(), "invalid attribute section"); SAFE_CHECK(!section.empty(), "invalid attribute section");
@ -396,7 +396,7 @@ bool Dwarf::readAbbreviation(std::string_view & section, DIEAbbreviation & abbr)
break; break;
} }
abbr.attributes = std::string_view(attributeBegin, section.data() - attributeBegin); abbr.attributes = std::string_view(attribute_begin, section.data() - attribute_begin);
return true; 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) bool Dwarf::findDebugInfoOffset(uintptr_t address, std::string_view aranges, uint64_t & offset)
{ {
Section arangesSection(aranges); Section aranges_section(aranges);
std::string_view chunk; std::string_view chunk;
while (arangesSection.next(chunk)) while (aranges_section.next(chunk))
{ {
auto version = read<uint16_t>(chunk); auto version = read<uint16_t>(chunk);
SAFE_CHECK(version == 2, "invalid aranges version"); SAFE_CHECK(version == 2, "invalid aranges version");
offset = readOffset(chunk, arangesSection.is64Bit()); offset = readOffset(chunk, aranges_section.is64Bit());
auto addressSize = read<uint8_t>(chunk); auto address_size = read<uint8_t>(chunk);
SAFE_CHECK(addressSize == sizeof(uintptr_t), "invalid address size"); SAFE_CHECK(address_size == sizeof(uintptr_t), "invalid address size");
auto segmentSize = read<uint8_t>(chunk); auto segment_size = read<uint8_t>(chunk);
SAFE_CHECK(segmentSize == 0, "segmented architecture not supported"); SAFE_CHECK(segment_size == 0, "segmented architecture not supported");
// Padded to a multiple of 2 addresses. // Padded to a multiple of 2 addresses.
// Strangely enough, this is the only place in the DWARF spec that requires // 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 // 3. debug_abbrev_offset (4B or 8B): offset into the .debug_abbrev section
// 4. address_size (1B) // 4. address_size (1B)
Section debugInfoSection(infoEntry); Section debug_info_section(infoEntry);
std::string_view chunk; 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); auto version = read<uint16_t>(chunk);
SAFE_CHECK(version >= 2 && version <= 4, "invalid info version"); SAFE_CHECK(version >= 2 && version <= 4, "invalid info version");
uint64_t abbrevOffset = readOffset(chunk, debugInfoSection.is64Bit()); uint64_t abbrev_offset = readOffset(chunk, debug_info_section.is64Bit());
auto addressSize = read<uint8_t>(chunk); auto address_size = read<uint8_t>(chunk);
SAFE_CHECK(addressSize == sizeof(uintptr_t), "invalid address size"); 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 // 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. // 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? // TODO(tudorb): Handle DW_TAG_partial_unit?
auto code = readULEB(chunk); auto code = readULEB(chunk);
SAFE_CHECK(code != 0, "invalid code"); 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"); SAFE_CHECK(abbr.tag == DW_TAG_compile_unit, "expecting compile unit entry");
// Skip children entries, remove_prefix to the next compilation unit entry. // Skip children entries, remove_prefix to the next compilation unit entry.
infoEntry.remove_prefix(chunk.end() - infoEntry.begin()); infoEntry.remove_prefix(chunk.end() - infoEntry.begin());
// Read attributes, extracting the few we care about // Read attributes, extracting the few we care about
bool foundLineOffset = false; bool found_line_offset = false;
uint64_t lineOffset = 0; uint64_t line_offset = 0;
std::string_view compilationDirectory; std::string_view compilation_directory;
std::string_view mainFileName; std::string_view main_file_name;
DIEAbbreviation::Attribute attr; DIEAbbreviation::Attribute attr;
std::string_view attributes = abbr.attributes; std::string_view attributes = abbr.attributes;
@ -573,43 +573,43 @@ bool Dwarf::findLocation(uintptr_t address, std::string_view & infoEntry, Locati
{ {
break; break;
} }
auto val = readAttributeValue(chunk, attr.form, debugInfoSection.is64Bit()); auto val = readAttributeValue(chunk, attr.form, debug_info_section.is64Bit());
switch (attr.name) switch (attr.name)
{ {
case DW_AT_stmt_list: case DW_AT_stmt_list:
// Offset in .debug_line for the line number VM program for this // Offset in .debug_line for the line number VM program for this
// compilation unit // compilation unit
lineOffset = std::get<uint64_t>(val); line_offset = std::get<uint64_t>(val);
foundLineOffset = true; found_line_offset = true;
break; break;
case DW_AT_comp_dir: case DW_AT_comp_dir:
// Compilation directory // Compilation directory
compilationDirectory = std::get<std::string_view>(val); compilation_directory = std::get<std::string_view>(val);
break; break;
case DW_AT_name: case DW_AT_name:
// File name of main file being compiled // File name of main file being compiled
mainFileName = std::get<std::string_view>(val); main_file_name = std::get<std::string_view>(val);
break; break;
} }
} }
if (!mainFileName.empty()) if (!main_file_name.empty())
{ {
locationInfo.hasMainFile = true; locationInfo.hasMainFile = true;
locationInfo.mainFile = Path(compilationDirectory, "", mainFileName); locationInfo.mainFile = Path(compilation_directory, "", main_file_name);
} }
if (!foundLineOffset) if (!found_line_offset)
{ {
return false; return false;
} }
std::string_view lineSection(line_); std::string_view line_section(line_);
lineSection.remove_prefix(lineOffset); line_section.remove_prefix(line_offset);
LineNumberVM lineVM(lineSection, compilationDirectory); LineNumberVM line_vm(line_section, compilation_directory);
// Execute line number VM program to find file and line // 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; return locationInfo.hasFileAndLine;
} }
@ -635,9 +635,9 @@ bool Dwarf::findAddress(uintptr_t address, LocationInfo & locationInfo, Location
if (findDebugInfoOffset(address, aranges_, offset)) if (findDebugInfoOffset(address, aranges_, offset))
{ {
// Read compilation unit header from .debug_info // Read compilation unit header from .debug_info
std::string_view infoEntry(info_); std::string_view info_entry(info_);
infoEntry.remove_prefix(offset); info_entry.remove_prefix(offset);
findLocation(address, infoEntry, locationInfo); findLocation(address, info_entry, locationInfo);
return locationInfo.hasFileAndLine; return locationInfo.hasFileAndLine;
} }
else if (mode == LocationInfoMode::FAST) 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 // Slow path (linear scan): Iterate over all .debug_info entries
// and look for the address in each compilation unit. // and look for the address in each compilation unit.
std::string_view infoEntry(info_); std::string_view info_entry(info_);
while (!infoEntry.empty() && !locationInfo.hasFileAndLine) while (!info_entry.empty() && !locationInfo.hasFileAndLine)
findLocation(address, infoEntry, locationInfo); findLocation(address, info_entry, locationInfo);
return locationInfo.hasFileAndLine; return locationInfo.hasFileAndLine;
} }
@ -693,16 +693,16 @@ void Dwarf::LineNumberVM::init()
{ {
version_ = read<uint16_t>(data_); version_ = read<uint16_t>(data_);
SAFE_CHECK(version_ >= 2 && version_ <= 4, "invalid version in line number VM"); SAFE_CHECK(version_ >= 2 && version_ <= 4, "invalid version in line number VM");
uint64_t headerLength = readOffset(data_, is64Bit_); uint64_t header_length = readOffset(data_, is64Bit_);
SAFE_CHECK(headerLength <= data_.size(), "invalid line number VM header length"); SAFE_CHECK(header_length <= data_.size(), "invalid line number VM header length");
std::string_view header(data_.data(), headerLength); std::string_view header(data_.data(), header_length);
data_ = std::string_view(header.end(), data_.end() - header.end()); data_ = std::string_view(header.end(), data_.end() - header.end());
minLength_ = read<uint8_t>(header); minLength_ = read<uint8_t>(header);
if (version_ == 4) if (version_ == 4)
{ // Version 2 and 3 records don't have this { // Version 2 and 3 records don't have this
uint8_t maxOpsPerInstruction = read<uint8_t>(header); uint8_t max_ops_per_instruction = read<uint8_t>(header);
SAFE_CHECK(maxOpsPerInstruction == 1, "VLIW not supported"); SAFE_CHECK(max_ops_per_instruction == 1, "VLIW not supported");
} }
defaultIsStmt_ = read<uint8_t>(header); defaultIsStmt_ = read<uint8_t>(header);
lineBase_ = read<int8_t>(header); // yes, signed lineBase_ = read<int8_t>(header); // yes, signed
@ -752,10 +752,10 @@ Dwarf::LineNumberVM::FileName Dwarf::LineNumberVM::getFileName(uint64_t index) c
FileName fn; FileName fn;
if (index <= fileNameCount_) if (index <= fileNameCount_)
{ {
std::string_view fileNames = fileNames_; std::string_view file_names = fileNames_;
for (; index; --index) for (; index; --index)
{ {
if (!readFileName(fileNames, fn)) if (!readFileName(file_names, fn))
{ {
abort(); abort();
} }
@ -783,11 +783,11 @@ std::string_view Dwarf::LineNumberVM::getIncludeDirectory(uint64_t index) const
SAFE_CHECK(index <= includeDirectoryCount_, "invalid include directory"); SAFE_CHECK(index <= includeDirectoryCount_, "invalid include directory");
std::string_view includeDirectories = includeDirectories_; std::string_view include_directories = includeDirectories_;
std::string_view dir; std::string_view dir;
for (; index; --index) for (; index; --index)
{ {
dir = readNullTerminated(includeDirectories); dir = readNullTerminated(include_directories);
if (dir.empty()) if (dir.empty())
{ {
abort(); // BUG abort(); // BUG
@ -825,8 +825,8 @@ bool Dwarf::LineNumberVM::nextDefineFile(std::string_view & program, FileName &
if (opcode != 0) if (opcode != 0)
{ // standard opcode { // standard opcode
// Skip, slurp the appropriate number of LEB arguments // Skip, slurp the appropriate number of LEB arguments
uint8_t argCount = standardOpcodeLengths_[opcode - 1]; uint8_t arg_count = standardOpcodeLengths_[opcode - 1];
while (argCount--) while (arg_count--)
{ {
readULEB(program); readULEB(program);
} }
@ -858,11 +858,11 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro
if (opcode >= opcodeBase_) if (opcode >= opcodeBase_)
{ // special opcode { // special opcode
uint8_t adjustedOpcode = opcode - opcodeBase_; uint8_t adjusted_opcode = opcode - opcodeBase_;
uint8_t opAdvance = adjustedOpcode / lineRange_; uint8_t op_advance = adjusted_opcode / lineRange_;
address_ += minLength_ * opAdvance; address_ += minLength_ * op_advance;
line_ += lineBase_ + adjustedOpcode % lineRange_; line_ += lineBase_ + adjusted_opcode % lineRange_;
basicBlock_ = false; basicBlock_ = false;
prologueEnd_ = 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 // Unrecognized standard opcode, slurp the appropriate number of LEB
// arguments. // arguments.
uint8_t argCount = standardOpcodeLengths_[opcode - 1]; uint8_t arg_count = standardOpcodeLengths_[opcode - 1];
while (argCount--) while (arg_count--)
{ {
readULEB(program); readULEB(program);
} }
@ -944,10 +944,10 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro
auto length = readULEB(program); auto length = readULEB(program);
// the opcode itself should be included in the length, so length >= 1 // the opcode itself should be included in the length, so length >= 1
SAFE_CHECK(length != 0, "invalid extended opcode length"); SAFE_CHECK(length != 0, "invalid extended opcode length");
auto extendedOpcode = read<uint8_t>(program); auto extended_opcode = read<uint8_t>(program);
--length; --length;
switch (extendedOpcode) switch (extended_opcode)
{ {
case DW_LNE_end_sequence: case DW_LNE_end_sequence:
return END; return END;
@ -987,15 +987,15 @@ bool Dwarf::LineNumberVM::findAddress(uintptr_t target, Path & file, uint64_t &
State state = START; State state = START;
reset(); reset();
uint64_t prevFile = 0; uint64_t prev_file = 0;
uint64_t prevLine = 0; uint64_t prev_line = 0;
while (!program.empty()) while (!program.empty())
{ {
bool seqEnd = !next(program); bool seq_end = !next(program);
if (state == START) if (state == START)
{ {
if (!seqEnd) if (!seq_end)
{ {
state = address_ <= target ? LOW_SEQ : HIGH_SEQ; 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 // Found it! Note that ">" is indeed correct (not ">="), as each
// sequence is guaranteed to have one entry past-the-end (emitted by // sequence is guaranteed to have one entry past-the-end (emitted by
// DW_LNE_end_sequence) // DW_LNE_end_sequence)
if (prevFile == 0) if (prev_file == 0)
{ {
return false; return false;
} }
auto fn = getFileName(prevFile); auto fn = getFileName(prev_file);
file = Path(compilationDirectory_, getIncludeDirectory(fn.directoryIndex), fn.relativeName); file = Path(compilationDirectory_, getIncludeDirectory(fn.directoryIndex), fn.relativeName);
line = prevLine; line = prev_line;
return true; return true;
} }
prevFile = file_; prev_file = file_;
prevLine = line_; prev_line = line_;
} }
if (seqEnd) if (seq_end)
{ {
state = START; state = START;
reset(); reset();

View File

@ -74,7 +74,7 @@ public:
/** The `Entry` object protects the resource from being used by another thread. /** 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 * 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 * q.execute (); // Someone else can use this Connection
*/ */
Object * operator->() && = delete; Object * operator->() && = delete;

View File

@ -64,7 +64,7 @@ public:
private: private:
/// A separate method which binds the lock holder to the owned lock /// A separate method which binds the lock holder to the owned lock
/// N.B. It is very important that this method produces no allocations /// 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) if (bound)
return false; 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 /// 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) all_read_locks.add(query_id); /// SM1: may throw on insertion (nothing to roll back)
++it_query->second; /// SM2: nothrow ++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(); finalize_metrics();
return lock_holder; 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(); finalize_metrics();
return lock_holder; return lock_holder;

View File

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

View File

@ -758,17 +758,17 @@ struct ZooKeeperMultiResponse final : MultiResponse, ZooKeeperResponse
{ {
ZooKeeper::OpNum op_num; ZooKeeper::OpNum op_num;
bool done; bool done;
int32_t error_; int32_t error_read;
Coordination::read(op_num, in); Coordination::read(op_num, in);
Coordination::read(done, in); Coordination::read(done, in);
Coordination::read(error_, in); Coordination::read(error_read, in);
if (!done) if (!done)
throw Exception("Too many results received for multi transaction", ZMARSHALLINGERROR); throw Exception("Too many results received for multi transaction", ZMARSHALLINGERROR);
if (op_num != -1) if (op_num != -1)
throw Exception("Unexpected op_num received at the end of results for multi transaction", ZMARSHALLINGERROR); 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); 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}; std::atomic<int> locked {false};
bool try_lock() bool tryLock()
{ {
int expected = 0; int expected = 0;
return locked.compare_exchange_strong(expected, 1, std::memory_order_acquire); 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; return (items_count * max_item_size_bits + 8) / 8;
} }
struct binary_value_info struct BinaryValueInfo
{ {
UInt8 leading_zero_bits; UInt8 leading_zero_bits;
UInt8 data_bits; UInt8 data_bits;
@ -69,7 +69,7 @@ struct binary_value_info
}; };
template <typename T> template <typename T>
binary_value_info getLeadingAndTrailingBits(const T & value) BinaryValueInfo getLeadingAndTrailingBits(const T & value)
{ {
constexpr UInt8 bit_size = sizeof(T) * 8; 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 tz = getTrailingZeroBits(value);
const UInt8 data_size = value == 0 ? 0 : static_cast<UInt8>(bit_size - lz - tz); 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> template <typename T>
@ -99,7 +99,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
T prev_value{}; T prev_value{};
// That would cause first XORed value to be written in-full. // 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) if (source < source_end)
{ {
@ -118,7 +118,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
source += sizeof(curr_value); source += sizeof(curr_value);
const auto xored_data = curr_value ^ prev_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) 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)); 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, // 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. // 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) for (UInt32 items_read = 1; items_read < items_count && !reader.eof(); ++items_read)
{ {
T curr_value = prev_value; T curr_value = prev_value;
binary_value_info curr_xored_info = prev_xored_info; BinaryValueInfo curr_xored_info = prev_xored_info;
T xored_data{}; T xored_data{};
if (reader.readBit() == 1) if (reader.readBit() == 1)

View File

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

View File

@ -191,7 +191,7 @@ private:
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get()); const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
A a = c0_const->template getValue<A>(); A a = c0_const->template getValue<A>();
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get())) 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 else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR); throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
} }
@ -200,7 +200,7 @@ private:
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get()); const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
B b = c1_const->template getValue<B>(); B b = c1_const->template getValue<B>();
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get())) 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 else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR); 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 ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
{ {
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.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 else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR); throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
} }
@ -260,7 +260,7 @@ private:
} }
template <bool scale_left, bool scale_right> 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) CompareInt scale)
{ {
size_t size = a.size(); size_t size = a.size();
@ -279,7 +279,7 @@ private:
} }
template <bool scale_left, bool scale_right> 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(); size_t size = a.size();
const A * a_pos = a.data(); const A * a_pos = a.data();
@ -295,7 +295,7 @@ private:
} }
template <bool scale_left, bool scale_right> 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(); size_t size = b.size();
const B * b_pos = b.data(); 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) getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name)
{ {
using NearestResultType = NearestFieldType<T>; using NearestResultType = NearestFieldType<T>;
const auto fieldType = Field::TypeToEnum<NearestResultType>::value; const auto field_type = Field::TypeToEnum<NearestResultType>::value;
const ASTLiteral * argument = nullptr; const ASTLiteral * argument = nullptr;
auto exceptionMessage = [=](const String & message) auto exception_message = [=](const String & message)
{ {
return std::string("Parameter #") + std::to_string(argument_index) + " '" return std::string("Parameter #") + std::to_string(argument_index) + " '"
+ argument_name + "' for " + context_data_type_name + argument_name + "' for " + context_data_type_name
+ message + message
+ ", expected: " + Field::Types::toString(fieldType) + " literal."; + ", expected: " + Field::Types::toString(field_type) + " literal.";
}; };
if (!arguments || arguments->children.size() <= argument_index 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) if constexpr (Kind == ArgumentKind::Optional)
return {}; return {};
else else
throw Exception(exceptionMessage(" is missing"), throw Exception(exception_message(" is missing"),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
} }
if (argument->value.getType() != fieldType) if (argument->value.getType() != field_type)
throw Exception(exceptionMessage(String(" has wrong type: ") + argument->value.getTypeName()), throw Exception(exception_message(String(" has wrong type: ") + argument->value.getTypeName()),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return argument->value.get<NearestResultType>(); return argument->value.get<NearestResultType>();

View File

@ -604,7 +604,7 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
auto * low_cardinality_state = checkAndGetLowCardinalityDeserializeState(state); auto * low_cardinality_state = checkAndGetLowCardinalityDeserializeState(state);
KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); 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; UInt64 num_keys;
readIntBinary(num_keys, *keys_stream); readIntBinary(num_keys, *keys_stream);
@ -617,7 +617,7 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
low_cardinality_state->global_dictionary = std::move(column_unique); 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; UInt64 num_keys;
readIntBinary(num_keys, *indexes_stream); 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(); auto indexes_type = low_cardinality_state->index_type.getDataType();
MutableColumnPtr indexes_column = indexes_type->createColumn(); 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; !global_dictionary || index_type.need_update_dictionary || low_cardinality_state->need_update_dictionary;
if (index_type.need_global_dictionary && need_update_dictionary) if (index_type.need_global_dictionary && need_update_dictionary)
{ {
readDictionary(); read_dictionary();
low_cardinality_state->need_update_dictionary = false; low_cardinality_state->need_update_dictionary = false;
} }
if (low_cardinality_state->index_type.has_additional_keys) if (low_cardinality_state->index_type.has_additional_keys)
readAdditionalKeys(); read_additional_keys();
else else
low_cardinality_state->additional_keys = nullptr; 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); 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; limit -= num_rows_to_read;
low_cardinality_state->num_pending_rows -= 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) DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_nothing, bool force_support_conversion)
{ {
auto get_nothing_or_throw = [throw_if_result_is_nothing, & types](const std::string & reason)
auto getNothingOrThrow = [throw_if_result_is_nothing, & types](const std::string & reason)
{ {
if (throw_if_result_is_nothing) if (throw_if_result_is_nothing)
throw Exception(getExceptionMessagePrefix(types) + reason, ErrorCodes::NO_COMMON_TYPE); 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) for (const auto & type : types)
if (typeid_cast<const DataTypeNothing *>(type.get())) 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 /// For Arrays
@ -117,7 +116,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (have_array) if (have_array)
{ {
if (!all_arrays) 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)); 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()); nested_types[elem_idx].reserve(types.size());
} }
else if (tuple_size != type_tuple->getElements().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; have_tuple = true;
@ -157,7 +156,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (have_tuple) if (have_tuple)
{ {
if (!all_tuples) 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); DataTypes common_tuple_types(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx) 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) if (!fixed_string_type)
fixed_string_type = type; fixed_string_type = type;
else if (!type->equals(*fixed_string_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)) else if (isString(type))
have_string = true; have_string = true;
@ -229,7 +228,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
if (have_string) if (have_string)
{ {
if (!all_strings) 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>(); 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 (have_date_or_datetime)
{ {
if (!all_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>(); 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 (min_bits_of_signed_integer || min_bits_of_unsigned_integer || min_mantissa_bits_of_floating)
{ {
if (!all_numbers) 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 the result must be floating.
if (!min_bits_of_signed_integer && !min_bits_of_unsigned_integer) 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). /// 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; " WHERE TABLE_SCHEMA = " << quote << database_name_in_mysql;
std::map<String, UInt64> tables_with_modification_time; 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()) 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"; << " 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; 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()) while (Block block = result.read())
{ {
size_t rows = block.rows(); size_t rows = block.rows();

View File

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

View File

@ -209,7 +209,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re
void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, 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); 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 DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const
{ {
auto configurationFromAST = getDictionaryConfigurationFromAST(ast); auto configuration = getDictionaryConfigurationFromAST(ast);
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context, true); return DictionaryFactory::create(name, *configuration, "dictionary", context, true);
} }
bool DictionaryFactory::isComplex(const std::string & layout_type) const bool DictionaryFactory::isComplex(const std::string & layout_type) const

View File

@ -216,7 +216,7 @@ std::string ExecutableDictionarySource::toString() const
void registerDictionarySourceExecutable(DictionarySourceFactory & factory) void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
@ -236,7 +236,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
dict_struct, config, config_prefix + ".executable", dict_struct, config, config_prefix + ".executable",
sample_block, context); 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) void registerDictionarySourceFile(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
@ -86,7 +86,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context, check_config); 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) void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
@ -206,7 +206,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
dict_struct, config, config_prefix + ".http", dict_struct, config, config_prefix + ".http",
sample_block, context, check_config); sample_block, context, check_config);
}; };
factory.registerSource("http", createTableSource); factory.registerSource("http", create_table_source);
} }
} }

View File

@ -151,9 +151,9 @@ LibraryDictionarySource::LibraryDictionarySource(
#endif #endif
); );
settings = std::make_shared<CStringsHolder>(getLibSettings(config, config_prefix + lib_config_settings)); 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")) "ClickHouseDictionary_v3_libNew"))
lib_data = libNew(&settings->strings, ClickHouseLibrary::log); lib_data = lib_new(&settings->strings, ClickHouseLibrary::log);
} }
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other) LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
@ -166,18 +166,18 @@ LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource &
, description{other.description} , description{other.description}
, settings{other.settings} , settings{other.settings}
{ {
if (auto libClone = library->tryGet<decltype(lib_data) (*)(decltype(other.lib_data))>("ClickHouseDictionary_v3_libClone")) if (auto lib_clone = library->tryGet<decltype(lib_data) (*)(decltype(other.lib_data))>("ClickHouseDictionary_v3_libClone"))
lib_data = libClone(other.lib_data); lib_data = lib_clone(other.lib_data);
else if ( 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")) "ClickHouseDictionary_v3_libNew"))
lib_data = libNew(&settings->strings, ClickHouseLibrary::log); lib_data = lib_new(&settings->strings, ClickHouseLibrary::log);
} }
LibraryDictionarySource::~LibraryDictionarySource() LibraryDictionarySource::~LibraryDictionarySource()
{ {
if (auto libDelete = library->tryGet<void (*)(decltype(lib_data))>("ClickHouseDictionary_v3_libDelete")) if (auto lib_delete = library->tryGet<void (*)(decltype(lib_data))>("ClickHouseDictionary_v3_libDelete"))
libDelete(lib_data); lib_delete(lib_data);
} }
BlockInputStreamPtr LibraryDictionarySource::loadAll() BlockInputStreamPtr LibraryDictionarySource::loadAll()
@ -196,10 +196,10 @@ BlockInputStreamPtr LibraryDictionarySource::loadAll()
void * data_ptr = nullptr; void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw. /// 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"); = 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); 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); 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)); SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v3_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block); return std::make_shared<OneBlockInputStream>(block);
@ -222,11 +222,11 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> &
void * data_ptr = nullptr; void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw. /// 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))>( = library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&columns_pass), decltype(&ids_data))>(
"ClickHouseDictionary_v3_loadIds"); "ClickHouseDictionary_v3_loadIds");
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v3_dataNew")(lib_data); 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); 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)); SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v3_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block); return std::make_shared<OneBlockInputStream>(block);
@ -256,10 +256,10 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column
void * data_ptr = nullptr; void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw. /// 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"); "ClickHouseDictionary_v3_loadKeys");
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v3_dataNew")(lib_data); 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); 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)); SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v3_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block); return std::make_shared<OneBlockInputStream>(block);
@ -267,17 +267,17 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column
bool LibraryDictionarySource::isModified() const 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")) = 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; return true;
} }
bool LibraryDictionarySource::supportsSelectiveLoad() const 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")) = 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; return true;
} }
@ -293,7 +293,7 @@ std::string LibraryDictionarySource::toString() const
void registerDictionarySourceLibrary(DictionarySourceFactory & factory) void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, 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); 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) void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
@ -29,7 +29,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
ErrorCodes::SUPPORT_IS_DISABLED}; ErrorCodes::SUPPORT_IS_DISABLED};
#endif #endif
}; };
factory.registerSource("mongodb", createTableSource); factory.registerSource("mongodb", create_table_source);
} }
} }
@ -319,16 +319,16 @@ BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_column
break; break;
case AttributeUnderlyingType::utString: 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 /// Convert string to ObjectID
if (attr.second.is_object_id) if (attr.second.is_object_id)
{ {
Poco::MongoDB::ObjectId::Ptr _id(new Poco::MongoDB::ObjectId(_str)); Poco::MongoDB::ObjectId::Ptr loaded_id(new Poco::MongoDB::ObjectId(loaded_str));
key.add(attr.second.name, _id); key.add(attr.second.name, loaded_id);
} }
else else
{ {
key.add(attr.second.name, _str); key.add(attr.second.name, loaded_str);
} }
break; break;
} }

View File

@ -14,7 +14,7 @@ namespace ErrorCodes
void registerDictionarySourceMysql(DictionarySourceFactory & factory) void registerDictionarySourceMysql(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
@ -31,7 +31,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
ErrorCodes::SUPPORT_IS_DISABLED}; ErrorCodes::SUPPORT_IS_DISABLED};
#endif #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", "")} , update_field{config.getString(config_prefix + ".update_field", "")}
, dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)} , dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)}
, sample_block{sample_block_} , 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} , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}
, load_all_query{query_builder.composeLoadAllQuery()} , load_all_query{query_builder.composeLoadAllQuery()}
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
@ -115,7 +115,7 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
BlockInputStreamPtr MySQLDictionarySource::loadAll() BlockInputStreamPtr MySQLDictionarySource::loadAll()
{ {
auto connection = pool.Get(); auto connection = pool.get();
last_modification = getLastModification(connection, false); last_modification = getLastModification(connection, false);
LOG_TRACE(log, load_all_query); LOG_TRACE(log, load_all_query);
@ -124,7 +124,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadAll()
BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
{ {
auto connection = pool.Get(); auto connection = pool.get();
last_modification = getLastModification(connection, false); last_modification = getLastModification(connection, false);
std::string load_update_query = getUpdateFieldAndDate(); 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. /// 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); 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) 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. /// 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); 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 bool MySQLDictionarySource::isModified() const
@ -161,7 +161,7 @@ bool MySQLDictionarySource::isModified() const
if (dont_check_update_time) if (dont_check_update_time)
return true; return true;
auto connection = pool.Get(); auto connection = pool.get();
return getLastModification(connection, true) > last_modification; return getLastModification(connection, true) > last_modification;
} }
@ -221,7 +221,7 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry &
if (auto row = result.fetch()) if (auto row = result.fetch())
{ {
++fetched_rows; ++fetched_rows;
const auto UPDATE_TIME_IDX = 12; static const auto UPDATE_TIME_IDX = 12;
const auto & update_time_value = row[UPDATE_TIME_IDX]; const auto & update_time_value = row[UPDATE_TIME_IDX];
if (!update_time_value.isNull()) if (!update_time_value.isNull())
@ -259,7 +259,7 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request
Block invalidate_sample_block; Block invalidate_sample_block;
ColumnPtr column(ColumnString::create()); ColumnPtr column(ColumnString::create());
invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block")); 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); return readInvalidateQuery(block_input_stream);
} }

View File

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

View File

@ -12,7 +12,7 @@ namespace DB
void registerDictionarySourceRedis(DictionarySourceFactory & factory) void registerDictionarySourceRedis(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const String & config_prefix, const String & config_prefix,
Block & sample_block, Block & sample_block,
@ -29,7 +29,7 @@ namespace DB
ErrorCodes::SUPPORT_IS_DISABLED}; ErrorCodes::SUPPORT_IS_DISABLED};
#endif #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; 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()) while (!stack.empty())
{ {
@ -703,13 +703,13 @@ static void trieTraverse(const btrie_t * trie, Getter && getter)
if (node && node->right) if (node && node->right)
{ {
stack.push(nullptr); stack.push(nullptr);
key |= getBit(stack.size()); key |= get_bit(stack.size());
stack.push(node->right); stack.push(node->right);
while (stack.top()->left) while (stack.top()->left)
stack.push(stack.top()->left); stack.push(stack.top()->left);
} }
else 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>; 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(); const auto & attr = dict_attributes.front();
return ColumnsWithTypeAndName( return ColumnsWithTypeAndName(
{ColumnWithTypeAndName(columns.front(), std::make_shared<DataTypeFixedString>(IPV6_BINARY_LENGTH), attr.name)}); {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(); auto column = ColumnString::create();
const auto & ip_column = assert_cast<const ColumnFixedString &>(*columns.front()); 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)}; ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), dict_attributes.front().name)};
}; };
return std::make_shared<BlockInputStreamType>( 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(); Poco::Data::ODBC::Connector::registerConnector();
#endif #endif
auto createTableSource = [=](const DictionaryStructure & dict_struct, auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
@ -256,12 +256,12 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
ErrorCodes::SUPPORT_IS_DISABLED}; ErrorCodes::SUPPORT_IS_DISABLED};
#endif #endif
}; };
factory.registerSource("odbc", createTableSource); factory.registerSource("odbc", create_table_source);
} }
void registerDictionarySourceJDBC(DictionarySourceFactory & factory) void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
{ {
auto createTableSource = [=](const DictionaryStructure & /* dict_struct */, auto create_table_source = [=](const DictionaryStructure & /* dict_struct */,
const Poco::Util::AbstractConfiguration & /* config */, const Poco::Util::AbstractConfiguration & /* config */,
const std::string & /* config_prefix */, const std::string & /* config_prefix */,
Block & /* sample_block */, 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")); // 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); // 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) void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTDictionaryRange * range, const NamesToTypeNames & all_attrs)
{ {
// appends <key><name>value</name></key> to root // 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> element(doc->createElement(key));
AutoPtr<Element> name_node(doc->createElement("name")); AutoPtr<Element> name_node(doc->createElement("name"));
@ -131,8 +131,8 @@ void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const
root->appendChild(element); root->appendChild(element);
}; };
appendElem("range_min", range->min_attr_name, all_attrs.at(range->min_attr_name)); append_element("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_max", range->max_attr_name, all_attrs.at(range->max_attr_name));
} }

View File

@ -61,7 +61,7 @@ namespace
struct Metadata struct Metadata
{ {
/// Metadata file version. /// Metadata file version.
const UInt32 VERSION = 1; static constexpr UInt32 VERSION = 1;
using PathAndSize = std::pair<String, size_t>; 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(sizes[i]) << ") for containing part the size of max_data_part_size ("
<< formatReadableSizeWithBinarySuffix(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) if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE)
LOG_WARNING( LOG_WARNING(
logger, logger,

View File

@ -50,14 +50,10 @@ template <typename T>
class DiskTest : public testing::Test class DiskTest : public testing::Test
{ {
public: 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_); } DB::DiskPtr disk;
const DB::DiskPtr & getDisk() { return disk_; }
private:
DB::DiskPtr disk_;
}; };
@ -67,49 +63,43 @@ TYPED_TEST_SUITE(DiskTest, DiskImplementations);
TYPED_TEST(DiskTest, createDirectories) 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/"); this->disk->createDirectories("test_dir2/nested_dir/");
EXPECT_TRUE(disk->isDirectory("test_dir1/")); EXPECT_TRUE(this->disk->isDirectory("test_dir2/nested_dir/"));
disk->createDirectories("test_dir2/nested_dir/");
EXPECT_TRUE(disk->isDirectory("test_dir2/nested_dir/"));
} }
TYPED_TEST(DiskTest, writeFile) 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); writeString("test data", *out);
} }
DB::String data; 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); readString(data, *in);
} }
EXPECT_EQ("test data", data); 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) 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); writeString("test data", *out);
} }
// Test SEEK_SET // Test SEEK_SET
{ {
String buf(4, '0'); 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); in->seek(5, SEEK_SET);
@ -119,7 +109,7 @@ TYPED_TEST(DiskTest, readFile)
// Test SEEK_CUR // 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'); String buf(4, '0');
in->readStrict(buf.data(), 4); in->readStrict(buf.data(), 4);
@ -136,12 +126,10 @@ TYPED_TEST(DiskTest, readFile)
TYPED_TEST(DiskTest, iterateDirectory) TYPED_TEST(DiskTest, iterateDirectory)
{ {
const auto & disk = this->getDisk(); this->disk->createDirectories("test_dir/nested_dir/");
disk->createDirectories("test_dir/nested_dir/");
{ {
auto iter = disk->iterateDirectory(""); auto iter = this->disk->iterateDirectory("");
EXPECT_TRUE(iter->isValid()); EXPECT_TRUE(iter->isValid());
EXPECT_EQ("test_dir/", iter->path()); EXPECT_EQ("test_dir/", iter->path());
iter->next(); 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_TRUE(iter->isValid());
EXPECT_EQ("test_dir/nested_dir/", iter->path()); EXPECT_EQ("test_dir/nested_dir/", iter->path());
iter->next(); iter->next();

View File

@ -28,7 +28,7 @@ struct CRCImpl
{ {
using ReturnType = T; 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); static CRCBase<ReturnType> base(polynomial);
@ -56,7 +56,7 @@ struct CRC32ZLIBImpl
using ReturnType = UInt32; using ReturnType = UInt32;
static constexpr auto name = "CRC32"; 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); return crc32_z(0L, buf, size);
} }
@ -85,20 +85,20 @@ struct CRCFunctionWrapper
ColumnString::Offset prev_offset = 0; ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i) 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]; 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; size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i) 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: 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; 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. /// 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; size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)

View File

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

View File

@ -249,7 +249,7 @@ struct Adder
{} {}
template <typename FromVectorType, typename ToVectorType> 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(); size_t size = vec_from.size();
vec_to.resize(size); vec_to.resize(size);
@ -259,7 +259,7 @@ struct Adder
} }
template <typename FromVectorType, typename ToVectorType> 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(); size_t size = vec_from.size();
vec_to.resize(size); vec_to.resize(size);
@ -268,11 +268,11 @@ struct Adder
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64, ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64, ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>( 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> 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(); size_t size = delta.size();
vec_to.resize(size); vec_to.resize(size);
@ -281,19 +281,19 @@ struct Adder
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64, ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64, ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>( 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: private:
template <typename FromVectorType, typename ToVectorType, typename DeltaColumnType> 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) for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], delta.getData()[i], time_zone); vec_to[i] = transform.execute(vec_from[i], delta.getData()[i], time_zone);
} }
template <typename FromType, typename ToVectorType, typename DeltaColumnType> 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) for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(from, delta.getData()[i], time_zone); 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; const IColumn & delta_column = *block.getByPosition(arguments[1]).column;
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_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 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())) 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 else
{ {

View File

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

View File

@ -67,7 +67,7 @@ public:
if (Impl::is_fixed_to_constant) if (Impl::is_fixed_to_constant)
{ {
ResultType res = 0; 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)); 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(); typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
vec_res.resize(col_fixed->size()); 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); 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())) else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
{ {
auto col_res = ColumnFixedString::create(col_fixed->getN()); 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); block.getByPosition(result).column = std::move(col_res);
} }
else else

View File

@ -78,7 +78,7 @@ template <typename A, typename B, typename Op>
struct NumComparisonImpl 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. /// 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. /** 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), * 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(); size_t size = a.size();
const A * __restrict a_pos = a.data(); 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); 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 & a_data, const ColumnString::Offsets & a_offsets,
const ColumnString::Chars & b_data, ColumnString::Offset b_size, const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c) 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 & a_data, ColumnString::Offset a_n,
const ColumnString::Chars & b_data, ColumnString::Offset b_size, const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c) PaddedPODArray<UInt8> & c)
@ -271,7 +271,7 @@ struct StringComparisonImpl
const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets,
PaddedPODArray<UInt8> & c) 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( static void constant_fixed_string_vector(
@ -279,10 +279,10 @@ struct StringComparisonImpl
const ColumnString::Chars & b_data, ColumnString::Offset b_n, const ColumnString::Chars & b_data, ColumnString::Offset b_n,
PaddedPODArray<UInt8> & c) 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 & a_data, ColumnString::Offset a_size,
const ColumnString::Chars & b_data, ColumnString::Offset b_size, const ColumnString::Chars & b_data, ColumnString::Offset b_size,
UInt8 & c) 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 & a_data, const ColumnString::Offsets & a_offsets,
const ColumnString::Chars & b_data, ColumnString::Offset b_size, const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c) 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 & a_data, ColumnString::Offset a_n,
const ColumnString::Chars & b_data, ColumnString::Offset b_size, const ColumnString::Chars & b_data, ColumnString::Offset b_size,
PaddedPODArray<UInt8> & c) PaddedPODArray<UInt8> & c)
@ -435,7 +435,7 @@ struct StringEqualsImpl
const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets,
PaddedPODArray<UInt8> & c) 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( static void constant_fixed_string_vector(
@ -443,10 +443,10 @@ struct StringEqualsImpl
const ColumnString::Chars & b_data, ColumnString::Offset b_n, const ColumnString::Chars & b_data, ColumnString::Offset b_n,
PaddedPODArray<UInt8> & c) 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 & a_data, ColumnString::Offset a_size,
const ColumnString::Chars & b_data, ColumnString::Offset b_size, const ColumnString::Chars & b_data, ColumnString::Offset b_size,
UInt8 & c) UInt8 & c)
@ -467,25 +467,25 @@ struct StringComparisonImpl<NotEqualsOp<A, B>> : StringEqualsImpl<false> {};
template <typename Op> template <typename Op>
struct GenericComparisonImpl 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) for (size_t i = 0, size = a.size(); i < size; ++i)
c[i] = Op::apply(a.compareAt(i, i, b, 1), 0); 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(); auto b_materialized = b.cloneResized(1)->convertToFullColumnIfConst();
for (size_t i = 0, size = a.size(); i < size; ++i) for (size_t i = 0, size = a.size(); i < size; ++i)
c[i] = Op::apply(a.compareAt(i, 0, *b_materialized, 1), 0); 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); c = Op::apply(a.compareAt(0, 0, b, 1), 0);
} }
@ -582,7 +582,7 @@ private:
ColumnUInt8::Container & vec_res = col_res->getData(); ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col_left->getData().size()); 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); block.getByPosition(result).column = std::move(col_res);
return true; return true;
@ -593,7 +593,7 @@ private:
ColumnUInt8::Container & vec_res = col_res->getData(); ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col_left->size()); 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); block.getByPosition(result).column = std::move(col_res);
return true; return true;
@ -611,7 +611,7 @@ private:
ColumnUInt8::Container & vec_res = col_res->getData(); ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col_left->size()); 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); block.getByPosition(result).column = std::move(col_res);
return true; return true;
@ -619,7 +619,7 @@ private:
else if (auto col_right_const = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped)) else if (auto col_right_const = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
{ {
UInt8 res = 0; 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)); block.getByPosition(result).column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
return true; return true;
@ -759,7 +759,7 @@ private:
if (c0_const && c1_const) if (c0_const && c1_const)
{ {
UInt8 res = 0; 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)); block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(c0_const->size(), toField(res));
return true; return true;
} }
@ -780,7 +780,7 @@ private:
c1_fixed_string->getChars(), c1_fixed_string->getN(), c1_fixed_string->getChars(), c1_fixed_string->getN(),
c_res->getData()); c_res->getData());
else if (c0_string && c1_const) else if (c0_string && c1_const)
StringImpl::string_vector_constant( StringImpl::string_vectorConstant(
c0_string->getChars(), c0_string->getOffsets(), c0_string->getChars(), c0_string->getOffsets(),
*c1_const_chars, c1_const_size, *c1_const_chars, c1_const_size,
c_res->getData()); c_res->getData());
@ -795,7 +795,7 @@ private:
c1_fixed_string->getChars(), c1_fixed_string->getN(), c1_fixed_string->getChars(), c1_fixed_string->getN(),
c_res->getData()); c_res->getData());
else if (c0_fixed_string && c1_const) else if (c0_fixed_string && c1_const)
StringImpl::fixed_string_vector_constant( StringImpl::fixed_string_vectorConstant(
c0_fixed_string->getChars(), c0_fixed_string->getN(), c0_fixed_string->getChars(), c0_fixed_string->getN(),
*c1_const_chars, c1_const_size, *c1_const_chars, c1_const_size,
c_res->getData()); c_res->getData());
@ -1081,7 +1081,7 @@ private:
if (c0_const && c1_const) if (c0_const && c1_const)
{ {
UInt8 res = 0; 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)); block.getByPosition(result).column = DataTypeUInt8().createColumnConst(c0->size(), toField(res));
} }
else else
@ -1091,11 +1091,11 @@ private:
vec_res.resize(c0->size()); vec_res.resize(c0->size());
if (c0_const) 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) else if (c1_const)
GenericComparisonImpl<Op<int, int>>::vector_constant(*c0, *c1, vec_res); GenericComparisonImpl<Op<int, int>>::vectorConstant(*c0, *c1, vec_res);
else 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); block.getByPosition(result).column = std::move(c_res);
} }

View File

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

View File

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

View File

@ -63,7 +63,7 @@ public:
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get())) if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception( throw Exception(
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); "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 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. /// The blame for resizing output is for the callee.
if (col_haystack_vector) if (col_haystack_vector)
Impl::vector_constant( Impl::vectorConstant(
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res, edit_distance); col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res, edit_distance);
else else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -155,7 +155,7 @@ struct PositionImpl
using ResultType = UInt64; using ResultType = UInt64;
/// Find one substring in many strings. /// 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 ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
{ {
const UInt8 * begin = data.data(); const UInt8 * begin = data.data();
@ -192,7 +192,7 @@ struct PositionImpl
} }
/// Search for substring in string. /// 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(data);
Impl::toLowerIfNeed(needle); Impl::toLowerIfNeed(needle);
@ -205,7 +205,7 @@ struct PositionImpl
} }
/// Search each time for a different single substring inside each time different string. /// 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::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data, const ColumnString::Chars & needle_data,
@ -255,7 +255,7 @@ struct PositionImpl
} }
/// Find many substrings in single string. /// Find many substrings in single string.
static void constant_vector( static void constantVector(
const String & haystack, const String & haystack,
const ColumnString::Chars & needle_data, const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets, const ColumnString::Offsets & needle_offsets,
@ -303,7 +303,7 @@ struct MultiSearchAllPositionsImpl
{ {
using ResultType = UInt64; using ResultType = UInt64;
static void vector_constant( static void vectorConstant(
const ColumnString::Chars & haystack_data, const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles, const std::vector<StringRef> & needles,
@ -344,9 +344,9 @@ struct MultiSearchImpl
/// Variable for understanding, if we used offsets for the output, most /// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray. /// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false; 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::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles, const std::vector<StringRef> & needles,
@ -381,9 +381,9 @@ struct MultiSearchFirstPositionImpl
/// Variable for understanding, if we used offsets for the output, most /// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray. /// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false; 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::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles, const std::vector<StringRef> & needles,
@ -428,9 +428,9 @@ struct MultiSearchFirstIndexImpl
/// Variable for understanding, if we used offsets for the output, most /// Variable for understanding, if we used offsets for the output, most
/// likely to determine whether the function returns ColumnVector of ColumnArray. /// likely to determine whether the function returns ColumnVector of ColumnArray.
static constexpr bool is_column_array = false; 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::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const std::vector<StringRef> & needles, const std::vector<StringRef> & needles,
@ -467,7 +467,7 @@ struct HasTokenImpl
static constexpr bool use_default_implementation_for_constants = true; 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) const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
{ {
if (offsets.empty()) if (offsets.empty())
@ -508,14 +508,14 @@ struct HasTokenImpl
} }
template <typename... Args> 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); throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
} }
/// Search different needles in single haystack. /// Search different needles in single haystack.
template <typename... Args> 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); 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) if (col_haystack_const && col_needle_const)
{ {
ResultType res{}; 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).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res)); = block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
return; return;
@ -135,17 +135,17 @@ public:
const ColumnString * col_needle_vector = checkAndGetColumn<ColumnString>(&*column_needle); const ColumnString * col_needle_vector = checkAndGetColumn<ColumnString>(&*column_needle);
if (col_haystack_vector && col_needle_vector) if (col_haystack_vector && col_needle_vector)
Impl::vector_vector( Impl::vectorVector(
col_haystack_vector->getChars(), col_haystack_vector->getChars(),
col_haystack_vector->getOffsets(), col_haystack_vector->getOffsets(),
col_needle_vector->getChars(), col_needle_vector->getChars(),
col_needle_vector->getOffsets(), col_needle_vector->getOffsets(),
vec_res); vec_res);
else if (col_haystack_vector && col_needle_const) 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); col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue<String>(), vec_res);
else if (col_haystack_const && col_needle_vector) 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); col_haystack_const->getValue<String>(), col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
else else
throw Exception( throw Exception(
@ -277,7 +277,7 @@ public:
vec_res.resize(column_haystack_size * refs.size()); vec_res.resize(column_haystack_size * refs.size());
if (col_haystack_vector) 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 else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); 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())) if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception( throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); "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 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. /// The blame for resizing output is for the callee.
if (col_haystack_vector) 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 else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); 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]; 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; 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]; UInt64 combined = (static_cast<UInt64>(code_points[0]) << 32) | code_points[1];
#ifdef __SSE4_2__ #ifdef __SSE4_2__
@ -249,12 +249,12 @@ struct NgramDistanceImpl
static inline auto dispatchSearcher(Callback callback, Args &&... args) static inline auto dispatchSearcher(Callback callback, Args &&... args)
{ {
if constexpr (!UTF8) if constexpr (!UTF8)
return callback(std::forward<Args>(args)..., readASCIICodePoints, ASCIIHash); return callback(std::forward<Args>(args)..., readASCIICodePoints, calculateASCIIHash);
else 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 = {}; NgramStats common_stats = {};
@ -284,7 +284,7 @@ struct NgramDistanceImpl
} }
} }
static void vector_vector( static void vectorVector(
const ColumnString::Chars & haystack_data, const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const ColumnString::Chars & needle_data, const ColumnString::Chars & needle_data,
@ -358,7 +358,7 @@ struct NgramDistanceImpl
} }
} }
static void constant_vector( static void constantVector(
std::string haystack, std::string haystack,
const ColumnString::Chars & needle_data, const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets, const ColumnString::Offsets & needle_offsets,
@ -367,7 +367,7 @@ struct NgramDistanceImpl
/// For symmetric version it is better to use vector_constant /// For symmetric version it is better to use vector_constant
if constexpr (symmetric) if constexpr (symmetric)
{ {
vector_constant(needle_data, needle_offsets, std::move(haystack), res); vectorConstant(needle_data, needle_offsets, std::move(haystack), res);
} }
else else
{ {
@ -423,7 +423,7 @@ struct NgramDistanceImpl
} }
} }
static void vector_constant( static void vectorConstant(
const ColumnString::Chars & data, const ColumnString::Chars & data,
const ColumnString::Offsets & offsets, const ColumnString::Offsets & offsets,
std::string needle, std::string needle,

View File

@ -72,7 +72,7 @@ public:
+ std::to_string(Impl::max_string_size), + std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_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).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res)); = block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
return; return;
@ -96,11 +96,11 @@ public:
+ std::to_string(Impl::max_string_size), + std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_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) else if (col_haystack_vector && col_needle_vector)
{ {
Impl::vector_vector( Impl::vectorVector(
col_haystack_vector->getChars(), col_haystack_vector->getChars(),
col_haystack_vector->getOffsets(), col_haystack_vector->getOffsets(),
col_needle_vector->getChars(), col_needle_vector->getChars(),
@ -117,7 +117,7 @@ public:
+ std::to_string(Impl::max_string_size), + std::to_string(Impl::max_string_size),
ErrorCodes::TOO_LARGE_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 else
{ {

View File

@ -81,7 +81,7 @@ struct ExtractParamImpl
static constexpr bool use_default_implementation_for_constants = true; static constexpr bool use_default_implementation_for_constants = true;
/// It is assumed that `res` is the correct size and initialized with zeros. /// 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, std::string needle,
PaddedPODArray<ResultType> & res) PaddedPODArray<ResultType> & res)
{ {
@ -121,12 +121,12 @@ struct ExtractParamImpl
memset(&res[i], 0, (res.size() - i) * sizeof(res[0])); 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); 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); 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()); 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()); res_data.resize(data.size());
array(data.data(), data.data() + data.size(), res_data.data()); 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()); 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); 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); 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); 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()); 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); 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); 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); 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; 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_data += last_3_periods[2] + 1 - begin;
res_size = last_3_periods[1] - last_3_periods[2] - 1; res_size = last_3_periods[1] - last_3_periods[2] - 1;

View File

@ -33,16 +33,16 @@
#define MAX_HASH_VALUE 95690 #define MAX_HASH_VALUE 95690
/* maximum key range = 95616, duplicates = 0 */ /* maximum key range = 95616, duplicates = 0 */
class tldLookupHash class TopLevelDomainLookupHash
{ {
private: private:
static inline unsigned int hash(const char * str, size_t len); static inline unsigned int hash(const char * str, size_t len);
public: 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[] = { 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, 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])]; 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[] static const char * const wordlist[]
= {"", = {"",

View File

@ -1,15 +1,15 @@
#pragma once #pragma once
// Definition of the class generated by gperf, present on gperf/tldLookup.gperf // Definition of the class generated by gperf, present on gperf/tldLookup.gperf
class tldLookupHash class TopLevelDomainLookupHash
{ {
private: private:
static inline unsigned int hash (const char *str, size_t len); static inline unsigned int hash(const char *str, size_t len);
public: public:
static const char *is_valid (const char *str, size_t len); static const char * isValid(const char *str, size_t len);
}; };
namespace DB 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 & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets,
const ColumnString::Chars & item_values, const ColumnString::Offsets & item_offsets, const ColumnString::Chars & item_values, const ColumnString::Offsets & item_offsets,
PaddedPODArray<typename IndexConv::ResultType> & result, PaddedPODArray<typename IndexConv::ResultType> & result,
@ -731,7 +731,7 @@ private:
} }
else if (const auto item_arg_vector = checkAndGetColumn<ColumnString>(item_arg)) 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_nested->getOffsets(), item_arg_vector->getChars(), item_arg_vector->getOffsets(),
col_res->getData(), null_map_data, null_map_item); 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]; 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; 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(); auto c_res = ColumnString::create();
std::vector<const ColumnString::Chars *> data(arguments.size()); std::vector<const ColumnString::Chars *> data(arguments.size());
std::vector<const ColumnString::Offsets *> offsets(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()); std::vector<String> constant_strings(arguments.size());
bool has_column_string = false; bool has_column_string = false;
bool has_column_fixed_string = false; bool has_column_fixed_string = false;
@ -136,7 +136,7 @@ private:
{ {
has_column_fixed_string = true; has_column_fixed_string = true;
data[i] = &fixed_col->getChars(); 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())) else if (const ColumnConst * const_col = checkAndGetColumnConstStringOrFixedString(column.get()))
{ {
@ -159,7 +159,7 @@ private:
std::move(pattern), std::move(pattern),
data, data,
offsets, offsets,
fixed_string_N, fixed_string_sizes,
constant_strings, constant_strings,
c_res->getChars(), c_res->getChars(),
c_res->getOffsets(), c_res->getOffsets(),

View File

@ -145,13 +145,13 @@ private:
ColumnInt64::Container & result) ColumnInt64::Container & result)
{ {
if (auto * y_vec_16 = checkAndGetColumn<ColumnUInt16>(&y)) 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)) 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)) 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)) 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 else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); 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) ColumnInt64::Container & result)
{ {
if (auto * y_vec_16 = checkAndGetColumn<ColumnUInt16>(&y)) 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)) 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 else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); 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> template <typename Transform, typename T1, typename T2>
void vector_vector( void vectorVector(
const ColumnVector<T1> & x, const ColumnVector<T2> & y, const ColumnVector<T1> & x, const ColumnVector<T2> & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result) ColumnInt64::Container & result)
@ -183,7 +183,7 @@ private:
} }
template <typename Transform, typename T1, typename T2> template <typename Transform, typename T1, typename T2>
void vector_constant( void vectorConstant(
const ColumnVector<T1> & x, T2 y, const ColumnVector<T1> & x, T2 y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result) ColumnInt64::Container & result)
@ -194,7 +194,7 @@ private:
} }
template <typename Transform, typename T1, typename T2> template <typename Transform, typename T1, typename T2>
void constant_vector( void constantVector(
T1 x, const ColumnVector<T2> & y, T1 x, const ColumnVector<T2> & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result) ColumnInt64::Container & result)

View File

@ -155,7 +155,7 @@ private:
writeNumber2(target, day); 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)); writeNumber4(target, ToYearImpl::execute(source, timezone));
writeNumber2(target + 5, ToMonthImpl::execute(source, timezone)); writeNumber2(target + 5, ToMonthImpl::execute(source, timezone));
@ -183,7 +183,7 @@ private:
*target += (day == 7 ? 0 : day); *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)); writeNumber2(target, ToISOWeekImpl::execute(source, timezone));
} }
@ -214,7 +214,7 @@ private:
writeNumber2(target, ToMinuteImpl::execute(source, timezone)); 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); auto hour = ToHourImpl::execute(source, timezone);
if (hour >= 12) if (hour >= 12)
@ -232,7 +232,7 @@ private:
writeNumber2(target, ToSecondImpl::execute(source, timezone)); 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, ToHourImpl::execute(source, timezone));
writeNumber2(target + 3, ToMinuteImpl::execute(source, timezone)); writeNumber2(target + 3, ToMinuteImpl::execute(source, timezone));
@ -394,7 +394,7 @@ public:
const char * end = pos + pattern.size(); const char * end = pos + pattern.size();
/// Add shift to previous action; or if there were none, add noop action with shift. /// 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()) if (instructions.empty())
instructions.emplace_back(&Action<T>::noop); 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). /// 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>) if constexpr (std::is_same_v<T, UInt32>)
instructions.emplace_back(func, shift); instructions.emplace_back(func, shift);
else else
addShift(shift); add_shift(shift);
}; };
while (true) while (true)
@ -419,7 +419,7 @@ public:
if (pos < percent_pos) if (pos < percent_pos)
{ {
result.append(pos, percent_pos); result.append(pos, percent_pos);
addShift(percent_pos - pos); add_shift(percent_pos - pos);
} }
pos = percent_pos + 1; pos = percent_pos + 1;
@ -505,58 +505,58 @@ public:
// Minute (00-59) // Minute (00-59)
case 'M': case 'M':
addInstructionOrShift(&Action<T>::minute, 2); add_instruction_or_shift(&Action<T>::minute, 2);
result.append("00"); result.append("00");
break; break;
// AM or PM // AM or PM
case 'p': case 'p':
addInstructionOrShift(&Action<T>::AMPM, 2); add_instruction_or_shift(&Action<T>::AMPM, 2);
result.append("AM"); result.append("AM");
break; break;
// 24-hour HH:MM time, equivalent to %H:%M 14:55 // 24-hour HH:MM time, equivalent to %H:%M 14:55
case 'R': case 'R':
addInstructionOrShift(&Action<T>::hhmm24, 5); add_instruction_or_shift(&Action<T>::hhmm24, 5);
result.append("00:00"); result.append("00:00");
break; break;
// Seconds // Seconds
case 'S': case 'S':
addInstructionOrShift(&Action<T>::second, 2); add_instruction_or_shift(&Action<T>::second, 2);
result.append("00"); result.append("00");
break; break;
// ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S 14:55:02 // ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S 14:55:02
case 'T': case 'T':
addInstructionOrShift(&Action<T>::ISO8601Time, 8); add_instruction_or_shift(&Action<T>::ISO8601Time, 8);
result.append("00:00:00"); result.append("00:00:00");
break; break;
// Hour in 24h format (00-23) // Hour in 24h format (00-23)
case 'H': case 'H':
addInstructionOrShift(&Action<T>::hour24, 2); add_instruction_or_shift(&Action<T>::hour24, 2);
result.append("00"); result.append("00");
break; break;
// Hour in 12h format (01-12) // Hour in 12h format (01-12)
case 'I': case 'I':
addInstructionOrShift(&Action<T>::hour12, 2); add_instruction_or_shift(&Action<T>::hour12, 2);
result.append("12"); result.append("12");
break; break;
/// Escaped literal characters. /// Escaped literal characters.
case '%': case '%':
result += '%'; result += '%';
addShift(1); add_shift(1);
break; break;
case 't': case 't':
result += '\t'; result += '\t';
addShift(1); add_shift(1);
break; break;
case 'n': case 'n':
result += '\n'; result += '\n';
addShift(1); add_shift(1);
break; break;
// Unimplemented // Unimplemented
@ -575,7 +575,7 @@ public:
else else
{ {
result.append(pos, end); result.append(pos, end);
addShift(end + 1 - pos); /// including zero terminator add_shift(end + 1 - pos); /// including zero terminator
break; break;
} }
} }

View File

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

View File

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

View File

@ -24,7 +24,7 @@ struct DivideIntegralByConstantImpl
using ResultType = typename DivideIntegralImpl<A, B>::ResultType; using ResultType = typename DivideIntegralImpl<A, B>::ResultType;
static const constexpr bool allow_fixed_string = false; 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)) if (unlikely(b == 0))
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION); 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; size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i) 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; size_t size = data.size() / n;

View File

@ -25,7 +25,7 @@ struct ModuloByConstantImpl
using ResultType = typename ModuloImpl<A, B>::ResultType; using ResultType = typename ModuloImpl<A, B>::ResultType;
static const constexpr bool allow_fixed_string = false; 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)) if (unlikely(b == 0))
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION); 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); 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) for (size_t i = 1; i < arguments.size(); ++i)
{ {
auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get()); auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (array == nullptr && i != 1) 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()); auto * tuple = checkAndGetDataType<DataTypeTuple>(array ? array->getNestedType().get() : arguments[i].get());
if (tuple == nullptr) 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(); const DataTypes & elements = tuple->getElements();
if (elements.size() != 2) 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())) for (auto j : ext::range(0, elements.size()))
{ {
if (!isNativeNumber(elements[j])) 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); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
} }
} }
@ -207,7 +207,7 @@ private:
{ {
Polygon<Type> polygon; 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) 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; auto tuple_col = array_col ? checkAndGetColumn<ColumnTuple>(&array_col->getData()) : nullptr;
if (!tuple_col) 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 & tuple_columns = tuple_col->getColumns();
const auto & column_x = tuple_columns[0]; const auto & column_x = tuple_columns[0];
@ -230,7 +230,7 @@ private:
auto size = column_x->size(); auto size = column_x->size();
if (size == 0) 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)) for (auto j : ext::range(0, size))
{ {
@ -244,13 +244,12 @@ private:
container.push_back(container.front()); container.push_back(container.front());
} }
auto callImpl = use_object_pool auto call_impl = use_object_pool
? FunctionPointInPolygonDetail::callPointInPolygonImplWithPool<Polygon<Type>, PointInPolygonImpl<Type>> ? FunctionPointInPolygonDetail::callPointInPolygonImplWithPool<Polygon<Type>, PointInPolygonImpl<Type>>
: FunctionPointInPolygonDetail::callPointInPolygonImpl<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()); res_data.resize(data.size());
size_t size = data.size() / n; size_t size = data.size() / n;
@ -100,7 +100,7 @@ public:
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get())) else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
{ {
auto col_res = ColumnFixedString::create(col_fixed->getN()); 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); block.getByPosition(result).column = std::move(col_res);
} }
else 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); throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN);
} }

View File

@ -34,7 +34,7 @@ namespace ErrorCodes
template <typename DurationType> template <typename DurationType>
struct TimeSlotsImpl struct TimeSlotsImpl
{ {
static void vector_vector( static void vectorVector(
const PaddedPODArray<UInt32> & starts, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size, const PaddedPODArray<UInt32> & starts, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets) 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, const PaddedPODArray<UInt32> & starts, DurationType duration, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets) 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, UInt32 start, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets) PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{ {
@ -168,17 +168,17 @@ public:
if (starts && durations) 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); block.getByPosition(result).column = std::move(res);
} }
else if (starts && const_durations) 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); block.getByPosition(result).column = std::move(res);
} }
else if (const_starts && durations) 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); block.getByPosition(result).column = std::move(res);
} }
else else

View File

@ -124,7 +124,7 @@ struct ToValidUTF8Impl
write_buffer.finalize(); 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); 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; static constexpr bool trim_right = true;
}; };
template <typename mode> template <typename Mode>
class FunctionTrimImpl class FunctionTrimImpl
{ {
public: 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); 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_data = reinterpret_cast<const char *>(data);
const char * char_end = char_data + size; const char * char_end = char_data + size;
if constexpr (mode::trim_left) if constexpr (Mode::trim_left)
{ // NOLINT { // NOLINT
const char * found = find_first_not_symbols<' '>(char_data, char_end); const char * found = find_first_not_symbols<' '>(char_data, char_end);
size_t num_chars = found - char_data; size_t num_chars = found - char_data;
char_data += num_chars; char_data += num_chars;
} }
if constexpr (mode::trim_right) if constexpr (Mode::trim_right)
{ // NOLINT { // NOLINT
const char * found = find_last_not_symbols_or_null<' '>(char_data, char_end); const char * found = find_last_not_symbols_or_null<' '>(char_data, char_end);
if (found) if (found)

View File

@ -164,10 +164,10 @@ namespace
/// We store exception messages in session data. /// We store exception messages in session data.
/// Poco HTTPSession also stores exception, but it can be removed at any time. /// Poco HTTPSession also stores exception, but it can be removed at any time.
const auto & sessionData = session->sessionData(); const auto & session_data = session->sessionData();
if (!sessionData.empty()) if (!session_data.empty())
{ {
auto msg = Poco::AnyCast<std::string>(sessionData); auto msg = Poco::AnyCast<std::string>(session_data);
if (!msg.empty()) if (!msg.empty())
{ {
LOG_TRACE((&Logger::get("HTTPCommon")), "Failed communicating with " << host << " with error '" << msg << "' will try to reconnect session"); 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 bool throw_exception = std::is_same_v<ReturnType, void>;
static constexpr auto DATE_TIME_BROKEN_DOWN_LENGTH = 19; static constexpr auto date_time_broken_down_length = 19;
static constexpr auto UNIX_TIMESTAMP_MAX_LENGTH = 10; 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; char * s_pos = s;
/// A piece similar to unix timestamp. /// 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 = *buf.position();
++s_pos; ++s_pos;
@ -803,7 +803,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
/// 2015-01-01 01:02:03 /// 2015-01-01 01:02:03
if (s_pos == s + 4 && !buf.eof() && (*buf.position() < '0' || *buf.position() > '9')) 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); size_t size = buf.read(s_pos, remaining_size);
if (remaining_size != size) if (remaining_size != size)
{ {

View File

@ -106,7 +106,7 @@ namespace S3
URI::URI(const Poco::URI & uri_) URI::URI(const Poco::URI & uri_)
{ {
static const std::regex BUCKET_KEY_PATTERN("([^/]+)/(.*)"); static const std::regex bucket_key_pattern("([^/]+)/(.*)"); /// TODO std::regex is discouraged
uri = uri_; uri = uri_;
@ -133,7 +133,7 @@ namespace S3
// Parse bucket and key from path. // Parse bucket and key from path.
std::smatch match; std::smatch match;
std::regex_search(uri.getPath(), match, BUCKET_KEY_PATTERN); std::regex_search(uri.getPath(), match, bucket_key_pattern);
if (!match.empty()) if (!match.empty())
{ {
bucket = match.str(1); bucket = match.str(1);

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