mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
More fixes
This commit is contained in:
parent
de2a0be025
commit
9626506585
@ -592,7 +592,7 @@ String RecalculateCommand::run()
|
||||
|
||||
String CleanResourcesCommand::run()
|
||||
{
|
||||
keeper_dispatcher.cleanResources();
|
||||
KeeperDispatcher::cleanResources();
|
||||
return "ok";
|
||||
}
|
||||
|
||||
|
@ -86,7 +86,7 @@ namespace
|
||||
auto & res = typeid_cast<ASTCreateFunctionQuery &>(*ptr);
|
||||
res.if_not_exists = false;
|
||||
res.or_replace = false;
|
||||
FunctionNameNormalizer().visit(res.function_core.get());
|
||||
FunctionNameNormalizer::visit(res.function_core.get());
|
||||
return ptr;
|
||||
}
|
||||
}
|
||||
|
@ -108,7 +108,7 @@ namespace
|
||||
// null terminate the sequence
|
||||
seq.push_back('\0');
|
||||
// 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
|
||||
seq.clear();
|
||||
if (res)
|
||||
|
@ -82,7 +82,7 @@ public:
|
||||
|
||||
std::string getContentType() const override
|
||||
{
|
||||
return writer.getContentType();
|
||||
return NativeWriter::getContentType();
|
||||
}
|
||||
|
||||
protected:
|
||||
|
@ -76,7 +76,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param
|
||||
if (params.patterns_typed)
|
||||
{
|
||||
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;
|
||||
else
|
||||
return params.patterns_tagged;
|
||||
|
@ -31,7 +31,7 @@ std::vector<std::pair<String, String>> describeJoinActions(const JoinPtr & join)
|
||||
description.emplace_back("ASOF inequality", toString(table_join.getAsofInequality()));
|
||||
|
||||
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;
|
||||
}
|
||||
|
@ -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());
|
||||
|
||||
return storage.createSelector(cluster, key_column);
|
||||
return StorageDistributed::createSelector(cluster, key_column);
|
||||
}
|
||||
|
||||
|
||||
|
@ -1964,8 +1964,8 @@ KeyCondition::Description KeyCondition::getDescription() const
|
||||
/// This means that logical NOT is applied to leaf.
|
||||
bool negate = false;
|
||||
|
||||
std::unique_ptr<Node> left;
|
||||
std::unique_ptr<Node> right;
|
||||
std::unique_ptr<Node> left = nullptr;
|
||||
std::unique_ptr<Node> right = nullptr;
|
||||
};
|
||||
|
||||
/// The algorithm is the same as in KeyCondition::checkInHyperrectangle
|
||||
|
@ -426,7 +426,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite
|
||||
ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
|
||||
|
||||
write_part_log({});
|
||||
storage.incrementMergedPartsProfileEvent(part->getType());
|
||||
StorageReplicatedMergeTree::incrementMergedPartsProfileEvent(part->getType());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -149,7 +149,7 @@ void MergePlainMergeTreeTask::finish()
|
||||
ThreadFuzzer::maybeInjectMemoryLimitException();
|
||||
|
||||
write_part_log({});
|
||||
storage.incrementMergedPartsProfileEvent(new_part->getType());
|
||||
StorageMergeTree::incrementMergedPartsProfileEvent(new_part->getType());
|
||||
transfer_profile_counters_to_initial_query();
|
||||
|
||||
if (auto txn_ = txn_holder.getTransaction())
|
||||
|
@ -590,7 +590,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals(
|
||||
|
||||
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;
|
||||
|
||||
auto converted = convertFieldToType(f, *actual_type);
|
||||
|
@ -210,7 +210,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksSync()
|
||||
|
||||
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)
|
||||
{
|
||||
auto callback = [this] { return loadMarksImpl(); };
|
||||
|
@ -158,7 +158,7 @@ bool MergeTreePartsMover::selectPartsForMove(
|
||||
{
|
||||
auto destination = data->getDestinationForMoveTTL(*ttl_entry);
|
||||
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.
|
||||
|
@ -230,7 +230,7 @@ void MergeTreeColumnSettings::validate(const SettingsChanges & changes)
|
||||
"Setting {} is unknown or not supported at column level, supported settings: {}",
|
||||
change.name,
|
||||
fmt::join(allowed_column_level_settings, ", "));
|
||||
merge_tree_settings.checkCanSet(change.name, change.value);
|
||||
MergeTreeSettings::checkCanSet(change.name, change.value);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -63,7 +63,7 @@ void MergeTreeSink::consume(Chunk chunk)
|
||||
if (!storage_snapshot->object_columns.empty())
|
||||
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>;
|
||||
DelayedPartitions partitions;
|
||||
|
@ -449,7 +449,7 @@ void ReplicatedMergeTreeSinkImpl<false>::finishDelayedChunk(const ZooKeeperWithF
|
||||
int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0;
|
||||
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));
|
||||
storage.incrementInsertedPartsProfileEvent(part->getType());
|
||||
StorageReplicatedMergeTree::incrementInsertedPartsProfileEvent(part->getType());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -43,7 +43,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
||||
{
|
||||
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];
|
||||
}
|
||||
|
||||
|
@ -364,7 +364,7 @@ JSONNode & fuzzSingleJSONNode(JSONNode & n, const StorageFuzzJSON::Configuration
|
||||
|
||||
if (val.fixed)
|
||||
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())
|
||||
val.array->push_back(generateRandomJSONNode(config, rnd, /*with_key*/ false, depth));
|
||||
@ -377,7 +377,7 @@ JSONNode & fuzzSingleJSONNode(JSONNode & n, const StorageFuzzJSON::Configuration
|
||||
}
|
||||
++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));
|
||||
++node_count;
|
||||
@ -619,11 +619,11 @@ void StorageFuzzJSON::processNamedCollectionResult(Configuration & configuration
|
||||
{
|
||||
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(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"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"))
|
||||
@ -638,11 +638,11 @@ void StorageFuzzJSON::processNamedCollectionResult(Configuration & configuration
|
||||
if (collection.has("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(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"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);
|
||||
}
|
||||
@ -650,11 +650,11 @@ void StorageFuzzJSON::processNamedCollectionResult(Configuration & configuration
|
||||
if (collection.has("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(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"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.min_key_length = std::min(configuration.min_key_length, configuration.max_key_length);
|
||||
}
|
||||
|
@ -426,7 +426,7 @@ TTLTableDescription TTLTableDescription::parse(const String & str, const Columns
|
||||
|
||||
ParserTTLExpressionList parser;
|
||||
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);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user