Merge pull request #53751 from Algunenano/clang18

Support clang-18 (Wmissing-field-initializers)
This commit is contained in:
alesapin 2023-08-25 23:27:06 +02:00 committed by GitHub
commit 0e601b856f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 86 additions and 63 deletions

View File

@ -100,8 +100,8 @@ private:
struct Constraint
{
SettingConstraintWritability writability = SettingConstraintWritability::WRITABLE;
Field min_value;
Field max_value;
Field min_value{};
Field max_value{};
bool operator ==(const Constraint & other) const;
bool operator !=(const Constraint & other) const { return !(*this == other); }

View File

@ -559,6 +559,7 @@ FieldInfo ColumnObject::Subcolumn::getFieldInfo() const
.have_nulls = base_type->isNullable(),
.need_convert = false,
.num_dimensions = least_common_type.getNumberOfDimensions(),
.need_fold_dimension = false,
};
}

View File

@ -182,6 +182,7 @@ AsyncLoader::AsyncLoader(std::vector<PoolInitializer> pool_initializers, bool lo
init.max_threads,
/* max_free_threads = */ 0,
init.max_threads),
.ready_queue = {},
.max_threads = init.max_threads
});
}

View File

@ -69,11 +69,11 @@ struct FunctionDocumentation
using Related = std::string;
Description description; /// E.g. "Returns the position (in bytes, starting at 1) of a substring needle in a string haystack."
Syntax syntax; /// E.g. "position(haystack, needle)"
Arguments arguments; /// E.g. ["haystack — String in which the search is performed. String.", "needle — Substring to be searched. String."]
ReturnedValue returned_value; /// E.g. "Starting position in bytes and counting from 1, if the substring was found."
Examples examples; ///
Categories categories; /// E.g. {"String Search"}
Syntax syntax = {}; /// E.g. "position(haystack, needle)"
Arguments arguments {}; /// E.g. ["haystack — String in which the search is performed. String.", "needle — Substring to be searched. String."]
ReturnedValue returned_value {};/// E.g. "Starting position in bytes and counting from 1, if the substring was found."
Examples examples {}; ///
Categories categories {}; /// E.g. {"String Search"}
std::string argumentsAsString() const;
std::string examplesAsString() const;

View File

@ -34,8 +34,8 @@ public:
StorageID table_id = StorageID::createEmpty();
bool ignore_unknown = false;
bool expand_special_macros_only = false;
std::optional<String> shard;
std::optional<String> replica;
std::optional<String> shard = {};
std::optional<String> replica = {};
/// Information about macro expansion
size_t level = 0;

View File

@ -863,7 +863,9 @@ void TestKeeper::reconfig(
.callback = [callback](const Response & response)
{
callback(dynamic_cast<const ReconfigResponse &>(response));
}
},
.watch = nullptr,
.time = {}
});
}

View File

@ -49,7 +49,7 @@ struct AsyncLoaderTest
}
explicit AsyncLoaderTest(size_t max_threads = 1)
: AsyncLoaderTest({{.max_threads = max_threads}})
: AsyncLoaderTest({{.max_threads = max_threads, .priority = {}}})
{}
std::vector<AsyncLoader::PoolInitializer> getPoolInitializers(std::vector<Initializer> initializers)

View File

@ -480,6 +480,7 @@ void KeeperDispatcher::shutdown()
.session_id = session,
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
.request = std::move(request),
.digest = std::nullopt
};
close_requests.push_back(std::move(request_info));
@ -576,6 +577,7 @@ void KeeperDispatcher::sessionCleanerTask()
.session_id = dead_session,
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
.request = std::move(request),
.digest = std::nullopt
};
if (!requests_queue->push(std::move(request_info)))
LOG_INFO(log, "Cannot push close request to queue while cleaning outdated sessions");

View File

@ -2127,7 +2127,7 @@ void KeeperStorage::preprocessRequest(
}
std::vector<Delta> new_deltas;
TransactionInfo transaction{.zxid = new_last_zxid};
TransactionInfo transaction{.zxid = new_last_zxid, .nodes_digest = {}};
uint64_t new_digest = getNodesDigest(false).value;
SCOPE_EXIT({
if (keeper_context->digestEnabled())

View File

@ -228,7 +228,7 @@ void RegExpTreeDictionary::initRegexNodes(Block & block)
else
{
Field field = parseStringToField(value, attr.type);
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = std::move(field), .original_value = value};
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = std::move(field), .pieces = {}, .original_value = value};
}
}
}

