Don't access static members through instance, pt. II

clang-tidy rightfully complains
(-readability-static-accessed-through-instance) that static member
functions are accessed through non-static members
This commit is contained in:
Robert Schulze 2024-04-07 09:51:45 +00:00
parent 633aeaaa76
commit 97281203b8
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
36 changed files with 88 additions and 78 deletions

View File

@ -119,7 +119,6 @@ Checks: [
'-readability-named-parameter',
'-readability-redundant-declaration',
'-readability-simplify-boolean-expr',
'-readability-static-accessed-through-instance',
'-readability-suspicious-call-argument',
'-readability-uppercase-literal-suffix',
'-readability-use-anyofallof',

View File

@ -166,7 +166,7 @@ int DisksApp::main(const std::vector<String> & /*args*/)
{
String config_path = config().getString("config-file", getDefaultConfigFileName());
ConfigProcessor config_processor(config_path, false, false);
config_processor.setConfigPath(fs::path(config_path).parent_path());
ConfigProcessor::setConfigPath(fs::path(config_path).parent_path());
auto loaded_config = config_processor.loadConfig();
config().add(loaded_config.configuration.duplicate(), false, false);
}

View File

@ -368,7 +368,7 @@ int KeeperClient::main(const std::vector<String> & /* args */)
DB::ConfigProcessor config_processor(config().getString("config-file", "config.xml"));
/// This will handle a situation when clickhouse is running on the embedded config, but config.d folder is also present.
config_processor.registerEmbeddedConfig("config.xml", "<clickhouse/>");
ConfigProcessor::registerEmbeddedConfig("config.xml", "<clickhouse/>");
auto clickhouse_config = config_processor.loadConfig();
Poco::Util::AbstractConfiguration::Keys keys;

View File

@ -122,7 +122,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
{
const auto config_path = config().getString("config-file", "config.xml");
ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(fs::path(config_path).parent_path());
ConfigProcessor::setConfigPath(fs::path(config_path).parent_path());
auto loaded_config = config_processor.loadConfig();
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
}

View File

@ -54,30 +54,30 @@ public:
{
const auto & value = columns[0]->getFloat64(row_num);
const auto & time = columns[1]->getFloat64(row_num);
this->data(place).add(value, time, half_decay);
data(place).add(value, time, half_decay);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), half_decay);
data(place).merge(data(rhs), half_decay);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinary(this->data(place).value, buf);
writeBinary(this->data(place).time, buf);
writeBinary(data(place).value, buf);
writeBinary(data(place).time, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinary(this->data(place).value, buf);
readBinary(this->data(place).time, buf);
readBinary(data(place).value, buf);
readBinary(data(place).time, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & column = assert_cast<ColumnVector<Float64> &>(to);
column.getData().push_back(this->data(place).get(half_decay));
column.getData().push_back(data(place).get(half_decay));
}
};

View File

@ -293,32 +293,32 @@ public:
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value, arena);
data(place).addY(value, arena);
else
this->data(place).addX(value, arena);
data(place).addX(value, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).merge(this->data(rhs), arena);
data(place).merge(data(rhs), arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if (!this->data(place).size_x || !this->data(place).size_y)
if (!data(place).size_x || !data(place).size_y)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());
auto [d_statistic, p_value] = this->data(place).getResult(alternative, method);
auto [d_statistic, p_value] = data(place).getResult(alternative, method);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));

View File

@ -147,6 +147,8 @@ public:
negative_store->merge(other.negative_store.get());
}
/// NOLINTBEGIN(readability-static-accessed-through-instance)
void serialize(WriteBuffer& buf) const
{
// Write the mapping
@ -201,6 +203,8 @@ public:
count = static_cast<Float64>(negative_store->count + zero_count + store->count);
}
/// NOLINTEND(readability-static-accessed-through-instance)
private:
std::unique_ptr<DDSketchLogarithmicMapping> mapping;
std::unique_ptr<DDSketchDenseStore> store;

View File

@ -87,6 +87,8 @@ public:
count += other->count;
}
/// NOLINTBEGIN(readability-static-accessed-through-instance)
void serialize(WriteBuffer& buf) const
{
@ -179,6 +181,8 @@ public:
}
}
/// NOLINTEND(readability-static-accessed-through-instance)
private:
UInt32 chunk_size;
DDSketchEncoding enc;

