From c7863e1ac68435d7d30da842b2cd7b95e9a4dc13 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 23 Mar 2020 05:12:31 +0300 Subject: [PATCH] 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 --- .clang-tidy | 41 ++++ base/common/shift10.cpp | 10 +- base/common/tests/date_lut4.cpp | 6 +- base/mysqlxx/Pool.cpp | 2 +- base/mysqlxx/Pool.h | 2 +- base/mysqlxx/PoolFactory.cpp | 6 +- base/mysqlxx/PoolFactory.h | 4 +- base/mysqlxx/PoolWithFailover.cpp | 6 +- base/mysqlxx/PoolWithFailover.h | 2 +- base/mysqlxx/Value.cpp | 4 +- dbms/programs/client/Client.cpp | 6 +- dbms/programs/copier/ClusterCopierApp.cpp | 10 +- dbms/programs/odbc-bridge/ODBCBridge.cpp | 10 +- dbms/programs/server/HTTPHandler.cpp | 6 +- dbms/programs/server/HTTPHandlerFactory.cpp | 4 +- dbms/programs/server/MySQLHandler.cpp | 4 +- dbms/programs/server/MySQLHandlerFactory.cpp | 24 +-- dbms/programs/server/Server.cpp | 6 +- dbms/src/Access/QuotaCache.cpp | 2 +- .../AggregateFunctionGroupArrayMoving.cpp | 18 +- .../AggregateFunctionGroupUniqArray.cpp | 28 +-- .../AggregateFunctionMLMethod.cpp | 14 +- .../AggregateFunctionMLMethod.h | 8 +- dbms/src/Columns/ColumnAggregateFunction.cpp | 8 +- dbms/src/Columns/ColumnArray.cpp | 35 ++-- dbms/src/Columns/ColumnFunction.cpp | 4 +- dbms/src/Columns/ColumnLowCardinality.cpp | 28 +-- dbms/src/Common/Dwarf.cpp | 180 +++++++++--------- dbms/src/Common/PoolBase.h | 2 +- dbms/src/Common/RWLock.cpp | 6 +- dbms/src/Common/ThreadFuzzer.cpp | 6 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 6 +- .../src/Common/tests/parallel_aggregation.cpp | 2 +- .../Compression/CompressionCodecGorilla.cpp | 14 +- .../tests/gtest_compressionCodec.cpp | 14 +- dbms/src/Core/DecimalComparison.h | 12 +- dbms/src/DataTypes/DataTypeDateTime64.cpp | 12 +- dbms/src/DataTypes/DataTypeLowCardinality.cpp | 12 +- dbms/src/DataTypes/getMostSubtype.cpp | 21 +- dbms/src/Databases/DatabaseMySQL.cpp | 4 +- dbms/src/Databases/DatabaseOrdinary.cpp | 4 +- .../ClickHouseDictionarySource.cpp | 4 +- dbms/src/Dictionaries/DictionaryFactory.cpp | 4 +- .../ExecutableDictionarySource.cpp | 4 +- .../src/Dictionaries/FileDictionarySource.cpp | 4 +- .../src/Dictionaries/HTTPDictionarySource.cpp | 4 +- .../Dictionaries/LibraryDictionarySource.cpp | 40 ++-- .../Dictionaries/MongoDBDictionarySource.cpp | 12 +- .../Dictionaries/MySQLDictionarySource.cpp | 20 +- .../Dictionaries/RedisBlockInputStream.cpp | 12 +- .../Dictionaries/RedisDictionarySource.cpp | 4 +- dbms/src/Dictionaries/TrieDictionary.cpp | 12 +- .../src/Dictionaries/XDBCDictionarySource.cpp | 8 +- .../getDictionaryConfigurationFromAST.cpp | 6 +- dbms/src/Disks/DiskS3.cpp | 2 +- dbms/src/Disks/DiskSpaceMonitor.cpp | 2 +- dbms/src/Disks/tests/gtest_disk.cpp | 44 ++--- dbms/src/Functions/CRC.cpp | 16 +- dbms/src/Functions/EmptyImpl.h | 6 +- dbms/src/Functions/FunctionBinaryArithmetic.h | 70 +++---- .../FunctionDateOrDateTimeAddInterval.h | 20 +- dbms/src/Functions/FunctionHelpers.cpp | 12 +- dbms/src/Functions/FunctionStringOrArrayToT.h | 4 +- dbms/src/Functions/FunctionStringToString.h | 2 +- dbms/src/Functions/FunctionsComparison.h | 62 +++--- dbms/src/Functions/FunctionsLogical.cpp | 6 +- dbms/src/Functions/FunctionsStringRegex.cpp | 30 +-- dbms/src/Functions/FunctionsStringRegex.h | 4 +- dbms/src/Functions/FunctionsStringSearch.cpp | 28 +-- dbms/src/Functions/FunctionsStringSearch.h | 14 +- .../Functions/FunctionsStringSimilarity.cpp | 18 +- .../src/Functions/FunctionsStringSimilarity.h | 8 +- dbms/src/Functions/FunctionsVisitParam.h | 6 +- dbms/src/Functions/LowerUpperImpl.h | 2 +- dbms/src/Functions/LowerUpperUTF8Impl.h | 2 +- dbms/src/Functions/URL/FunctionsURL.h | 4 +- dbms/src/Functions/URL/decodeURLComponent.cpp | 2 +- .../Functions/URL/firstSignificantSubdomain.h | 2 +- .../src/Functions/URL/tldLookup.generated.cpp | 8 +- dbms/src/Functions/URL/tldLookup.h | 8 +- dbms/src/Functions/array/arrayIndex.h | 4 +- dbms/src/Functions/array/length.cpp | 4 +- dbms/src/Functions/concat.cpp | 6 +- dbms/src/Functions/dateDiff.cpp | 18 +- dbms/src/Functions/formatDateTime.cpp | 38 ++-- dbms/src/Functions/formatString.cpp | 6 +- dbms/src/Functions/if.cpp | 34 ++-- dbms/src/Functions/intDiv.cpp | 2 +- dbms/src/Functions/isValidUTF8.cpp | 4 +- dbms/src/Functions/lengthUTF8.cpp | 4 +- dbms/src/Functions/modulo.cpp | 2 +- dbms/src/Functions/pointInPolygon.cpp | 21 +- dbms/src/Functions/reverse.cpp | 4 +- dbms/src/Functions/reverseUTF8.cpp | 2 +- dbms/src/Functions/timeSlots.cpp | 12 +- dbms/src/Functions/toValidUTF8.cpp | 2 +- dbms/src/Functions/trim.cpp | 8 +- dbms/src/IO/HTTPCommon.cpp | 6 +- dbms/src/IO/ReadHelpers.cpp | 10 +- dbms/src/IO/S3Common.cpp | 4 +- dbms/src/IO/tests/gtest_DateTimeToString.cpp | 8 +- dbms/src/IO/tests/hashing_read_buffer.cpp | 4 +- .../src/IO/tests/o_direct_and_dirty_pages.cpp | 4 +- dbms/src/IO/tests/ryu_test.cpp | 18 +- dbms/src/Interpreters/ActionsVisitor.cpp | 12 +- .../Interpreters/ArrayJoinedColumnsVisitor.h | 2 +- dbms/src/Interpreters/CatBoostModel.cpp | 46 +++-- dbms/src/Interpreters/Context.cpp | 6 +- dbms/src/Interpreters/DatabaseCatalog.cpp | 7 +- dbms/src/Interpreters/ExpressionActions.cpp | 8 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 12 +- dbms/src/Interpreters/ExpressionJIT.cpp | 20 +- dbms/src/Interpreters/ExternalLoader.cpp | 24 +-- .../Interpreters/InternalTextLogsQueue.cpp | 3 +- .../Interpreters/InterpreterSelectQuery.cpp | 4 +- .../JoinToSubqueryTransformVisitor.cpp | 2 +- .../PredicateExpressionsOptimizer.cpp | 2 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 4 +- .../evaluateConstantExpression.cpp | 10 +- dbms/src/Interpreters/tests/hash_map.cpp | 4 +- dbms/src/Parsers/ASTSelectQuery.cpp | 10 +- dbms/src/Parsers/ASTSelectQuery.h | 8 +- dbms/src/Parsers/Lexer.cpp | 6 +- .../Processors/Executors/PipelineExecutor.cpp | 4 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 14 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 10 +- .../Transforms/CreatingSetsTransform.cpp | 6 +- .../tests/processors_test_aggregation.cpp | 4 +- .../tests/processors_test_chain.cpp | 6 +- .../tests/processors_test_expand_pipeline.cpp | 6 +- .../tests/processors_test_merge.cpp | 14 +- ...rocessors_test_merge_sorting_transform.cpp | 8 +- ...ocessors_test_merging_sorted_transform.cpp | 14 +- .../Storages/Kafka/KafkaBlockInputStream.cpp | 24 +-- .../Storages/Kafka/KafkaBlockOutputStream.cpp | 2 +- .../Kafka/ReadBufferFromKafkaConsumer.cpp | 6 +- .../Kafka/WriteBufferToKafkaProducer.cpp | 2 +- .../Kafka/WriteBufferToKafkaProducer.h | 2 +- .../src/Storages/LiveView/StorageLiveView.cpp | 4 +- .../MergeTree/IMergedBlockOutputStream.h | 2 +- .../MergeTreeBaseSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 +- .../MergeTree/MergeTreeDataMergerMutator.h | 10 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 38 ++-- .../MergeTree/MergeTreeDataPartChecksum.h | 6 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTreeIndexConditionBloomFilter.cpp | 4 +- .../MergeTree/MergeTreeReaderCompact.cpp | 16 +- .../MergeTree/MergeTreeReaderCompact.h | 12 +- .../MergeTree/MergeTreeReaderWide.cpp | 14 +- .../Storages/MergeTree/MergeTreeReaderWide.h | 12 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +- .../ReplicatedMergeTreeQuorumAddedParts.h | 8 +- dbms/src/Storages/StorageFactory.cpp | 10 +- dbms/src/Storages/StorageFile.cpp | 5 +- dbms/src/Storages/StorageGenerateRandom.cpp | 4 +- dbms/src/Storages/StorageLog.cpp | 6 +- dbms/src/Storages/StorageMaterializedView.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 2 +- dbms/src/Storages/StorageMySQL.cpp | 4 +- .../Storages/StorageReplicatedMergeTree.cpp | 4 +- .../System/StorageSystemPartsBase.cpp | 8 +- dbms/src/Storages/tests/gtest_storage_log.cpp | 16 +- .../src/TableFunctions/TableFunctionMySQL.cpp | 2 +- .../TableFunctions/TableFunctionRemote.cpp | 8 +- 167 files changed, 948 insertions(+), 923 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index d7380a9a89b..3c222fbf8da 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -57,6 +57,7 @@ Checks: '-*, readability-simplify-subscript-expr, readability-simplify-boolean-expr, readability-inconsistent-declaration-parameter-name, + readability-identifier-naming, bugprone-undelegated-constructor, bugprone-argument-comment, @@ -162,3 +163,43 @@ Checks: '-*, boost-use-to-string, ' WarningsAsErrors: '*' + +CheckOptions: + - key: readability-identifier-naming.ClassCase + value: CamelCase + - key: readability-identifier-naming.EnumCase + value: CamelCase + - key: readability-identifier-naming.LocalVariableCase + value: lower_case + - key: readability-identifier-naming.StaticConstantCase + value: aNy_CasE + - key: readability-identifier-naming.MemberCase + value: lower_case + - key: readability-identifier-naming.PrivateMemberPrefix + value: '' + - key: readability-identifier-naming.ProtectedMemberPrefix + value: '' + - key: readability-identifier-naming.PublicMemberCase + value: lower_case + - key: readability-identifier-naming.MethodCase + value: camelBack + - key: readability-identifier-naming.PrivateMethodPrefix + value: '' + - key: readability-identifier-naming.ProtectedMethodPrefix + value: '' + - key: readability-identifier-naming.ParameterPackCase + value: lower_case + - key: readability-identifier-naming.StructCase + value: CamelCase + - key: readability-identifier-naming.TemplateTemplateParameterCase + value: CamelCase + - key: readability-identifier-naming.TemplateUsingCase + value: lower_case + - key: readability-identifier-naming.TypeTemplateParameterCase + value: CamelCase + - key: readability-identifier-naming.TypedefCase + value: CamelCase + - key: readability-identifier-naming.UnionCase + value: CamelCase + - key: readability-identifier-naming.UsingCase + value: CamelCase diff --git a/base/common/shift10.cpp b/base/common/shift10.cpp index 341ced6aa81..b7b39182145 100644 --- a/base/common/shift10.cpp +++ b/base/common/shift10.cpp @@ -8,8 +8,8 @@ template static T shift10Impl(T x, int exponent) { - static constexpr ssize_t MIN_EXPONENT = -323; - static constexpr ssize_t MAX_EXPONENT = 308; + static constexpr ssize_t min_exponent = -323; + static constexpr ssize_t max_exponent = 308; static const long double powers10[] = { @@ -47,12 +47,12 @@ static T shift10Impl(T x, int exponent) 1e291L,1e292L,1e293L,1e294L,1e295L,1e296L,1e297L,1e298L,1e299L,1e300L,1e301L,1e302L,1e303L,1e304L,1e305L,1e306L,1e307L,1e308L }; - if (unlikely(exponent < MIN_EXPONENT)) /// Note: there are some values below MIN_EXPONENT that is greater than zero. + if (unlikely(exponent < min_exponent)) /// Note: there are some values below MIN_EXPONENT that is greater than zero. x *= 0; /// Multiplying to keep the sign of zero. - else if (unlikely(exponent > MAX_EXPONENT)) + else if (unlikely(exponent > max_exponent)) x *= std::numeric_limits::infinity(); /// Multiplying to keep the sign of infinity. else - x *= powers10[exponent - MIN_EXPONENT]; + x *= powers10[exponent - min_exponent]; return x; } diff --git a/base/common/tests/date_lut4.cpp b/base/common/tests/date_lut4.cpp index a82ec25f183..86a4708dc79 100644 --- a/base/common/tests/date_lut4.cpp +++ b/base/common/tests/date_lut4.cpp @@ -5,12 +5,12 @@ int main(int, char **) { /** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */ - static const time_t TIME = 66130; + static const time_t time = 66130; const auto & date_lut = DateLUT::instance(); - std::cerr << date_lut.toHour(TIME) << std::endl; - std::cerr << date_lut.toDayNum(TIME) << std::endl; + std::cerr << date_lut.toHour(time) << std::endl; + std::cerr << date_lut.toDayNum(time) << std::endl; const auto * values = reinterpret_cast(&date_lut); diff --git a/base/mysqlxx/Pool.cpp b/base/mysqlxx/Pool.cpp index b8216c254e6..99815363a56 100644 --- a/base/mysqlxx/Pool.cpp +++ b/base/mysqlxx/Pool.cpp @@ -119,7 +119,7 @@ Pool::~Pool() } -Pool::Entry Pool::Get() +Pool::Entry Pool::get() { std::unique_lock lock(mutex); diff --git a/base/mysqlxx/Pool.h b/base/mysqlxx/Pool.h index b5e0c71a5cc..bf9365a064a 100644 --- a/base/mysqlxx/Pool.h +++ b/base/mysqlxx/Pool.h @@ -185,7 +185,7 @@ public: ~Pool(); /// Allocates connection. - Entry Get(); + Entry get(); /// Allocates connection. /// If database is not accessible, returns empty Entry object. diff --git a/base/mysqlxx/PoolFactory.cpp b/base/mysqlxx/PoolFactory.cpp index d9c1ec1519d..f0a5543d723 100644 --- a/base/mysqlxx/PoolFactory.cpp +++ b/base/mysqlxx/PoolFactory.cpp @@ -17,10 +17,10 @@ struct PoolFactory::Impl std::mutex mutex; }; -PoolWithFailover PoolFactory::Get(const std::string & config_name, unsigned default_connections, +PoolWithFailover PoolFactory::get(const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries) { - return Get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries); + return get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries); } /// Duplicate of code from StringUtils.h. Copied here for less dependencies. @@ -72,7 +72,7 @@ static std::string getPoolEntryName(const Poco::Util::AbstractConfiguration & co return entry_name; } -PoolWithFailover PoolFactory::Get(const Poco::Util::AbstractConfiguration & config, +PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries) { diff --git a/base/mysqlxx/PoolFactory.h b/base/mysqlxx/PoolFactory.h index a21c2baca2f..a7bb97cd0c6 100644 --- a/base/mysqlxx/PoolFactory.h +++ b/base/mysqlxx/PoolFactory.h @@ -27,13 +27,13 @@ public: PoolFactory(const PoolFactory &) = delete; /** Allocates a PoolWithFailover to connect to MySQL. */ - PoolWithFailover Get(const std::string & config_name, + PoolWithFailover get(const std::string & config_name, unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, size_t max_tries = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); /** Allocates a PoolWithFailover to connect to MySQL. */ - PoolWithFailover Get(const Poco::Util::AbstractConfiguration & config, + PoolWithFailover get(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index 2beb50d3da8..a763988331b 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -68,7 +68,7 @@ PoolWithFailover::PoolWithFailover(const PoolWithFailover & other) } } -PoolWithFailover::Entry PoolWithFailover::Get() +PoolWithFailover::Entry PoolWithFailover::get() { Poco::Util::Application & app = Poco::Util::Application::instance(); std::lock_guard locker(mutex); @@ -89,7 +89,7 @@ PoolWithFailover::Entry PoolWithFailover::Get() try { - Entry entry = shareable ? pool->Get() : pool->tryGet(); + Entry entry = shareable ? pool->get() : pool->tryGet(); if (!entry.isNull()) { @@ -121,7 +121,7 @@ PoolWithFailover::Entry PoolWithFailover::Get() if (full_pool) { app.logger().error("All connections failed, trying to wait on a full pool " + (*full_pool)->getDescription()); - return (*full_pool)->Get(); + return (*full_pool)->get(); } std::stringstream message; diff --git a/base/mysqlxx/PoolWithFailover.h b/base/mysqlxx/PoolWithFailover.h index fe151240fa5..9db29db5efa 100644 --- a/base/mysqlxx/PoolWithFailover.h +++ b/base/mysqlxx/PoolWithFailover.h @@ -105,6 +105,6 @@ namespace mysqlxx PoolWithFailover(const PoolWithFailover & other); /** Allocates a connection to use. */ - Entry Get(); + Entry get(); }; } diff --git a/base/mysqlxx/Value.cpp b/base/mysqlxx/Value.cpp index 4fed33c5da9..ed66167e8ea 100644 --- a/base/mysqlxx/Value.cpp +++ b/base/mysqlxx/Value.cpp @@ -154,7 +154,7 @@ double Value::readFloatText(const char * buf, size_t length) const void Value::throwException(const char * text) const { - static constexpr size_t MYSQLXX_QUERY_PREVIEW_LENGTH = 1000; + static constexpr size_t preview_length = 1000; std::stringstream info; info << text; @@ -166,7 +166,7 @@ void Value::throwException(const char * text) const } if (res && res->getQuery()) - info << ", query: " << res->getQuery()->str().substr(0, MYSQLXX_QUERY_PREVIEW_LENGTH); + info << ", query: " << res->getQuery()->str().substr(0, preview_length); throw CannotParseValue(info.str()); } diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 21768911beb..42a06925c7d 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1122,7 +1122,7 @@ private: /// to avoid losing sync. if (!cancelled) { - auto cancelQuery = [&] { + auto cancel_query = [&] { connection->sendCancel(); cancelled = true; if (is_interactive) @@ -1134,7 +1134,7 @@ private: if (interrupt_listener.check()) { - cancelQuery(); + cancel_query(); } else { @@ -1145,7 +1145,7 @@ private: << " Waited for " << static_cast(elapsed) << " seconds," << " timeout is " << receive_timeout.totalSeconds() << " seconds." << std::endl; - cancelQuery(); + cancel_query(); } } } diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp index c70e79313b9..e49c0d8eca4 100644 --- a/dbms/programs/copier/ClusterCopierApp.cpp +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -43,11 +43,11 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) void ClusterCopierApp::handleHelp(const std::string &, const std::string &) { - Poco::Util::HelpFormatter helpFormatter(options()); - helpFormatter.setCommand(commandName()); - helpFormatter.setHeader("Copies tables from one cluster to another"); - helpFormatter.setUsage("--config-file --task-path "); - helpFormatter.format(std::cerr); + Poco::Util::HelpFormatter help_formatter(options()); + help_formatter.setCommand(commandName()); + help_formatter.setHeader("Copies tables from one cluster to another"); + help_formatter.setUsage("--config-file --task-path "); + help_formatter.format(std::cerr); stopOptionsProcessing(); } diff --git a/dbms/programs/odbc-bridge/ODBCBridge.cpp b/dbms/programs/odbc-bridge/ODBCBridge.cpp index 9e1e8400cc1..e7e8aca7147 100644 --- a/dbms/programs/odbc-bridge/ODBCBridge.cpp +++ b/dbms/programs/odbc-bridge/ODBCBridge.cpp @@ -79,11 +79,11 @@ namespace void ODBCBridge::handleHelp(const std::string &, const std::string &) { - Poco::Util::HelpFormatter helpFormatter(options()); - helpFormatter.setCommand(commandName()); - helpFormatter.setHeader("HTTP-proxy for odbc requests"); - helpFormatter.setUsage("--http-port "); - helpFormatter.format(std::cerr); + Poco::Util::HelpFormatter help_formatter(options()); + help_formatter.setCommand(commandName()); + help_formatter.setHeader("HTTP-proxy for odbc requests"); + help_formatter.setUsage("--http-port "); + help_formatter.format(std::cerr); stopOptionsProcessing(); } diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 65b605c993d..ec890c0a96d 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -546,7 +546,7 @@ void HTTPHandler::processQuery( client_info.http_method = http_method; client_info.http_user_agent = request.get("User-Agent", ""); - auto appendCallback = [&context] (ProgressCallback callback) + auto append_callback = [&context] (ProgressCallback callback) { auto prev = context.getProgressCallback(); @@ -561,13 +561,13 @@ void HTTPHandler::processQuery( /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. if (settings.send_progress_in_http_headers) - appendCallback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); + append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); - appendCallback([&context, &socket](const Progress &) + append_callback([&context, &socket](const Progress &) { /// Assume that at the point this method is called no one is reading data from the socket any more. /// True for read-only queries. diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp index ab8fb4f7336..7a1bcaf4fc6 100644 --- a/dbms/programs/server/HTTPHandlerFactory.cpp +++ b/dbms/programs/server/HTTPHandlerFactory.cpp @@ -23,9 +23,9 @@ Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHand << ", Content Type: " << request.getContentType() << ", Transfer Encoding: " << request.getTransferEncoding()); - for (auto & handlerFactory: child_handler_factories) + for (auto & handler_factory : child_handler_factories) { - auto handler = handlerFactory->createRequestHandler(request); + auto handler = handler_factory->createRequestHandler(request); if (handler != nullptr) return handler; } diff --git a/dbms/programs/server/MySQLHandler.cpp b/dbms/programs/server/MySQLHandler.cpp index aaa8cec90b2..3e1432dbfce 100644 --- a/dbms/programs/server/MySQLHandler.cpp +++ b/dbms/programs/server/MySQLHandler.cpp @@ -252,8 +252,8 @@ void MySQLHandler::comFieldList(ReadBuffer & payload) ComFieldList packet; packet.readPayload(payload); String database = connection_context.getCurrentDatabase(); - StoragePtr tablePtr = DatabaseCatalog::instance().getTable({database, packet.table}); - for (const NameAndTypePair & column: tablePtr->getColumns().getAll()) + StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}); + for (const NameAndTypePair & column: table_ptr->getColumns().getAll()) { ColumnDefinition column_definition( database, packet.table, packet.table, column.name, column.name, CharacterSet::binary, 100, ColumnType::MYSQL_TYPE_STRING, 0, 0 diff --git a/dbms/programs/server/MySQLHandlerFactory.cpp b/dbms/programs/server/MySQLHandlerFactory.cpp index dceb3d2293e..925a1e0af5f 100644 --- a/dbms/programs/server/MySQLHandlerFactory.cpp +++ b/dbms/programs/server/MySQLHandlerFactory.cpp @@ -56,26 +56,26 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_) void MySQLHandlerFactory::readRSAKeys() { const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config(); - String certificateFileProperty = "openSSL.server.certificateFile"; - String privateKeyFileProperty = "openSSL.server.privateKeyFile"; + String certificate_file_property = "openSSL.server.certificateFile"; + String private_key_file_property = "openSSL.server.privateKeyFile"; - if (!config.has(certificateFileProperty)) + if (!config.has(certificate_file_property)) throw Exception("Certificate file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); - if (!config.has(privateKeyFileProperty)) + if (!config.has(private_key_file_property)) throw Exception("Private key file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); { - String certificateFile = config.getString(certificateFileProperty); - FILE * fp = fopen(certificateFile.data(), "r"); + String certificate_file = config.getString(certificate_file_property); + FILE * fp = fopen(certificate_file.data(), "r"); if (fp == nullptr) - throw Exception("Cannot open certificate file: " + certificateFile + ".", ErrorCodes::CANNOT_OPEN_FILE); + throw Exception("Cannot open certificate file: " + certificate_file + ".", ErrorCodes::CANNOT_OPEN_FILE); SCOPE_EXIT(fclose(fp)); X509 * x509 = PEM_read_X509(fp, nullptr, nullptr, nullptr); SCOPE_EXIT(X509_free(x509)); if (x509 == nullptr) - throw Exception("Failed to read PEM certificate from " + certificateFile + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); + throw Exception("Failed to read PEM certificate from " + certificate_file + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); EVP_PKEY * p = X509_get_pubkey(x509); if (p == nullptr) @@ -88,16 +88,16 @@ void MySQLHandlerFactory::readRSAKeys() } { - String privateKeyFile = config.getString(privateKeyFileProperty); + String private_key_file = config.getString(private_key_file_property); - FILE * fp = fopen(privateKeyFile.data(), "r"); + FILE * fp = fopen(private_key_file.data(), "r"); if (fp == nullptr) - throw Exception ("Cannot open private key file " + privateKeyFile + ".", ErrorCodes::CANNOT_OPEN_FILE); + throw Exception ("Cannot open private key file " + private_key_file + ".", ErrorCodes::CANNOT_OPEN_FILE); SCOPE_EXIT(fclose(fp)); private_key.reset(PEM_read_RSAPrivateKey(fp, nullptr, nullptr, nullptr)); if (!private_key) - throw Exception("Failed to read RSA private key from " + privateKeyFile + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); + throw Exception("Failed to read RSA private key from " + private_key_file + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); } } diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 96ba2883480..5322514e7c2 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -162,12 +162,12 @@ int Server::run() { if (config().hasOption("help")) { - Poco::Util::HelpFormatter helpFormatter(Server::options()); + Poco::Util::HelpFormatter help_formatter(Server::options()); std::stringstream header; header << commandName() << " [OPTION] [-- [ARG]...]\n"; header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010"; - helpFormatter.setHeader(header.str()); - helpFormatter.format(std::cout); + help_formatter.setHeader(header.str()); + help_formatter.format(std::cout); return 0; } if (config().hasOption("version")) diff --git a/dbms/src/Access/QuotaCache.cpp b/dbms/src/Access/QuotaCache.cpp index cdf298d0e57..6db3eb66c5d 100644 --- a/dbms/src/Access/QuotaCache.cpp +++ b/dbms/src/Access/QuotaCache.cpp @@ -101,7 +101,7 @@ boost::shared_ptr QuotaCache::QuotaInfo::rebuildI new_intervals->quota_key = key; auto & intervals = new_intervals->intervals; intervals.reserve(quota->all_limits.size()); - constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; + static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; for (const auto & limits : quota->all_limits) { intervals.emplace_back(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp index 52fe70a30b5..31a921c3b2c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp @@ -20,32 +20,32 @@ namespace ErrorCodes namespace { -template +template struct MovingSum { using DataType = MovingSumData; - using Function = MovingImpl; + using Function = MovingImpl; }; -template +template struct MovingAvg { using DataType = MovingAvgData; - using Function = MovingImpl; + using Function = MovingImpl; }; -template using MovingSumTemplate = typename MovingSum::Function; -template using MovingAvgTemplate = typename MovingAvg::Function; +template using MovingSumTemplate = typename MovingSum::Function; +template using MovingAvgTemplate = typename MovingAvg::Function; -template