View File

@ -165,6 +165,7 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s
.file_segment_range = { file_segment_range.left, file_segment_right_bound },
.requested_range = {},
.cache_type = FilesystemCacheLogElement::CacheType::WRITE_THROUGH_CACHE,
.file_segment_key = {},
.file_segment_size = file_segment_range.size(),
.read_from_cache_attempted = false,
.read_buffer_id = {},

View File

@ -62,8 +62,16 @@ IOUringReader::IOUringReader(uint32_t entries_)
struct io_uring_params params =
{
.sq_entries = 0, // filled by the kernel, initializing to silence warning
.cq_entries = 0, // filled by the kernel, initializing to silence warning
.flags = 0,
.sq_thread_cpu = 0, // Unused (IORING_SETUP_SQ_AFF isn't set). Silences warning
.sq_thread_idle = 0, // Unused (IORING_SETUP_SQPOL isn't set). Silences warning
.features = 0, // filled by the kernel, initializing to silence warning
.wq_fd = 0, // Unused (IORING_SETUP_ATTACH_WQ isn't set). Silences warning.
.resv = {0, 0, 0}, // "The resv array must be initialized to zero."
.sq_off = {}, // filled by the kernel, initializing to silence warning
.cq_off = {}, // filled by the kernel, initializing to silence warning
};
int ret = io_uring_queue_init_params(entries_, &ring, &params);

View File

@ -64,7 +64,7 @@ public:
/// Optional. Useful when implementation needs to do ignore().
size_t offset = 0;
std::unique_ptr<Stopwatch> execution_watch;
std::unique_ptr<Stopwatch> execution_watch = {};
operator std::tuple<size_t &, size_t &>() { return {size, offset}; }
};

View File

@ -16,7 +16,7 @@ struct ObjectInfo
size_t size = 0;
time_t last_modification_time = 0;
std::map<String, String> metadata; /// Set only if getObjectInfo() is called with `with_metadata = true`.
std::map<String, String> metadata = {}; /// Set only if getObjectInfo() is called with `with_metadata = true`.
};
ObjectInfo getObjectInfo(

View File

@ -251,6 +251,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context)
return PushResult
{
.status = PushResult::TOO_MUCH_DATA,
.future = {},
.insert_data_buffer = std::make_unique<ConcatReadBuffer>(std::move(buffers)),
};
}
@ -318,6 +319,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context)
{
.status = PushResult::OK,
.future = std::move(insert_future),
.insert_data_buffer = nullptr,
};
}

View File

@ -30,12 +30,12 @@ struct FilesystemCacheLogElement
std::pair<size_t, size_t> file_segment_range{};
std::pair<size_t, size_t> requested_range{};
CacheType cache_type{};
std::string file_segment_key;
size_t file_segment_offset;
size_t file_segment_size;
std::string file_segment_key{};
size_t file_segment_offset = 0;
size_t file_segment_size = 0;
bool read_from_cache_attempted;
String read_buffer_id;
std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters;
String read_buffer_id{};
std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters = nullptr;
static std::string name() { return "FilesystemCacheLog"; }

View File

@ -23,11 +23,9 @@ public:
{
const char * assert_no_aggregates = nullptr;
const char * assert_no_windows = nullptr;
// Explicit empty initializers are needed to make designated initializers
// work on GCC 10.
std::unordered_set<String> uniq_names {};
ASTs aggregates;
ASTs window_functions;
ASTs aggregates{};
ASTs window_functions{};
};
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child)

View File

@ -20,9 +20,9 @@ struct QueryStatusInfo;
struct QueryResultDetails
{
String query_id;
std::optional<String> content_type;
std::optional<String> format;
std::optional<String> timezone;
std::optional<String> content_type = {};
std::optional<String> format = {};
std::optional<String> timezone = {};
};
using SetResultDetailsFunc = std::function<void(const QueryResultDetails &)>;

View File