View File

@ -5793,7 +5793,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
return result_projection_names;
}
FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters);
FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); /// NOLINT(readability-static-accessed-through-instance)
bool is_executable_udf = true;
IdentifierResolveScope::ResolvedFunctionsCache * function_cache = nullptr;
@ -5823,7 +5823,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
{
std::vector<std::string> possible_function_names;
auto function_names = UserDefinedExecutableFunctionFactory::instance().getRegisteredNames(scope.context);
auto function_names = UserDefinedExecutableFunctionFactory::instance().getRegisteredNames(scope.context); /// NOLINT(readability-static-accessed-through-instance)
possible_function_names.insert(possible_function_names.end(), function_names.begin(), function_names.end());
function_names = UserDefinedSQLFunctionFactory::instance().getAllRegisteredNames();
@ -5841,8 +5841,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
possible_function_names.push_back(name);
}
NamePrompter<2> name_prompter;
auto hints = name_prompter.getHints(function_name, possible_function_names);
auto hints = NamePrompter<2>::getHints(function_name, possible_function_names);
throw Exception(ErrorCodes::UNKNOWN_FUNCTION,
"Function with name '{}' does not exists. In scope {}{}",

View File

@ -141,7 +141,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
async_callback(socket->impl()->sockfd(), connection_timeout, AsyncEventTimeoutType::CONNECT, description, AsyncTaskExecutor::READ | AsyncTaskExecutor::WRITE | AsyncTaskExecutor::ERROR);
if (auto err = socket->impl()->socketError())
socket->impl()->error(err); // Throws an exception
socket->impl()->error(err); // Throws an exception /// NOLINT(readability-static-accessed-through-instance)
socket->setBlocking(true);
}

View File

@ -940,7 +940,7 @@ void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo &
if (nested_node)
{
/// Find any leaf of Nested subcolumn.
const auto * leaf = subcolumns.findLeaf(nested_node, [&](const auto &) { return true; });
const auto * leaf = Subcolumns::findLeaf(nested_node, [&](const auto &) { return true; });
assert(leaf);
/// Recreate subcolumn with default values and the same sizes of arrays.
@ -983,7 +983,7 @@ const ColumnObject::Subcolumns::Node * ColumnObject::getLeafOfTheSameNested(cons
while (current_node)
{
/// Try to find the first Nested up to the current node.
const auto * node_nested = subcolumns.findParent(current_node,
const auto * node_nested = Subcolumns::findParent(current_node,
[](const auto & candidate) { return candidate.isNested(); });
if (!node_nested)
@ -993,7 +993,7 @@ const ColumnObject::Subcolumns::Node * ColumnObject::getLeafOfTheSameNested(cons
/// for the last rows.
/// If there are no leaves, skip current node and find
/// the next node up to the current.
leaf = subcolumns.findLeaf(node_nested,
leaf = Subcolumns::findLeaf(node_nested,
[&](const auto & candidate)
{
return candidate.data.size() > old_size;

View File

@ -483,7 +483,7 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe
ASSERT_TRUE(EqualByteContainers(test_sequence.data_type->getSizeOfValueInMemory(), source_data, decoded));
const auto header_size = codec.getHeaderSize();
const auto header_size = ICompressionCodec::getHeaderSize();
const auto compression_ratio = (encoded_size - header_size) / (source_data.size() * 1.0);
if (expected_compression_ratio)

View File

@ -665,7 +665,7 @@ void BaseDaemon::reloadConfiguration()
*/
config_path = config().getString("config-file", getDefaultConfigFileName());
ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(fs::path(config_path).parent_path());
ConfigProcessor::setConfigPath(fs::path(config_path).parent_path());
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (last_configuration != nullptr)

View File

@ -18,7 +18,7 @@ SerializationPtr DataTypeDate32::doGetDefaultSerialization() const
Field DataTypeDate32::getDefault() const
{
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch());
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance)
}
void registerDataTypeDate32(DataTypeFactory & factory)

View File

@ -34,7 +34,7 @@ TEST(JSONDataParser, ReadJSON)
JSONDataParser<SimdJSONParser> parser;
ReadBufferFromString buf(json_bad);
String res;
parser.readJSON(res, buf);
JSONDataParser<SimdJSONParser>::readJSON(res, buf);
ASSERT_EQ(json1, res);
}
@ -44,7 +44,7 @@ TEST(JSONDataParser, ReadJSON)
JSONDataParser<SimdJSONParser> parser;
ReadBufferFromString buf(json_bad);
String res;
parser.readJSON(res, buf);
JSONDataParser<SimdJSONParser>::readJSON(res, buf);
ASSERT_EQ(json2, res);
}
}

View File

@ -346,7 +346,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_s
}
auto downloader_id = file_segment.getOrSetDownloader();
if (downloader_id == file_segment.getCallerId())
if (downloader_id == FileSegment::getCallerId())
{
if (canStartFromCache(file_offset_of_buffer_end, file_segment))
{

View File

@ -279,7 +279,7 @@ struct ToDate32Transform32Or64Signed
static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl & time_zone)
{
static const Int32 daynum_min_offset = -static_cast<Int32>(time_zone.getDayNumOffsetEpoch());
static const Int32 daynum_min_offset = -static_cast<Int32>(DateLUTImpl::getDayNumOffsetEpoch());
if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw)
{
@ -1092,7 +1092,7 @@ struct ConvertThroughParsing
{
if constexpr (std::is_same_v<ToDataType, DataTypeDate32>)
{
vec_to[i] = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch());
vec_to[i] = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance)
}
else
{

View File

@ -106,7 +106,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeRegistered(const ContextPtr & cont
if (AggregateFunctionFactory::instance().hasNameOrAlias(function_name))
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The aggregate function '{}' already exists", function_name);
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context))
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) /// NOLINT(readability-static-accessed-through-instance)
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "User defined executable function '{}' already exists", function_name);
validateFunction(assert_cast<const ASTCreateFunctionQuery &>(create_function_query).function_core, function_name);
@ -118,7 +118,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeUnregistered(const ContextPtr & co
AggregateFunctionFactory::instance().hasNameOrAlias(function_name))
throw Exception(ErrorCodes::CANNOT_DROP_FUNCTION, "Cannot drop system function '{}'", function_name);
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context))
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) /// NOLINT(readability-static-accessed-through-instance)
throw Exception(ErrorCodes::CANNOT_DROP_FUNCTION, "Cannot drop user defined executable function '{}'", function_name);
}

