More fixes

This commit is contained in:
Robert Schulze 2024-04-03 19:18:31 +00:00
parent de2a0be025
commit 9626506585
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
19 changed files with 27 additions and 27 deletions

View File

@ -592,7 +592,7 @@ String RecalculateCommand::run()
String CleanResourcesCommand::run() String CleanResourcesCommand::run()
{ {
keeper_dispatcher.cleanResources(); KeeperDispatcher::cleanResources();
return "ok"; return "ok";
} }

View File

@ -86,7 +86,7 @@ namespace
auto & res = typeid_cast<ASTCreateFunctionQuery &>(*ptr); auto & res = typeid_cast<ASTCreateFunctionQuery &>(*ptr);
res.if_not_exists = false; res.if_not_exists = false;
res.or_replace = false; res.or_replace = false;
FunctionNameNormalizer().visit(res.function_core.get()); FunctionNameNormalizer::visit(res.function_core.get());
return ptr; return ptr;
} }
} }

View File

@ -108,7 +108,7 @@ namespace
// null terminate the sequence // null terminate the sequence
seq.push_back('\0'); seq.push_back('\0');
// lookup the html sequence in the perfect hashmap. // lookup the html sequence in the perfect hashmap.
const auto * res = hash.Lookup(seq.data(), strlen(seq.data())); const auto * res = HTMLCharacterHash::Lookup(seq.data(), strlen(seq.data()));
// reset so that it's reused in the next iteration // reset so that it's reused in the next iteration
seq.clear(); seq.clear();
if (res) if (res)

View File

@ -82,7 +82,7 @@ public:
std::string getContentType() const override std::string getContentType() const override
{ {
return writer.getContentType(); return NativeWriter::getContentType();
} }
protected: protected:

View File

@ -76,7 +76,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param
if (params.patterns_typed) if (params.patterns_typed)
{ {
std::string_view path_view = path; std::string_view path_view = path;
if (path_view.find("?"sv) == path_view.npos) if (path_view.find("?"sv) == std::string::npos)
return params.patterns_plain; return params.patterns_plain;
else else
return params.patterns_tagged; return params.patterns_tagged;

View File

@ -31,7 +31,7 @@ std::vector<std::pair<String, String>> describeJoinActions(const JoinPtr & join)
description.emplace_back("ASOF inequality", toString(table_join.getAsofInequality())); description.emplace_back("ASOF inequality", toString(table_join.getAsofInequality()));
if (!table_join.getClauses().empty()) if (!table_join.getClauses().empty())
description.emplace_back("Clauses", table_join.formatClauses(table_join.getClauses(), true /*short_format*/)); description.emplace_back("Clauses", TableJoin::formatClauses(table_join.getClauses(), true /*short_format*/));
return description; return description;
} }

View File

@ -613,7 +613,7 @@ IColumn::Selector DistributedSink::createSelector(const Block & source_block) co
const auto & key_column = current_block_with_sharding_key_expr.getByName(storage.getShardingKeyColumnName()); const auto & key_column = current_block_with_sharding_key_expr.getByName(storage.getShardingKeyColumnName());
return storage.createSelector(cluster, key_column); return StorageDistributed::createSelector(cluster, key_column);
} }

View File

@ -1964,8 +1964,8 @@ KeyCondition::Description KeyCondition::getDescription() const
/// This means that logical NOT is applied to leaf. /// This means that logical NOT is applied to leaf.
bool negate = false; bool negate = false;
std::unique_ptr<Node> left; std::unique_ptr<Node> left = nullptr;
std::unique_ptr<Node> right; std::unique_ptr<Node> right = nullptr;
}; };
/// The algorithm is the same as in KeyCondition::checkInHyperrectangle /// The algorithm is the same as in KeyCondition::checkInHyperrectangle

View File

@ -426,7 +426,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite
ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges); ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
write_part_log({}); write_part_log({});
storage.incrementMergedPartsProfileEvent(part->getType()); StorageReplicatedMergeTree::incrementMergedPartsProfileEvent(part->getType());
return true; return true;
} }

View File

@ -149,7 +149,7 @@ void MergePlainMergeTreeTask::finish()
ThreadFuzzer::maybeInjectMemoryLimitException(); ThreadFuzzer::maybeInjectMemoryLimitException();
write_part_log({}); write_part_log({});
storage.incrementMergedPartsProfileEvent(new_part->getType()); StorageMergeTree::incrementMergedPartsProfileEvent(new_part->getType());
transfer_profile_counters_to_initial_query(); transfer_profile_counters_to_initial_query();
if (auto txn_ = txn_holder.getTransaction()) if (auto txn_ = txn_holder.getTransaction())