@ -605,7 +605,7 @@ void writeColumnImpl(
if (use_dictionary)
{
dict_encoded_pages.push_back({.header = std::move(header)});
dict_encoded_pages.push_back({.header = std::move(header), .data = {}});
std::swap(dict_encoded_pages.back().data, compressed);
}
else

View File

@ -596,7 +596,13 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un
auto tmp_table = arrow::Table::FromRecordBatches({*batch});
size_t approx_chunk_original_size = static_cast<size_t>(std::ceil(static_cast<double>(row_group_batch.total_bytes_compressed) / row_group_batch.total_rows * (*tmp_table)->num_rows()));
PendingChunk res = {.chunk_idx = row_group_batch.next_chunk_idx, .row_group_batch_idx = row_group_batch_idx, .approx_original_chunk_size = approx_chunk_original_size};
PendingChunk res = {
.chunk = {},
.block_missing_values = {},
.chunk_idx = row_group_batch.next_chunk_idx,
.row_group_batch_idx = row_group_batch_idx,
.approx_original_chunk_size = approx_chunk_original_size
};
/// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields.
/// Otherwise fill the missing columns with zero values of its type.

View File

@ -78,10 +78,10 @@ public:
struct IndexStat
{
IndexType type;
std::string name;
std::string description;
std::string condition;
std::vector<std::string> used_keys;
std::string name = {};
std::string description = {};
std::string condition = {};
std::vector<std::string> used_keys = {};
size_t num_parts_after;
size_t num_granules_after;
};

View File

@ -46,9 +46,9 @@ public:
/// decide whether to deny or to accept that request.
struct Extension
{
std::shared_ptr<TaskIterator> task_iterator;
std::shared_ptr<ParallelReplicasReadingCoordinator> parallel_reading_coordinator;
std::optional<IConnections::ReplicaInfo> replica_info;
std::shared_ptr<TaskIterator> task_iterator = nullptr;
std::shared_ptr<ParallelReplicasReadingCoordinator> parallel_reading_coordinator = nullptr;
std::optional<IConnections::ReplicaInfo> replica_info = {};
};
/// Takes already set connection.

View File

@ -130,7 +130,7 @@ std::optional<ExternalDataSourceInfo> getExternalDataSourceConfiguration(
"Named collection of connection parameters is missing some "
"of the parameters and dictionary parameters are not added");
}
return ExternalDataSourceInfo{ .configuration = configuration, .specific_args = {}, .settings_changes = config_settings };
return ExternalDataSourceInfo{.configuration = configuration, .settings_changes = config_settings};
}
return std::nullopt;
}

View File

@ -39,7 +39,6 @@ using StorageSpecificArgs = std::vector<std::pair<String, ASTPtr>>;
struct ExternalDataSourceInfo
{
ExternalDataSourceConfiguration configuration;
StorageSpecificArgs specific_args;
SettingsChanges settings_changes;
};
@ -85,7 +84,6 @@ struct URLBasedDataSourceConfiguration
struct URLBasedDataSourceConfig
{
URLBasedDataSourceConfiguration configuration;
StorageSpecificArgs specific_args;
};
std::optional<URLBasedDataSourceConfig> getURLBasedDataSourceConfiguration(

View File

@ -232,6 +232,7 @@ PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(Merg
bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context, zookeeper);
result.push_back(PartitionCommandResultInfo{
.command_type = "UNFREEZE PART",
.partition_id = partition_id,
.part_name = partition_directory,
.backup_path = disk->getPath() + table_directory.generic_string(),
@ -239,11 +240,11 @@ PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(Merg
.backup_name = backup_name,
});
LOG_DEBUG(log, "Unfreezed part by path {}, keep shared data: {}", disk->getPath() + path, keep_shared);
LOG_DEBUG(log, "Unfrozen part by path {}, keep shared data: {}", disk->getPath() + path, keep_shared);
}
}
LOG_DEBUG(log, "Unfreezed {} parts", result.size());
LOG_DEBUG(log, "Unfrozen {} parts", result.size());
return result;
}

View File

@ -242,7 +242,7 @@ public:
MergeTreeTransactionPtr txn = NO_TRANSACTION_PTR;
HardlinkedFiles * hardlinked_files = nullptr;
bool copy_instead_of_hardlink = false;
NameSet files_to_copy_instead_of_hardlinks;
NameSet files_to_copy_instead_of_hardlinks = {};
bool keep_metadata_version = false;
bool make_source_readonly = false;
DiskTransactionPtr external_transaction = nullptr;

View File