View File

@ -37,7 +37,7 @@ MMapReadBufferFromFileWithCache::MMapReadBufferFromFileWithCache(
MMapReadBufferFromFileWithCache::MMapReadBufferFromFileWithCache(
MMappedFileCache & cache, const std::string & file_name, size_t offset)
{
mapped = cache.getOrSet(cache.hash(file_name, offset, -1), [&]
mapped = cache.getOrSet(MMappedFileCache::hash(file_name, offset, -1), [&]
{
return std::make_shared<MMappedFile>(file_name, offset);
});

View File

@ -1071,7 +1071,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
auto current_context = data.getContext();
if (UserDefinedExecutableFunctionFactory::instance().has(node.name, current_context))
if (UserDefinedExecutableFunctionFactory::instance().has(node.name, current_context)) /// NOLINT(readability-static-accessed-through-instance)
{
Array parameters;
if (node.parameters)
@ -1087,7 +1087,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
}
}
function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(node.name, current_context, parameters);
function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(node.name, current_context, parameters); /// NOLINT(readability-static-accessed-through-instance)
}
if (!function_builder)

View File

@ -1056,7 +1056,7 @@ void NO_INLINE Aggregator::executeImplBatch(
/// During processing of row #i we will prefetch HashTable cell for row #(i + prefetch_look_ahead).
PrefetchingHelper prefetching;
size_t prefetch_look_ahead = prefetching.getInitialLookAheadValue();
size_t prefetch_look_ahead = PrefetchingHelper::getInitialLookAheadValue();
/// Optimization for special case when there are no aggregate functions.
if (params.aggregates_size == 0)
@ -1077,7 +1077,7 @@ void NO_INLINE Aggregator::executeImplBatch(
{
if constexpr (prefetch && HasPrefetchMemberFunc<decltype(method.data), KeyHolder>)
{
if (i == row_begin + prefetching.iterationsToMeasure())
if (i == row_begin + PrefetchingHelper::iterationsToMeasure())
prefetch_look_ahead = prefetching.calcPrefetchLookAhead();
if (i + prefetch_look_ahead < row_end)
@ -1163,7 +1163,7 @@ void NO_INLINE Aggregator::executeImplBatch(
if constexpr (prefetch && HasPrefetchMemberFunc<decltype(method.data), KeyHolder>)
{
if (i == key_start + prefetching.iterationsToMeasure())
if (i == key_start + PrefetchingHelper::iterationsToMeasure())
prefetch_look_ahead = prefetching.calcPrefetchLookAhead();
if (i + prefetch_look_ahead < row_end)

View File

@ -56,7 +56,7 @@ InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextP
BlockIO InterpreterAlterQuery::execute()
{
FunctionNameNormalizer().visit(query_ptr.get());
FunctionNameNormalizer::visit(query_ptr.get());
const auto & alter = query_ptr->as<ASTAlterQuery &>();
if (alter.alter_object == ASTAlterQuery::AlterObjectType::DATABASE)
{
@ -131,7 +131,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
if (modify_query)
{
// Expand CTE before filling default database
ApplyWithSubqueryVisitor().visit(*modify_query);
ApplyWithSubqueryVisitor::visit(*modify_query);
}
/// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc.

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
BlockIO InterpreterCreateIndexQuery::execute()
{
FunctionNameNormalizer().visit(query_ptr.get());
FunctionNameNormalizer::visit(query_ptr.get());
auto current_context = getContext();
const auto & create_index = query_ptr->as<ASTCreateIndexQuery &>();

View File

@ -1114,7 +1114,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
// Table SQL definition is available even if the table is detached (even permanently)
auto query = database->getCreateTableQuery(create.getTable(), getContext());
FunctionNameNormalizer().visit(query.get());
FunctionNameNormalizer::visit(query.get());
auto create_query = query->as<ASTCreateQuery &>();
if (!create.is_dictionary && create_query.is_dictionary)
@ -1184,7 +1184,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (create.select && create.isView())
{
// Expand CTE before filling default database
ApplyWithSubqueryVisitor().visit(*create.select);
ApplyWithSubqueryVisitor::visit(*create.select);
AddDefaultDatabaseVisitor visitor(getContext(), current_database);
visitor.visit(*create.select);
}
@ -1763,7 +1763,7 @@ BlockIO InterpreterCreateQuery::executeQueryOnCluster(ASTCreateQuery & create)
BlockIO InterpreterCreateQuery::execute()
{
FunctionNameNormalizer().visit(query_ptr.get());
FunctionNameNormalizer::visit(query_ptr.get());
auto & create = query_ptr->as<ASTCreateQuery &>();
bool is_create_database = create.database && !create.table;

View File

@ -168,7 +168,7 @@ void Set::setHeader(const ColumnsWithTypeAndName & header)
}
/// Choose data structure to use for the set.
data.init(data.chooseMethod(key_columns, key_sizes));
data.init(SetVariants::chooseMethod(key_columns, key_sizes));
}
void Set::fillSetElements()

View File

@ -144,7 +144,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context)
}
else
{
FunctionOverloadResolverPtr function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(function->name, context);
FunctionOverloadResolverPtr function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(function->name, context); /// NOLINT(readability-static-accessed-through-instance)
if (!function_builder)
function_builder = function_factory.get(function->name, context);

View File

@ -923,7 +923,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Propagate WITH statement to children ASTSelect.
if (settings.enable_global_with_statement)
{
ApplyWithGlobalVisitor().visit(ast);
ApplyWithGlobalVisitor::visit(ast);
}
{

View File

@ -364,7 +364,7 @@ TEST_F(FileCacheTest, LRUPolicy)
std::cerr << "Step 1\n";
auto cache = DB::FileCache("1", settings);
cache.initialize();
auto key = cache.createKeyForPath("key1");
auto key = DB::FileCache::createKeyForPath("key1");
auto get_or_set = [&](size_t offset, size_t size)
{
@ -728,7 +728,7 @@ TEST_F(FileCacheTest, LRUPolicy)
auto cache2 = DB::FileCache("2", settings);
cache2.initialize();
auto key = cache2.createKeyForPath("key1");
auto key = DB::FileCache::createKeyForPath("key1");
/// Get [2, 29]
assertEqual(
@ -747,7 +747,7 @@ TEST_F(FileCacheTest, LRUPolicy)
fs::create_directories(settings2.base_path);
auto cache2 = DB::FileCache("3", settings2);
cache2.initialize();
auto key = cache2.createKeyForPath("key1");
auto key = DB::FileCache::createKeyForPath("key1");
/// Get [0, 24]
assertEqual(
@ -762,7 +762,7 @@ TEST_F(FileCacheTest, LRUPolicy)
auto cache = FileCache("4", settings);
cache.initialize();
const auto key = cache.createKeyForPath("key10");
const auto key = FileCache::createKeyForPath("key10");
const auto key_path = cache.getKeyPath(key, user);
cache.removeAllReleasable(user.user_id);
@ -786,7 +786,7 @@ TEST_F(FileCacheTest, LRUPolicy)
auto cache = DB::FileCache("5", settings);
cache.initialize();
const auto key = cache.createKeyForPath("key10");
const auto key = FileCache::createKeyForPath("key10");
const auto key_path = cache.getKeyPath(key, user);
cache.removeAllReleasable(user.user_id);
@ -823,7 +823,7 @@ TEST_F(FileCacheTest, writeBuffer)
segment_settings.kind = FileSegmentKind::Temporary;
segment_settings.unbounded = true;
auto cache_key = cache.createKeyForPath(key);
auto cache_key = FileCache::createKeyForPath(key);
auto holder = cache.set(cache_key, 0, 3, segment_settings, user);
/// The same is done in TemporaryDataOnDisk::createStreamToCacheFile.
std::filesystem::create_directories(cache.getKeyPath(cache_key, user));
@ -949,7 +949,7 @@ TEST_F(FileCacheTest, temporaryData)
const auto user = FileCache::getCommonUser();
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(nullptr, &file_cache, TemporaryDataOnDiskSettings{});
auto some_data_holder = file_cache.getOrSet(file_cache.createKeyForPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user);
auto some_data_holder = file_cache.getOrSet(FileCache::createKeyForPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user);
{
ASSERT_EQ(some_data_holder->size(), 5);
@ -1199,7 +1199,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
{
auto cache = DB::FileCache(std::to_string(++file_cache_name), settings);
cache.initialize();
auto key = cache.createKeyForPath("key1");
auto key = FileCache::createKeyForPath("key1");
auto add_range = [&](size_t offset, size_t size)
{

View File

@ -53,6 +53,8 @@ static std::string renderFileNameTemplate(time_t now, const std::string & file_p
return path.replace_filename(ss.str());
}
/// NOLINTBEGIN(readability-static-accessed-through-instance)
void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name)
{
auto current_logger = config.getString("logger", "");
@ -393,6 +395,8 @@ void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Log
}
}
/// NOLINTEND(readability-static-accessed-through-instance)
void Loggers::closeLogs(Poco::Logger & logger)
{
if (log_file)

View File

@ -86,7 +86,7 @@ static const Graphite::Pattern undef_pattern =
.regexp_str = "",
.function = nullptr,
.retentions = Graphite::Retentions(),
.type = undef_pattern.TypeUndef,
.type = Graphite::Pattern::TypeUndef,
};
inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, std::string_view path)
@ -118,18 +118,18 @@ Graphite::RollupRule selectPatternForPath(
if (!pattern.regexp)
{
/// Default pattern
if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll)
if (first_match->type == Graphite::Pattern::TypeUndef && pattern.type == Graphite::Pattern::TypeAll)
{
/// There is only default pattern for both retention and aggregation
return {&pattern, &pattern};
}
if (pattern.type != first_match->type)
{
if (first_match->type == first_match->TypeRetention)
if (first_match->type == Graphite::Pattern::TypeRetention)
{
return {first_match, &pattern};
}
if (first_match->type == first_match->TypeAggregation)
if (first_match->type == Graphite::Pattern::TypeAggregation)
{
return {&pattern, first_match};
}
@ -140,23 +140,23 @@ Graphite::RollupRule selectPatternForPath(
if (pattern.regexp->match(path.data(), path.size()))
{
/// General pattern with matched path
if (pattern.type == pattern.TypeAll)
if (pattern.type == Graphite::Pattern::TypeAll)
{
/// Only for not default patterns with both function and retention parameters
return {&pattern, &pattern};
}
if (first_match->type == first_match->TypeUndef)
if (first_match->type == Graphite::Pattern::TypeUndef)
{
first_match = &pattern;
continue;
}
if (pattern.type != first_match->type)
{
if (first_match->type == first_match->TypeRetention)
if (first_match->type == Graphite::Pattern::TypeRetention)
{
return {first_match, &pattern};
}
if (first_match->type == first_match->TypeAggregation)
if (first_match->type == Graphite::Pattern::TypeAggregation)
{
return {&pattern, first_match};
}
@ -415,24 +415,24 @@ static const Pattern & appendGraphitePattern(
if (!pattern.function)
{
pattern.type = pattern.TypeRetention;
pattern.type = Graphite::Pattern::TypeRetention;
}
else if (pattern.retentions.empty())
{
pattern.type = pattern.TypeAggregation;
pattern.type = Graphite::Pattern::TypeAggregation;
}
else
{
pattern.type = pattern.TypeAll;
pattern.type = Graphite::Pattern::TypeAll;
}
if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll
if (pattern.type & Graphite::Pattern::TypeAggregation) /// TypeAggregation or TypeAll
if (pattern.function->allocatesMemoryInArena())
throw Exception(DB::ErrorCodes::NOT_IMPLEMENTED,
"Aggregate function {} isn't supported in GraphiteMergeTree", pattern.function->getName());
/// retention should be in descending order of age.
if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll
if (pattern.type & Graphite::Pattern::TypeRetention) /// TypeRetention or TypeAll
::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions);
patterns.emplace_back(pattern);

View File

@ -467,7 +467,7 @@ void StorageFileLog::openFilesAndSetPos()
auto & reader = file_ctx.reader.value();
assertStreamGood(reader);
reader.seekg(0, reader.end);
reader.seekg(0, reader.end); /// NOLINT(readability-static-accessed-through-instance)
assertStreamGood(reader);
auto file_end = reader.tellg();

View File

@ -2994,7 +2994,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
commands.apply(new_metadata, local_context);
if (commands.hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index)
if (AlterCommands::hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')");

View File

@ -116,7 +116,7 @@ StorageDictionary::StorageDictionary(
: StorageDictionary(
table_id,
table_id.getFullNameNotQuoted(),
context_->getExternalDictionariesLoader().getDictionaryStructure(*dictionary_configuration),
context_->getExternalDictionariesLoader().getDictionaryStructure(*dictionary_configuration), /// NOLINT(readability-static-accessed-through-instance)
dictionary_configuration->getString("dictionary.comment", ""),
Location::SameDatabaseAndNameAsDictionary,
context_)

View File

@ -437,7 +437,7 @@ void fuzzJSONObject(
bool first = true;
for (const auto & ptr : node_list)
{
if (node_count >= config.value_number_limit)
if (node_count >= StorageFuzzJSON::Configuration::value_number_limit)
break;
WriteBufferFromOwnString child_out;

View File

@ -179,7 +179,7 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c
}
const auto & user_defined_executable_functions_factory = UserDefinedExecutableFunctionFactory::instance();
const auto & user_defined_executable_functions_names = user_defined_executable_functions_factory.getRegisteredNames(context);
const auto & user_defined_executable_functions_names = user_defined_executable_functions_factory.getRegisteredNames(context); /// NOLINT(readability-static-accessed-through-instance)
for (const auto & function_name : user_defined_executable_functions_names)
{
fillRow(res_columns, function_name, 0, {0}, "", FunctionOrigin::EXECUTABLE_USER_DEFINED, user_defined_executable_functions_factory);

View File

@ -136,7 +136,7 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt
ColumnsDescription TableFunctionExplain::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const
{
Block sample_block = getInterpreter(context).getSampleBlock(query->as<ASTExplainQuery>()->getKind());
Block sample_block = getInterpreter(context).getSampleBlock(query->as<ASTExplainQuery>()->getKind()); /// NOLINT(readability-static-accessed-through-instance)
ColumnsDescription columns_description;
for (const auto & column : sample_block.getColumnsWithTypeAndName())
columns_description.add(ColumnDescription(column.name, column.type));