From 93dac0c8809e99fbbabb2ef4122908ce88308b95 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 23 Aug 2023 14:07:02 +0200 Subject: [PATCH] Support clang-18 (Wmissing-field-initializers) --- src/Access/SettingsConstraints.h | 4 ++-- src/Columns/ColumnObject.cpp | 1 + src/Common/AsyncLoader.cpp | 1 + src/Common/FunctionDocumentation.h | 10 +++++----- src/Common/Macros.h | 4 ++-- src/Common/ZooKeeper/TestKeeper.cpp | 4 +++- src/Common/tests/gtest_async_loader.cpp | 2 +- src/Coordination/KeeperDispatcher.cpp | 2 ++ src/Coordination/KeeperStorage.cpp | 2 +- src/Dictionaries/RegExpTreeDictionary.cpp | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 1 + src/Disks/IO/IOUringReader.cpp | 8 ++++++++ src/IO/AsynchronousReader.h | 2 +- src/IO/S3/getObjectInfo.h | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 2 ++ src/Interpreters/FilesystemCacheLog.h | 10 +++++----- src/Interpreters/GetAggregatesVisitor.h | 6 ++---- src/Interpreters/executeQuery.h | 6 +++--- src/Processors/Formats/Impl/Parquet/Write.cpp | 2 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 8 +++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 8 ++++---- src/QueryPipeline/RemoteQueryExecutor.h | 6 +++--- .../ExternalDataSourceConfiguration.cpp | 2 +- src/Storages/ExternalDataSourceConfiguration.h | 2 -- src/Storages/Freeze.cpp | 5 +++-- src/Storages/MergeTree/IDataPartStorage.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- .../MergeTree/MergeTreeMutationStatus.h | 12 ++++++------ src/Storages/MutationCommands.h | 18 +++++++++--------- src/Storages/PartitionCommands.h | 10 +++++----- src/Storages/StorageMergeTree.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 1 + 32 files changed, 86 insertions(+), 63 deletions(-) diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index d09e60cc9d5..c5cd79c96f9 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -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); } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 2fb6d2c3028..2052ec3c968 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -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, }; } diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 9662146e137..2e96d3eab7d 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -182,6 +182,7 @@ AsyncLoader::AsyncLoader(std::vector pool_initializers, bool lo init.max_threads, /* max_free_threads = */ 0, init.max_threads), + .ready_queue = {}, .max_threads = init.max_threads }); } diff --git a/src/Common/FunctionDocumentation.h b/src/Common/FunctionDocumentation.h index 52b8aed2639..11d1c7df08e 100644 --- a/src/Common/FunctionDocumentation.h +++ b/src/Common/FunctionDocumentation.h @@ -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; diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 6ef87dfd51a..d403f5d2cee 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -34,8 +34,8 @@ public: StorageID table_id = StorageID::createEmpty(); bool ignore_unknown = false; bool expand_special_macros_only = false; - std::optional shard; - std::optional replica; + std::optional shard = {}; + std::optional replica = {}; /// Information about macro expansion size_t level = 0; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index f46c3f814a9..a19892736ea 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -863,7 +863,9 @@ void TestKeeper::reconfig( .callback = [callback](const Response & response) { callback(dynamic_cast(response)); - } + }, + .watch = nullptr, + .time = {} }); } diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 6504f1c97a9..eed66cca0af 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -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 getPoolInitializers(std::vector initializers) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 1fd67aef72c..b782c8b8f9a 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -480,6 +480,7 @@ void KeeperDispatcher::shutdown() .session_id = session, .time = duration_cast(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(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"); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 7fe85857ccb..95983dbcf3c 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2127,7 +2127,7 @@ void KeeperStorage::preprocessRequest( } std::vector 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()) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 068bbce8423..e9ba4c6268e 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -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}; } } } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 128b8df2046..b069d1289a0 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -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 = {}, diff --git a/src/Disks/IO/IOUringReader.cpp b/src/Disks/IO/IOUringReader.cpp index 7b68e0ee2de..f007cf00d50 100644 --- a/src/Disks/IO/IOUringReader.cpp +++ b/src/Disks/IO/IOUringReader.cpp @@ -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, ¶ms); diff --git a/src/IO/AsynchronousReader.h b/src/IO/AsynchronousReader.h index 6b8b93fcc20..467a3f1d6a9 100644 --- a/src/IO/AsynchronousReader.h +++ b/src/IO/AsynchronousReader.h @@ -64,7 +64,7 @@ public: /// Optional. Useful when implementation needs to do ignore(). size_t offset = 0; - std::unique_ptr execution_watch; + std::unique_ptr execution_watch = {}; operator std::tuple() { return {size, offset}; } }; diff --git a/src/IO/S3/getObjectInfo.h b/src/IO/S3/getObjectInfo.h index 06c7d386e43..a57d807644b 100644 --- a/src/IO/S3/getObjectInfo.h +++ b/src/IO/S3/getObjectInfo.h @@ -16,7 +16,7 @@ struct ObjectInfo size_t size = 0; time_t last_modification_time = 0; - std::map metadata; /// Set only if getObjectInfo() is called with `with_metadata = true`. + std::map metadata = {}; /// Set only if getObjectInfo() is called with `with_metadata = true`. }; ObjectInfo getObjectInfo( diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 132c1627e28..9e20ef803b3 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -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(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, }; } diff --git a/src/Interpreters/FilesystemCacheLog.h b/src/Interpreters/FilesystemCacheLog.h index 0d088a922e0..41a7c8e0fe6 100644 --- a/src/Interpreters/FilesystemCacheLog.h +++ b/src/Interpreters/FilesystemCacheLog.h @@ -30,12 +30,12 @@ struct FilesystemCacheLogElement std::pair file_segment_range{}; std::pair 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 profile_counters; + String read_buffer_id{}; + std::shared_ptr profile_counters = nullptr; static std::string name() { return "FilesystemCacheLog"; } diff --git a/src/Interpreters/GetAggregatesVisitor.h b/src/Interpreters/GetAggregatesVisitor.h index 7bf6591af69..863c4f81a1b 100644 --- a/src/Interpreters/GetAggregatesVisitor.h +++ b/src/Interpreters/GetAggregatesVisitor.h @@ -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 uniq_names {}; - ASTs aggregates; - ASTs window_functions; + ASTs aggregates{}; + ASTs window_functions{}; }; static bool needChildVisit(const ASTPtr & node, const ASTPtr & child) diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index f2a12bbef18..c2c2e081b38 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -20,9 +20,9 @@ struct QueryStatusInfo; struct QueryResultDetails { String query_id; - std::optional content_type; - std::optional format; - std::optional timezone; + std::optional content_type = {}; + std::optional format = {}; + std::optional timezone = {}; }; using SetResultDetailsFunc = std::function; diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 1d0b72ba5ec..d9cfd40e168 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -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 diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index bf7e035e601..8c431c42854 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -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(std::ceil(static_cast(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. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 3e3edd4dc5c..cb2a3a8ddf9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -78,10 +78,10 @@ public: struct IndexStat { IndexType type; - std::string name; - std::string description; - std::string condition; - std::vector used_keys; + std::string name = {}; + std::string description = {}; + std::string condition = {}; + std::vector used_keys = {}; size_t num_parts_after; size_t num_granules_after; }; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index a843ce520de..fb3baf4f983 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -46,9 +46,9 @@ public: /// decide whether to deny or to accept that request. struct Extension { - std::shared_ptr task_iterator; - std::shared_ptr parallel_reading_coordinator; - std::optional replica_info; + std::shared_ptr task_iterator = nullptr; + std::shared_ptr parallel_reading_coordinator = nullptr; + std::optional replica_info = {}; }; /// Takes already set connection. diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index e503c5edaab..4454fdd9cbd 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -130,7 +130,7 @@ std::optional 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; } diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index b825548debe..d4e737a7de1 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -39,7 +39,6 @@ using StorageSpecificArgs = std::vector>; 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 getURLBasedDataSourceConfiguration( diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index d2e19551c92..b9642ec7907 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -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; } diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 19af6085547..79aeec002d7 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -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; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2c09ab9725c..eb975b0d97c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7680,6 +7680,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(), @@ -7689,7 +7690,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( ++parts_processed; } - LOG_DEBUG(log, "Freezed {} parts", parts_processed); + LOG_DEBUG(log, "Froze {} parts", parts_processed); return result; } diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.h b/src/Storages/MergeTree/MergeTreeMutationStatus.h index 5f29b777293..b6f62e1218f 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.h +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.h @@ -13,20 +13,20 @@ namespace DB struct MergeTreeMutationStatus { - String id; - String command; + String id = ""; + String command = ""; time_t create_time = 0; - std::map block_numbers; + std::map 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; diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 5ef0cfda1be..dff77c629c1 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -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 column_to_update_expression; + std::unordered_map 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 parse(ASTAlterCommand * command, bool parse_alter_commands = false); diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 4921cf8e53b..b8b2ec47e71 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -80,7 +80,7 @@ struct PartitionCommand using PartitionCommands = std::vector; -/// 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; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9d7f6903b46..4e43c5b818b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -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, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 76a2ad9883c..65550464495 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -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,