@ -7720,6 +7720,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
part->is_frozen.store(true, std::memory_order_relaxed);
result.push_back(PartitionCommandResultInfo{
.command_type = "FREEZE PART",
.partition_id = part->info.partition_id,
.part_name = part->name,
.backup_path = new_storage->getFullRootPath(),
@ -7729,7 +7730,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
++parts_processed;
}
LOG_DEBUG(log, "Freezed {} parts", parts_processed);
LOG_DEBUG(log, "Froze {} parts", parts_processed);
return result;
}

View File

@ -13,20 +13,20 @@ namespace DB
struct MergeTreeMutationStatus
{
String id;
String command;
String id = "";
String command = "";
time_t create_time = 0;
std::map<String, Int64> block_numbers;
std::map<String, Int64> block_numbers{};
/// Parts that should be mutated/merged or otherwise moved to Obsolete state for this mutation to complete.
Names parts_to_do_names;
Names parts_to_do_names = {};
/// If the mutation is done. Note that in case of ReplicatedMergeTree parts_to_do == 0 doesn't imply is_done == true.
bool is_done = false;
String latest_failed_part;
String latest_failed_part = "";
time_t latest_fail_time = 0;
String latest_fail_reason;
String latest_fail_reason = "";
/// FIXME: currently unused, but would be much better to report killed mutations with this flag.
bool is_killed = false;

View File

@ -21,7 +21,7 @@ class ReadBuffer;
/// to values from set of columns which satisfy predicate.
struct MutationCommand
{
ASTPtr ast; /// The AST of the whole command
ASTPtr ast = {}; /// The AST of the whole command
enum Type
{
@ -43,27 +43,27 @@ struct MutationCommand
Type type = EMPTY;
/// WHERE part of mutation
ASTPtr predicate;
ASTPtr predicate = {};
/// Columns with corresponding actions
std::unordered_map<String, ASTPtr> column_to_update_expression;
std::unordered_map<String, ASTPtr> column_to_update_expression = {};
/// For MATERIALIZE INDEX and PROJECTION
String index_name;
String projection_name;
String index_name = {};
String projection_name = {};
/// For MATERIALIZE INDEX, UPDATE and DELETE.
ASTPtr partition;
ASTPtr partition = {};
/// For reads, drops and etc.
String column_name;
DataTypePtr data_type; /// Maybe empty if we just want to drop column
String column_name = {};
DataTypePtr data_type = {}; /// Maybe empty if we just want to drop column
/// We need just clear column, not drop from metadata.
bool clear = false;
/// Column rename_to
String rename_to;
String rename_to = {};
/// If parse_alter_commands, than consider more Alter commands as mutation commands
static std::optional<MutationCommand> parse(ASTAlterCommand * command, bool parse_alter_commands = false);

View File

@ -80,7 +80,7 @@ struct PartitionCommand
using PartitionCommands = std::vector<PartitionCommand>;
/// Result of exectuin of a single partition commands. Partition commands quite
/// Result of executing of a single partition commands. Partition commands quite
/// different, so some fields will be empty for some commands. Currently used in
/// ATTACH and FREEZE commands.
struct PartitionCommandResultInfo
@ -92,14 +92,14 @@ struct PartitionCommandResultInfo
/// Part name, always filled
String part_name;
/// Part name in /detached directory, filled in ATTACH
String old_part_name;
String old_part_name = {};
/// Absolute path to backup directory, filled in FREEZE
String backup_path;
String backup_path = {};
/// Absolute path part backup, filled in FREEZE
String part_backup_path;
String part_backup_path = {};
/// Name of the backup (specified by user or increment value), filled in
/// FREEZE
String backup_name;
String backup_name = {};
};
using PartitionCommandsResultInfo = std::vector<PartitionCommandResultInfo>;

View File

@ -1989,6 +1989,7 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition(
renamed_parts.old_and_new_names[i].old_name.clear();
results.push_back(PartitionCommandResultInfo{
.command_type = "ATTACH_PART",
.partition_id = loaded_parts[i]->info.partition_id,
.part_name = loaded_parts[i]->name,
.old_part_name = old_name,

View File

@ -6154,6 +6154,7 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition(
LOG_DEBUG(log, "Attached part {} as {}", old_name, loaded_parts[i]->name);
results.push_back(PartitionCommandResultInfo{
.command_type = "ATTACH PART",
.partition_id = loaded_parts[i]->info.partition_id,
.part_name = loaded_parts[i]->name,
.old_part_name = old_name,