View File

@ -590,7 +590,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals(
for (const auto & f : value_field.get<Array>()) for (const auto & f : value_field.get<Array>())
{ {
if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) /// NOLINT(readability-static-accessed-through-instance)
return false; return false;
auto converted = convertFieldToType(f, *actual_type); auto converted = convertFieldToType(f, *actual_type);

View File

@ -210,7 +210,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksSync()
if (mark_cache) if (mark_cache)
{ {
auto key = mark_cache->hash(fs::path(data_part_storage->getFullPath()) / mrk_path); auto key = MarkCache::hash(fs::path(data_part_storage->getFullPath()) / mrk_path);
if (save_marks_in_cache) if (save_marks_in_cache)
{ {
auto callback = [this] { return loadMarksImpl(); }; auto callback = [this] { return loadMarksImpl(); };

View File

@ -158,7 +158,7 @@ bool MergeTreePartsMover::selectPartsForMove(
{ {
auto destination = data->getDestinationForMoveTTL(*ttl_entry); auto destination = data->getDestinationForMoveTTL(*ttl_entry);
if (destination && !data->isPartInTTLDestination(*ttl_entry, *part)) if (destination && !data->isPartInTTLDestination(*ttl_entry, *part))
reservation = data->tryReserveSpace(part->getBytesOnDisk(), data->getDestinationForMoveTTL(*ttl_entry)); reservation = MergeTreeData::tryReserveSpace(part->getBytesOnDisk(), data->getDestinationForMoveTTL(*ttl_entry));
} }
if (reservation) /// Found reservation by TTL rule. if (reservation) /// Found reservation by TTL rule.

View File

@ -230,7 +230,7 @@ void MergeTreeColumnSettings::validate(const SettingsChanges & changes)
"Setting {} is unknown or not supported at column level, supported settings: {}", "Setting {} is unknown or not supported at column level, supported settings: {}",
change.name, change.name,
fmt::join(allowed_column_level_settings, ", ")); fmt::join(allowed_column_level_settings, ", "));
merge_tree_settings.checkCanSet(change.name, change.value); MergeTreeSettings::checkCanSet(change.name, change.value);
} }
} }

View File

@ -63,7 +63,7 @@ void MergeTreeSink::consume(Chunk chunk)
if (!storage_snapshot->object_columns.empty()) if (!storage_snapshot->object_columns.empty())
convertDynamicColumnsToTuples(block, storage_snapshot); convertDynamicColumnsToTuples(block, storage_snapshot);
auto part_blocks = storage.writer.splitBlockIntoParts(std::move(block), max_parts_per_block, metadata_snapshot, context); auto part_blocks = MergeTreeDataWriter::splitBlockIntoParts(std::move(block), max_parts_per_block, metadata_snapshot, context);
using DelayedPartitions = std::vector<MergeTreeSink::DelayedChunk::Partition>; using DelayedPartitions = std::vector<MergeTreeSink::DelayedChunk::Partition>;
DelayedPartitions partitions; DelayedPartitions partitions;

View File

@ -449,7 +449,7 @@ void ReplicatedMergeTreeSinkImpl<false>::finishDelayedChunk(const ZooKeeperWithF
int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0;
auto counters_snapshot = std::make_shared<ProfileEvents::Counters::Snapshot>(partition.part_counters.getPartiallyAtomicSnapshot()); auto counters_snapshot = std::make_shared<ProfileEvents::Counters::Snapshot>(partition.part_counters.getPartiallyAtomicSnapshot());
PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error));
storage.incrementInsertedPartsProfileEvent(part->getType()); StorageReplicatedMergeTree::incrementInsertedPartsProfileEvent(part->getType());
} }
catch (...) catch (...)
{ {

View File

@ -43,7 +43,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
{ {
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{ {
auto minmax_idx_column_names = data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()); auto minmax_idx_column_names = MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->getPartitionKey());
date_column = minmax_idx_column_names[data.minmax_idx_date_column_pos]; date_column = minmax_idx_column_names[data.minmax_idx_date_column_pos];
} }

View File

@ -364,7 +364,7 @@ JSONNode & fuzzSingleJSONNode(JSONNode & n, const StorageFuzzJSON::Configuration
if (val.fixed) if (val.fixed)
val.fixed = generateRandomFixedValue(config, rnd); val.fixed = generateRandomFixedValue(config, rnd);
else if (val.array && val.array->size() < config.max_array_size && node_count + val.array->size() < config.value_number_limit) else if (val.array && val.array->size() < config.max_array_size && node_count + val.array->size() < StorageFuzzJSON::Configuration::value_number_limit)
{ {
if (val.array->empty()) if (val.array->empty())
val.array->push_back(generateRandomJSONNode(config, rnd, /*with_key*/ false, depth)); val.array->push_back(generateRandomJSONNode(config, rnd, /*with_key*/ false, depth));
@ -377,7 +377,7 @@ JSONNode & fuzzSingleJSONNode(JSONNode & n, const StorageFuzzJSON::Configuration
} }
++node_count; ++node_count;
} }
else if (val.object && val.object->size() < config.max_object_size && node_count + val.object->size() < config.value_number_limit) else if (val.object && val.object->size() < config.max_object_size && node_count + val.object->size() < StorageFuzzJSON::Configuration::value_number_limit)
{ {
val.object->push_back(generateRandomJSONNode(config, rnd, /*with_key*/ true, depth)); val.object->push_back(generateRandomJSONNode(config, rnd, /*with_key*/ true, depth));
++node_count; ++node_count;
@ -619,11 +619,11 @@ void StorageFuzzJSON::processNamedCollectionResult(Configuration & configuration
{ {
configuration.max_output_length = collection.get<UInt64>("max_output_length"); configuration.max_output_length = collection.get<UInt64>("max_output_length");
if (configuration.max_output_length < 2 || configuration.max_output_length > configuration.output_length_limit) if (configuration.max_output_length < 2 || configuration.max_output_length > StorageFuzzJSON::Configuration::output_length_limit)
throw Exception( throw Exception(
ErrorCodes::BAD_ARGUMENTS, ErrorCodes::BAD_ARGUMENTS,
"The value of the 'max_output_length' argument must be within the interval [2, {}.]", "The value of the 'max_output_length' argument must be within the interval [2, {}.]",
configuration.output_length_limit); StorageFuzzJSON::Configuration::output_length_limit);
} }
if (collection.has("max_nesting_level")) if (collection.has("max_nesting_level"))
@ -638,11 +638,11 @@ void StorageFuzzJSON::processNamedCollectionResult(Configuration & configuration
if (collection.has("max_string_value_length")) if (collection.has("max_string_value_length"))
{ {
auto max_string_value_length = collection.get<UInt64>("max_string_value_length"); auto max_string_value_length = collection.get<UInt64>("max_string_value_length");
if (max_string_value_length > configuration.output_length_limit) if (max_string_value_length > StorageFuzzJSON::Configuration::output_length_limit)
throw Exception( throw Exception(
ErrorCodes::BAD_ARGUMENTS, ErrorCodes::BAD_ARGUMENTS,
"The value of the 'max_string_value_length' argument must be at most {}.", "The value of the 'max_string_value_length' argument must be at most {}.",
configuration.output_length_limit); StorageFuzzJSON::Configuration::output_length_limit);
configuration.max_string_value_length = std::min(max_string_value_length, configuration.max_output_length); configuration.max_string_value_length = std::min(max_string_value_length, configuration.max_output_length);
} }
@ -650,11 +650,11 @@ void StorageFuzzJSON::processNamedCollectionResult(Configuration & configuration
if (collection.has("max_key_length")) if (collection.has("max_key_length"))
{ {
auto max_key_length = collection.get<UInt64>("max_key_length"); auto max_key_length = collection.get<UInt64>("max_key_length");
if (max_key_length > configuration.output_length_limit) if (max_key_length > StorageFuzzJSON::Configuration::output_length_limit)
throw Exception( throw Exception(
ErrorCodes::BAD_ARGUMENTS, ErrorCodes::BAD_ARGUMENTS,
"The value of the 'max_key_length' argument must be less or equal than {}.", "The value of the 'max_key_length' argument must be less or equal than {}.",
configuration.output_length_limit); StorageFuzzJSON::Configuration::output_length_limit);
configuration.max_key_length = std::min(max_key_length, configuration.max_output_length); configuration.max_key_length = std::min(max_key_length, configuration.max_output_length);
configuration.min_key_length = std::min(configuration.min_key_length, configuration.max_key_length); configuration.min_key_length = std::min(configuration.min_key_length, configuration.max_key_length);
} }

View File

@ -426,7 +426,7 @@ TTLTableDescription TTLTableDescription::parse(const String & str, const Columns
ParserTTLExpressionList parser; ParserTTLExpressionList parser;
ASTPtr ast = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); ASTPtr ast = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
FunctionNameNormalizer().visit(ast.get()); FunctionNameNormalizer::visit(ast.get());
return getTTLForTableFromAST(ast, columns, context, primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); return getTTLForTableFromAST(ast, columns, context, primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions);
} }