mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #53751 from Algunenano/clang18
Support clang-18 (Wmissing-field-initializers)
This commit is contained in:
commit
0e601b856f
@ -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); }
|
||||
|
@ -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,
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
});
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -863,7 +863,9 @@ void TestKeeper::reconfig(
|
||||
.callback = [callback](const Response & response)
|
||||
{
|
||||
callback(dynamic_cast<const ReconfigResponse &>(response));
|
||||
}
|
||||
},
|
||||
.watch = nullptr,
|
||||
.time = {}
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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");
|
||||
|
@ -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())
|
||||
|
@ -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};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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 = {},
|
||||
|
@ -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);
|
||||
|
@ -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}; }
|
||||
};
|
||||
|
@ -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(
|
||||
|
@ -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,
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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 &)>;
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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>;
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
Loading…
Reference in New Issue
Block a user