From b7fbd9f0f23a88a4aef5a4dcb8725821739b8c61 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Jan 2024 03:51:59 +0800 Subject: [PATCH 001/197] Make it work --- .../QueryPlan/ReadFromMergeTree.cpp | 5 + src/Storages/ColumnsDescription.cpp | 7 +- src/Storages/ColumnsDescription.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + src/Storages/MergeTree/IMergeTreeReader.cpp | 78 +++++++ src/Storages/MergeTree/IMergeTreeReader.h | 10 + .../MergeTree/MergeTreeBlockReadUtils.cpp | 31 ++- .../MergeTree/MergeTreeBlockReadUtils.h | 4 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 18 +- .../MergeTree/MergeTreeDataPartCompact.h | 1 + .../MergeTree/MergeTreeDataPartInMemory.cpp | 9 +- .../MergeTree/MergeTreeDataPartInMemory.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 15 +- .../MergeTree/MergeTreeDataPartWide.h | 1 + .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 + .../MergeTree/MergeTreePrefetchedReadPool.h | 3 +- .../MergeTree/MergeTreeRangeReader.cpp | 46 ++-- src/Storages/MergeTree/MergeTreeRangeReader.h | 6 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 + src/Storages/MergeTree/MergeTreeReadPool.h | 1 + .../MergeTree/MergeTreeReadPoolBase.cpp | 26 ++- .../MergeTree/MergeTreeReadPoolBase.h | 6 +- .../MergeTree/MergeTreeReadPoolInOrder.cpp | 2 + .../MergeTree/MergeTreeReadPoolInOrder.h | 1 + .../MergeTreeReadPoolParallelReplicas.cpp | 2 + .../MergeTreeReadPoolParallelReplicas.h | 1 + ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 2 + ...MergeTreeReadPoolParallelReplicasInOrder.h | 1 + src/Storages/MergeTree/MergeTreeReadTask.cpp | 40 ++-- src/Storages/MergeTree/MergeTreeReadTask.h | 69 +++--- .../MergeTree/MergeTreeReaderCompact.cpp | 2 + .../MergeTree/MergeTreeReaderCompact.h | 1 + .../MergeTree/MergeTreeReaderInMemory.cpp | 2 + .../MergeTree/MergeTreeReaderInMemory.h | 1 + .../MergeTree/MergeTreeReaderWide.cpp | 2 + src/Storages/MergeTree/MergeTreeReaderWide.h | 1 + .../MergeTree/MergeTreeSelectProcessor.cpp | 203 ++++-------------- .../MergeTree/MergeTreeSelectProcessor.h | 6 +- .../MergeTree/MergeTreeSequentialSource.cpp | 13 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 1 + 40 files changed, 337 insertions(+), 289 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index d02e387afc3..473969b3e74 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -353,6 +353,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( reader_settings, required_columns, virt_column_names, + data.getPartitionValueType(), pool_settings, context); @@ -434,6 +435,7 @@ Pipe ReadFromMergeTree::readFromPool( reader_settings, required_columns, virt_column_names, + data.getPartitionValueType(), pool_settings, context); } @@ -447,6 +449,7 @@ Pipe ReadFromMergeTree::readFromPool( reader_settings, required_columns, virt_column_names, + data.getPartitionValueType(), pool_settings, context); } @@ -523,6 +526,7 @@ Pipe ReadFromMergeTree::readInOrder( reader_settings, required_columns, virt_column_names, + data.getPartitionValueType(), pool_settings, context); } @@ -538,6 +542,7 @@ Pipe ReadFromMergeTree::readInOrder( reader_settings, required_columns, virt_column_names, + data.getPartitionValueType(), pool_settings, context); } diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 1712b984596..c3b8c3742f7 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -465,6 +465,10 @@ NamesAndTypesList ColumnsDescription::get(const GetColumnsOptions & options) con NamesAndTypesList res; switch (options.kind) { + case GetColumnsOptions::None: + { + break; + } case GetColumnsOptions::All: { res = getAll(); @@ -555,7 +559,8 @@ static GetColumnsOptions::Kind defaultKindToGetKind(ColumnDefaultKind kind) case ColumnDefaultKind::Ephemeral: return GetColumnsOptions::Ephemeral; } - UNREACHABLE(); + + return GetColumnsOptions::None; } NamesAndTypesList ColumnsDescription::getByNames(const GetColumnsOptions & options, const Names & names) const diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 9a133f81d7a..b6cf58b5464 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -32,6 +32,7 @@ struct GetColumnsOptions { enum Kind : UInt8 { + None = 0, Ordinary = 1, Materialized = 2, Aliases = 4, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 640a1f1d0a3..a5ec8744033 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -47,6 +47,8 @@ class MarkCache; class UncompressedCache; class MergeTreeTransaction; +struct MergeTreeReadTaskInfo; +using MergeTreeReadTaskInfoPtr = std::shared_ptr; enum class DataPartRemovalState { @@ -93,6 +95,7 @@ public: const NamesAndTypesList & columns_, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, + const MergeTreeReadTaskInfoPtr & read_task_info_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 63ed8021f58..7ff69f7cc4b 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -1,7 +1,10 @@ #include +#include +#include #include #include #include +#include #include #include #include @@ -25,6 +28,7 @@ namespace ErrorCodes IMergeTreeReader::IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -47,14 +51,21 @@ IMergeTreeReader::IMergeTreeReader( , part_columns(data_part_info_for_read->isWidePart() ? data_part_info_for_read->getColumnsDescriptionWithCollectedNested() : data_part_info_for_read->getColumnsDescription()) + , read_task_info(read_task_info_) { columns_to_read.reserve(requested_columns.size()); serializations.reserve(requested_columns.size()); + size_t pos = 0; for (const auto & column : requested_columns) { columns_to_read.emplace_back(getColumnInPart(column)); serializations.emplace_back(getSerializationInPart(column)); + + if (read_task_info && read_task_info->virt_column_names.contains(column.name)) + virt_column_pos_to_name.emplace(pos, column.name); + + ++pos; } } @@ -63,6 +74,73 @@ const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints() return avg_value_size_hints; } +void IMergeTreeReader::fillVirtualColumns(Columns & columns, size_t rows) const +{ + if (std::all_of( + virt_column_pos_to_name.begin(), + virt_column_pos_to_name.end(), + [&columns](auto & elem) + { + chassert(elem.first < columns.size()); + return columns[elem.first] != nullptr; + })) + return; + + chassert(read_task_info != nullptr); + + const IMergeTreeDataPart * part = read_task_info->data_part.get(); + if (part->isProjectionPart()) + part = part->getParentPart(); + + for (auto [pos, name] : virt_column_pos_to_name) + { + auto & column = columns[pos]; + + if (column != nullptr) + continue; + + if (name == "_part_offset") + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} must have been filled by part reader", name); + } + else if (name == LightweightDeleteDescription::FILTER_COLUMN.name) + { + /// If _row_exists column isn't present in the part then fill it here with 1s + column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumnConst(rows, 1)->convertToFullColumnIfConst(); + } + else if (name == BlockNumberColumn::name) + { + column = BlockNumberColumn::type->createColumnConst(rows, part->info.min_block)->convertToFullColumnIfConst(); + } + else if (name == "_part") + { + column = DataTypeLowCardinality{std::make_shared()} + .createColumnConst(rows, part->name) + ->convertToFullColumnIfConst(); + } + else if (name == "_part_index") + { + column = DataTypeUInt64().createColumnConst(rows, read_task_info->part_index_in_query)->convertToFullColumnIfConst(); + } + else if (name == "_part_uuid") + { + column = DataTypeUUID().createColumnConst(rows, part->uuid)->convertToFullColumnIfConst(); + } + else if (name == "_partition_id") + { + column = DataTypeLowCardinality{std::make_shared()} + .createColumnConst(rows, part->info.partition_id) + ->convertToFullColumnIfConst(); + } + else if (name == "_partition_value") + { + column = read_task_info->partition_value_type + ->createColumnConst(rows, Tuple(part->partition.value.begin(), part->partition.value.end())) + ->convertToFullColumnIfConst(); + } + } +} + void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number) const { try diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 997be064f28..cd4417265fa 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -23,6 +23,7 @@ public: IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -42,6 +43,9 @@ public: const ValueSizeMap & getAvgValueSizeHints() const; + /// Add virtual columns that are not present in the block. + void fillVirtualColumns(Columns & columns, size_t rows) const; + /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// num_rows is needed in case if all res_columns are nullptr. @@ -113,6 +117,12 @@ private: /// Actual columns description in part. const ColumnsDescription & part_columns; + + /// Shared information required for reading. + MergeTreeReadTaskInfoPtr read_task_info; + + /// Map of positions in requested_columns which are virtual columns to their names. + std::map virt_column_pos_to_name; }; } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index f5f0fa6f726..fc6599b2851 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -111,11 +111,18 @@ NameSet injectRequiredColumns( if (with_subcolumns) options.withSubcolumns(); + auto virtuals_options = GetColumnsOptions(GetColumnsOptions::None).withVirtuals(); + for (size_t i = 0; i < columns.size(); ++i) { - /// We are going to fetch only physical columns and system columns + /// We are going to fetch physical columns and system columns first if (!storage_snapshot->tryGetColumn(options, columns[i])) - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no physical column or subcolumn {} in table", columns[i]); + { + if (storage_snapshot->tryGetColumn(virtuals_options, columns[i])) + continue; + else + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column or subcolumn {} in table", columns[i]); + } have_at_least_one_physical_column |= injectRequiredColumnsRecursively( columns[i], storage_snapshot, alter_conversions, @@ -258,11 +265,10 @@ void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Colum } -MergeTreeReadTask::Columns getReadTaskColumns( +MergeTreeReadTaskColumns getReadTaskColumns( const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, const StorageSnapshotPtr & storage_snapshot, const Names & required_columns, - const Names & system_columns, const PrewhereInfoPtr & prewhere_info, const ExpressionActionsSettings & actions_settings, const MergeTreeReaderSettings & reader_settings, @@ -270,16 +276,11 @@ MergeTreeReadTask::Columns getReadTaskColumns( { Names column_to_read_after_prewhere = required_columns; - /// Read system columns such as lightweight delete mask "_row_exists" if it is persisted in the part - for (const auto & name : system_columns) - if (data_part_info_for_reader.getColumns().contains(name)) - column_to_read_after_prewhere.push_back(name); - /// Inject columns required for defaults evaluation injectRequiredColumns( data_part_info_for_reader, storage_snapshot, with_subcolumns, column_to_read_after_prewhere); - MergeTreeReadTask::Columns result; + MergeTreeReadTaskColumns result; auto options = GetColumnsOptions(GetColumnsOptions::All) .withExtendedObjects() .withSystemColumns(); @@ -287,6 +288,9 @@ MergeTreeReadTask::Columns getReadTaskColumns( if (with_subcolumns) options.withSubcolumns(); + options.withVirtuals(); + + bool has_part_offset = std::find(required_columns.begin(), required_columns.end(), "_part_offset") != required_columns.end(); NameSet columns_from_previous_steps; auto add_step = [&](const PrewhereExprStep & step) { @@ -302,6 +306,13 @@ MergeTreeReadTask::Columns getReadTaskColumns( if (!columns_from_previous_steps.contains(name)) step_column_names.push_back(name); + /// Make sure _part_offset is read in STEP 0 + if (columns_from_previous_steps.empty() && has_part_offset) + { + if (std::find(step_column_names.begin(), step_column_names.end(), "_part_offset") == step_column_names.end()) + step_column_names.push_back("_part_offset"); + } + if (!step_column_names.empty()) injectRequiredColumns( data_part_info_for_reader, storage_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 9417d47814a..b19c42c8db8 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include @@ -22,11 +21,10 @@ NameSet injectRequiredColumns( bool with_subcolumns, Names & columns); -MergeTreeReadTask::Columns getReadTaskColumns( +MergeTreeReadTaskColumns getReadTaskColumns( const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, const StorageSnapshotPtr & storage_snapshot, const Names & required_columns, - const Names & system_columns, const PrewhereInfoPtr & prewhere_info, const ExpressionActionsSettings & actions_settings, const MergeTreeReaderSettings & reader_settings, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 0ecd7abe183..7baba26d15c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -33,6 +33,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const NamesAndTypesList & columns_to_read, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, + const MergeTreeReadTaskInfoPtr & read_task_info_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, @@ -41,12 +42,21 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { auto read_info = std::make_shared(shared_from_this(), alter_conversions); - auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr; + auto * load_marks_threadpool + = reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr; return std::make_unique( - read_info, columns_to_read, storage_snapshot, uncompressed_cache, - mark_cache, mark_ranges, reader_settings, load_marks_threadpool, - avg_value_size_hints, profile_callback); + read_info, + columns_to_read, + read_task_info_, + storage_snapshot, + uncompressed_cache, + mark_cache, + mark_ranges, + reader_settings, + load_marks_threadpool, + avg_value_size_hints, + profile_callback); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 35a358b3720..479cb23c3e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -32,6 +32,7 @@ public: const NamesAndTypesList & columns, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, + const MergeTreeReadTaskInfoPtr & read_task_info_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 2f01dbfe04b..d4d04bd55c2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -33,6 +33,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( const NamesAndTypesList & columns_to_read, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, + const MergeTreeReadTaskInfoPtr & read_task_info_, UncompressedCache * /* uncompressed_cache */, MarkCache * /* mark_cache */, const AlterConversionsPtr & alter_conversions, @@ -44,7 +45,13 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - read_info, ptr, columns_to_read, storage_snapshot, mark_ranges, reader_settings); + read_info, + ptr, + columns_to_read, + read_task_info_, + storage_snapshot, + mark_ranges, + reader_settings); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 27f8ba4bccb..d3e6a9a5b27 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -21,6 +21,7 @@ public: const NamesAndTypesList & columns, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, + const MergeTreeReadTaskInfoPtr & read_task_info_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index dc6c1f0019d..5c8b3f9e357 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -31,6 +31,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( const NamesAndTypesList & columns_to_read, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, + const MergeTreeReadTaskInfoPtr & read_task_info_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, @@ -40,10 +41,16 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( { auto read_info = std::make_shared(shared_from_this(), alter_conversions); return std::make_unique( - read_info, columns_to_read, - storage_snapshot, uncompressed_cache, - mark_cache, mark_ranges, reader_settings, - avg_value_size_hints, profile_callback); + read_info, + columns_to_read, + read_task_info_, + storage_snapshot, + uncompressed_cache, + mark_cache, + mark_ranges, + reader_settings, + avg_value_size_hints, + profile_callback); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 14147c4ad56..7bc4fe8c777 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -27,6 +27,7 @@ public: const NamesAndTypesList & columns, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, + const MergeTreeReadTaskInfoPtr & read_task_info_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 3f9632637b6..da50a39d5b1 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -114,6 +114,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -124,6 +125,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( reader_settings_, column_names_, virtual_column_names_, + partition_value_type_, settings_, context_) , WithContext(context_) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 9925d4e2fa4..5845e72a873 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -24,6 +24,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); @@ -67,7 +68,7 @@ private: struct ThreadTask { - using InfoPtr = MergeTreeReadTask::InfoPtr; + using InfoPtr = MergeTreeReadTaskInfoPtr; ThreadTask(InfoPtr read_info_, MarkRanges ranges_, Priority priority_) : read_info(std::move(read_info_)), ranges(std::move(ranges_)), priority(priority_) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index cce7e56dda9..81263085f75 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -808,8 +808,7 @@ MergeTreeRangeReader::MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, const PrewhereExprStep * prewhere_info_, - bool last_reader_in_chain_, - const Names & non_const_virtual_column_names_) + bool last_reader_in_chain_) : merge_tree_reader(merge_tree_reader_) , index_granularity(&(merge_tree_reader->data_part_info_for_read->getIndexGranularity())) , prev_reader(prev_reader_) @@ -826,21 +825,6 @@ MergeTreeRangeReader::MergeTreeRangeReader( result_sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name}); } - for (const auto & column_name : non_const_virtual_column_names_) - { - if (result_sample_block.has(column_name)) - continue; - - non_const_virtual_column_names.push_back(column_name); - - if (column_name == "_part_offset" && !prev_reader) - { - /// _part_offset column is filled by the first reader. - read_sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); - result_sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), column_name)); - } - } - if (prewhere_info) { const auto & step = *prewhere_info; @@ -1006,6 +990,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (num_read_rows == 0) num_read_rows = read_result.num_rows; + merge_tree_reader->fillVirtualColumns(columns, num_read_rows); + /// fillMissingColumns() must be called after reading but befoe any filterings because /// some columns (e.g. arrays) might be only partially filled and thus not be valid and /// fillMissingColumns() fixes this. @@ -1056,22 +1042,23 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar { /// Physical columns go first and then some virtual columns follow - size_t physical_columns_count = merge_tree_reader->getColumns().size(); - Columns physical_columns(read_result.columns.begin(), read_result.columns.begin() + physical_columns_count); + size_t columns_count = merge_tree_reader->getColumns().size(); + Columns columns(read_result.columns.begin(), read_result.columns.begin() + columns_count); + merge_tree_reader->fillVirtualColumns(columns, read_result.num_rows); bool should_evaluate_missing_defaults; - merge_tree_reader->fillMissingColumns(physical_columns, should_evaluate_missing_defaults, read_result.num_rows); + merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, read_result.num_rows); /// If some columns absent in part, then evaluate default values if (should_evaluate_missing_defaults) - merge_tree_reader->evaluateMissingDefaults({}, physical_columns); + merge_tree_reader->evaluateMissingDefaults({}, columns); /// If result not empty, then apply on-fly alter conversions if any required if (!prewhere_info || prewhere_info->perform_alter_conversions) - merge_tree_reader->performRequiredConversions(physical_columns); + merge_tree_reader->performRequiredConversions(columns); - for (size_t i = 0; i < physical_columns.size(); ++i) - read_result.columns[i] = std::move(physical_columns[i]); + for (size_t i = 0; i < columns.size(); ++i) + read_result.columns[i] = std::move(columns[i]); } size_t total_bytes = 0; @@ -1163,12 +1150,17 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t result.adjustLastGranule(); if (read_sample_block.has("_part_offset")) - fillPartOffsetColumn(result, leading_begin_part_offset, leading_end_part_offset); + { + size_t pos = read_sample_block.getPositionByName("_part_offset"); + chassert(pos < result.columns.size()); + chassert(result.columns[pos] == nullptr); + result.columns[pos] = fillPartOffsetColumn(result, leading_begin_part_offset, leading_end_part_offset); + } return result; } -void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) +ColumnPtr MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) { size_t num_rows = result.numReadRows(); @@ -1194,7 +1186,7 @@ void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 lead *pos++ = start_part_offset++; } - result.columns.emplace_back(std::move(column)); + return column; } Columns MergeTreeRangeReader::continueReadingChain(const ReadResult & result, size_t & num_rows) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 04d42138963..77dcc8853bb 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -101,8 +101,7 @@ public: IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, const PrewhereExprStep * prewhere_info_, - bool last_reader_in_chain_, - const Names & non_const_virtual_column_names); + bool last_reader_in_chain_); MergeTreeRangeReader() = default; @@ -309,7 +308,7 @@ private: ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); Columns continueReadingChain(const ReadResult & result, size_t & num_rows); void executePrewhereActionsAndFilterColumns(ReadResult & result) const; - void fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); + ColumnPtr fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; @@ -323,7 +322,6 @@ private: bool last_reader_in_chain = false; bool is_initialized = false; - Names non_const_virtual_column_names; Poco::Logger * log = &Poco::Logger::get("MergeTreeRangeReader"); }; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 8ed7a9d8707..dbd27aebc21 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -40,6 +40,7 @@ MergeTreeReadPool::MergeTreeReadPool( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -50,6 +51,7 @@ MergeTreeReadPool::MergeTreeReadPool( reader_settings_, column_names_, virtual_column_names_, + partition_value_type_, settings_, context_) , min_marks_for_concurrent_read(pool_settings.min_marks_for_concurrent_read) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 3a1af947cae..769d1bfdd28 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -32,6 +32,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 446baccd961..d5811c33f00 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -13,6 +13,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & pool_settings_, const ContextPtr & context_) : parts_ranges(std::move(parts_)) @@ -22,6 +23,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( , reader_settings(reader_settings_) , column_names(column_names_) , virtual_column_names(virtual_column_names_) + , partition_value_type(partition_value_type_) , pool_settings(pool_settings_) , owned_mark_cache(context_->getGlobalContext()->getMarkCache()) , owned_uncompressed_cache(pool_settings_.use_uncompressed_cache ? context_->getGlobalContext()->getUncompressedCache() : nullptr) @@ -44,7 +46,7 @@ void MergeTreeReadPoolBase::fillPerPartInfos() assertSortedAndNonIntersecting(part_with_ranges.ranges); #endif - MergeTreeReadTask::Info read_task_info; + MergeTreeReadTaskInfo read_task_info; read_task_info.data_part = part_with_ranges.data_part; read_task_info.part_index_in_query = part_with_ranges.part_index_in_query; @@ -52,10 +54,22 @@ void MergeTreeReadPoolBase::fillPerPartInfos() LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, part_with_ranges.alter_conversions); + Names column_and_virtual_column_names; + column_and_virtual_column_names.reserve(column_names.size() + virtual_column_names.size()); + column_and_virtual_column_names.insert(column_and_virtual_column_names.end(), column_names.begin(), column_names.end()); + column_and_virtual_column_names.insert( + column_and_virtual_column_names.end(), virtual_column_names.begin(), virtual_column_names.end()); read_task_info.task_columns = getReadTaskColumns( - part_info, storage_snapshot, column_names, virtual_column_names, - prewhere_info, actions_settings, - reader_settings, /*with_subcolumns=*/ true); + part_info, + storage_snapshot, + column_and_virtual_column_names, + prewhere_info, + actions_settings, + reader_settings, + /*with_subcolumns=*/true); + + read_task_info.virt_column_names = {virtual_column_names.begin(), virtual_column_names.end()}; + read_task_info.partition_value_type = partition_value_type; if (pool_settings.preferred_block_size_bytes > 0) { @@ -75,7 +89,7 @@ void MergeTreeReadPoolBase::fillPerPartInfos() } is_part_on_remote_disk.push_back(part_with_ranges.data_part->isStoredOnRemoteDisk()); - per_part_infos.push_back(std::make_shared(std::move(read_task_info))); + per_part_infos.push_back(std::make_shared(std::move(read_task_info))); } } @@ -97,7 +111,7 @@ std::vector MergeTreeReadPoolBase::getPerPartSumMarks() const } MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( - MergeTreeReadTask::InfoPtr read_info, + MergeTreeReadTaskInfoPtr read_info, MarkRanges ranges, MergeTreeReadTask * previous_task) const { diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 0081063cd37..3aa9eb8670e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -29,6 +29,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); @@ -43,6 +44,7 @@ protected: const MergeTreeReaderSettings reader_settings; const Names column_names; const Names virtual_column_names; + const DataTypePtr partition_value_type; const PoolSettings pool_settings; const MarkCachePtr owned_mark_cache; const UncompressedCachePtr owned_uncompressed_cache; @@ -52,13 +54,13 @@ protected: std::vector getPerPartSumMarks() const; MergeTreeReadTaskPtr createTask( - MergeTreeReadTask::InfoPtr read_info, + MergeTreeReadTaskInfoPtr read_info, MarkRanges ranges, MergeTreeReadTask * previous_task) const; MergeTreeReadTask::Extras getExtras() const; - std::vector per_part_infos; + std::vector per_part_infos; std::vector is_part_on_remote_disk; ReadBufferFromFileBase::ProfileCallback profile_callback; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index 1b621ad5055..692e45993c7 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -18,6 +18,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -28,6 +29,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( reader_settings_, column_names_, virtual_column_names_, + partition_value_type_, settings_, context_) , has_limit_below_one_block(has_limit_below_one_block_) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index d9cc1ba4984..de7457dfab8 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -17,6 +17,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 47436ed1407..fb14dfe9a6e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -19,6 +19,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -29,6 +30,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( reader_settings_, column_names_, virtual_column_names_, + partition_value_type_, settings_, context_) , extension(std::move(extension_)) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index 7579a892b67..0b17bcee52b 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -17,6 +17,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index a822a517933..d242d1e81fe 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -18,6 +18,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -28,6 +29,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd reader_settings_, column_names_, virtual_column_names_, + partition_value_type_, settings_, context_) , extension(std::move(extension_)) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 3e5f8f5dfba..09935e1be2f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -18,6 +18,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, + const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadTask.cpp b/src/Storages/MergeTree/MergeTreeReadTask.cpp index dcfed700fac..f08ab55adbd 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.cpp +++ b/src/Storages/MergeTree/MergeTreeReadTask.cpp @@ -10,7 +10,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -String MergeTreeReadTask::Columns::dump() const +String MergeTreeReadTaskColumns::dump() const { WriteBufferFromOwnString s; for (size_t i = 0; i < pre_columns.size(); ++i) @@ -22,7 +22,7 @@ String MergeTreeReadTask::Columns::dump() const } MergeTreeReadTask::MergeTreeReadTask( - InfoPtr info_, + MergeTreeReadTaskInfoPtr info_, Readers readers_, MarkRanges mark_ranges_, MergeTreeBlockSizePredictorPtr size_predictor_) @@ -34,16 +34,23 @@ MergeTreeReadTask::MergeTreeReadTask( } MergeTreeReadTask::Readers MergeTreeReadTask::createReaders( - const InfoPtr & read_info, const Extras & extras, const MarkRanges & ranges) + const MergeTreeReadTaskInfoPtr & read_info, const Extras & extras, const MarkRanges & ranges) { Readers new_readers; auto create_reader = [&](const NamesAndTypesList & columns_to_read) { return read_info->data_part->getReader( - columns_to_read, extras.storage_snapshot, ranges, - extras.uncompressed_cache, extras.mark_cache, - read_info->alter_conversions, extras.reader_settings, extras.value_size_map, extras.profile_callback); + columns_to_read, + extras.storage_snapshot, + ranges, + read_info, + extras.uncompressed_cache, + extras.mark_cache, + read_info->alter_conversions, + extras.reader_settings, + extras.value_size_map, + extras.profile_callback); }; new_readers.main = create_reader(read_info->task_columns.columns); @@ -58,10 +65,8 @@ MergeTreeReadTask::Readers MergeTreeReadTask::createReaders( return new_readers; } -MergeTreeReadTask::RangeReaders MergeTreeReadTask::createRangeReaders( - const Readers & task_readers, - const PrewhereExprInfo & prewhere_actions, - const Names & non_const_virtual_column_names) +MergeTreeReadTask::RangeReaders +MergeTreeReadTask::createRangeReaders(const Readers & task_readers, const PrewhereExprInfo & prewhere_actions) { MergeTreeReadTask::RangeReaders new_range_readers; if (prewhere_actions.steps.size() != task_readers.prewhere.size()) @@ -77,10 +82,7 @@ MergeTreeReadTask::RangeReaders MergeTreeReadTask::createRangeReaders( { last_reader = task_readers.main->getColumns().empty() && (i + 1 == prewhere_actions.steps.size()); - MergeTreeRangeReader current_reader( - task_readers.prewhere[i].get(), - prev_reader, prewhere_actions.steps[i].get(), - last_reader, non_const_virtual_column_names); + MergeTreeRangeReader current_reader(task_readers.prewhere[i].get(), prev_reader, prewhere_actions.steps[i].get(), last_reader); new_range_readers.prewhere.push_back(std::move(current_reader)); prev_reader = &new_range_readers.prewhere.back(); @@ -88,11 +90,11 @@ MergeTreeReadTask::RangeReaders MergeTreeReadTask::createRangeReaders( if (!last_reader) { - new_range_readers.main = MergeTreeRangeReader(task_readers.main.get(), prev_reader, nullptr, true, non_const_virtual_column_names); + new_range_readers.main = MergeTreeRangeReader(task_readers.main.get(), prev_reader, nullptr, true); } else { - /// If all columns are read by prewhere range readers than move last prewhere range reader to main. + /// If all columns are read by prewhere range readers, move last prewhere range reader to main. new_range_readers.main = std::move(new_range_readers.prewhere.back()); new_range_readers.prewhere.pop_back(); } @@ -100,14 +102,12 @@ MergeTreeReadTask::RangeReaders MergeTreeReadTask::createRangeReaders( return new_range_readers; } -void MergeTreeReadTask::initializeRangeReaders( - const PrewhereExprInfo & prewhere_actions, - const Names & non_const_virtual_column_names) +void MergeTreeReadTask::initializeRangeReaders(const PrewhereExprInfo & prewhere_actions) { if (range_readers.main.isInitialized()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Range reader is already initialized"); - range_readers = createRangeReaders(readers, prewhere_actions, non_const_virtual_column_names); + range_readers = createRangeReaders(readers, prewhere_actions); } UInt64 MergeTreeReadTask::estimateNumRows(const BlockSizeParams & params) const diff --git a/src/Storages/MergeTree/MergeTreeReadTask.h b/src/Storages/MergeTree/MergeTreeReadTask.h index 8d2f0657fd1..7e935f5d28d 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.h +++ b/src/Storages/MergeTree/MergeTreeReadTask.h @@ -40,36 +40,40 @@ enum class MergeTreeReadType ParallelReplicas, }; +struct MergeTreeReadTaskColumns +{ + /// Column names to read during WHERE + NamesAndTypesList columns; + /// Column names to read during each PREWHERE step + std::vector pre_columns; + + String dump() const; +}; + +struct MergeTreeReadTaskInfo +{ + /// Data part which should be read while performing this task + DataPartPtr data_part; + /// For `part_index` virtual column + size_t part_index_in_query; + /// Alter converversionss that should be applied on-fly for part. + AlterConversionsPtr alter_conversions; + /// Column names to read during PREWHERE and WHERE + MergeTreeReadTaskColumns task_columns; + /// Virtual column names to read + NameSet virt_column_names; + /// For `partition_value` virtual column + DataTypePtr partition_value_type; + /// Shared initialized size predictor. It is copied for each new task. + MergeTreeBlockSizePredictorPtr shared_size_predictor; +}; + +using MergeTreeReadTaskInfoPtr = std::shared_ptr; + /// A batch of work for MergeTreeSelectProcessor struct MergeTreeReadTask : private boost::noncopyable { public: - struct Columns - { - /// Column names to read during WHERE - NamesAndTypesList columns; - /// Column names to read during each PREWHERE step - std::vector pre_columns; - - String dump() const; - }; - - struct Info - { - /// Data part which should be read while performing this task - DataPartPtr data_part; - /// For virtual `part_index` virtual column - size_t part_index_in_query; - /// Alter converversionss that should be applied on-fly for part. - AlterConversionsPtr alter_conversions; - /// Column names to read during PREWHERE and WHERE - Columns task_columns; - /// Shared initialized size predictor. It is copied for each new task. - MergeTreeBlockSizePredictorPtr shared_size_predictor; - }; - - using InfoPtr = std::shared_ptr; - /// Extra params that required for creation of reader. struct Extras { @@ -115,27 +119,28 @@ public: size_t num_read_bytes = 0; }; - MergeTreeReadTask(InfoPtr info_, Readers readers_, MarkRanges mark_ranges_, MergeTreeBlockSizePredictorPtr size_predictor_); + MergeTreeReadTask( + MergeTreeReadTaskInfoPtr info_, Readers readers_, MarkRanges mark_ranges_, MergeTreeBlockSizePredictorPtr size_predictor_); - void initializeRangeReaders(const PrewhereExprInfo & prewhere_actions, const Names & non_const_virtual_column_names); + void initializeRangeReaders(const PrewhereExprInfo & prewhere_actions); BlockAndProgress read(const BlockSizeParams & params); bool isFinished() const { return mark_ranges.empty() && range_readers.main.isCurrentRangeFinished(); } - const Info & getInfo() const { return *info; } + const MergeTreeReadTaskInfo & getInfo() const { return *info; } const MergeTreeRangeReader & getMainRangeReader() const { return range_readers.main; } const IMergeTreeReader & getMainReader() const { return *readers.main; } Readers releaseReaders() { return std::move(readers); } - static Readers createReaders(const InfoPtr & read_info, const Extras & extras, const MarkRanges & ranges); - static RangeReaders createRangeReaders(const Readers & readers, const PrewhereExprInfo & prewhere_actions, const Names & non_const_virtual_column_names); + static Readers createReaders(const MergeTreeReadTaskInfoPtr & read_info, const Extras & extras, const MarkRanges & ranges); + static RangeReaders createRangeReaders(const Readers & readers, const PrewhereExprInfo & prewhere_actions); private: UInt64 estimateNumRows(const BlockSizeParams & params) const; /// Shared information required for reading. - InfoPtr info; + MergeTreeReadTaskInfoPtr info; /// Readers for data_part of this task. /// May be reused and released to the next task. diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 02048009296..65b578e065d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes MergeTreeReaderCompact::MergeTreeReaderCompact( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -29,6 +30,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( : IMergeTreeReader( data_part_info_for_read_, columns_, + read_task_info_, storage_snapshot_, uncompressed_cache_, mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index dace4ec468e..c87e4889d26 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -21,6 +21,7 @@ public: MergeTreeReaderCompact( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index bacd86511f5..e628bd76cb4 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -19,12 +19,14 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_) : IMergeTreeReader( data_part_info_for_read_, columns_, + read_task_info_, storage_snapshot_, nullptr, nullptr, diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h index e26a98f0916..161b615a511 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.h +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -18,6 +18,7 @@ public: MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 640432ef755..8270f2452c2 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -24,6 +24,7 @@ namespace MergeTreeReaderWide::MergeTreeReaderWide( MergeTreeDataPartInfoForReaderPtr data_part_info_, NamesAndTypesList columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -35,6 +36,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( : IMergeTreeReader( data_part_info_, columns_, + read_task_info_, storage_snapshot_, uncompressed_cache_, mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 2a850cc2814..ecfaa43a3f8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -17,6 +17,7 @@ public: MergeTreeReaderWide( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, + const MergeTreeReadTaskInfoPtr & read_task_info_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index aeff438f509..264ceae3655 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -24,19 +24,6 @@ namespace ErrorCodes extern const int QUERY_WAS_CANCELLED; } -static void injectNonConstVirtualColumns( - size_t rows, - Block & block, - const Names & virtual_columns, - MergeTreeReadTask * task = nullptr); - -static void injectPartConstVirtualColumns( - size_t rows, - Block & block, - MergeTreeReadTask * task, - const DataTypePtr & partition_value_type, - const Names & virtual_columns); - MergeTreeSelectProcessor::MergeTreeSelectProcessor( MergeTreeReadPoolPtr pool_, MergeTreeSelectAlgorithmPtr algorithm_, @@ -71,15 +58,9 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( lightweight_delete_filter_step = std::make_shared(std::move(step)); } - header_without_const_virtual_columns = applyPrewhereActions(pool->getHeader(), prewhere_info); - size_t non_const_columns_offset = header_without_const_virtual_columns.columns(); - injectNonConstVirtualColumns(0, header_without_const_virtual_columns, virt_column_names); - - for (size_t col_num = non_const_columns_offset; col_num < header_without_const_virtual_columns.columns(); ++col_num) - non_const_virtual_column_names.emplace_back(header_without_const_virtual_columns.getByPosition(col_num).name); - - result_header = header_without_const_virtual_columns; - injectPartConstVirtualColumns(0, result_header, nullptr, partition_value_type, virt_column_names); + result_header = pool->getHeader(); + injectVirtualColumns(result_header, partition_value_type, virt_column_names); + result_header = applyPrewhereActions(result_header, prewhere_info); if (!prewhere_actions.steps.empty()) LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions.steps.size(), prewhere_actions.dumpConditions()); @@ -163,8 +144,6 @@ ChunkAndProgress MergeTreeSelectProcessor::read() if (res.row_count) { - injectVirtualColumns(res.block, res.row_count, task.get(), partition_value_type, virt_column_names); - /// Reorder the columns according to result_header Columns ordered_columns; ordered_columns.reserve(result_header.columns()); @@ -198,7 +177,7 @@ void MergeTreeSelectProcessor::initializeRangeReaders() for (const auto & step : prewhere_actions.steps) all_prewhere_actions.steps.push_back(step); - task->initializeRangeReaders(all_prewhere_actions, non_const_virtual_column_names); + task->initializeRangeReaders(all_prewhere_actions); } @@ -208,8 +187,6 @@ namespace { explicit VirtualColumnsInserter(Block & block_) : block(block_) {} - bool columnExists(const String & name) const { return block.has(name); } - void insertUInt8Column(const ColumnPtr & column, const String & name) { block.insert({column, std::make_shared(), name}); @@ -230,16 +207,9 @@ namespace block.insert({column, std::make_shared(std::make_shared()), name}); } - void insertPartitionValueColumn( - size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name) + void insertPartitionValueColumn(const DataTypePtr & partition_value_type, const String & name) { - ColumnPtr column; - if (rows) - column = partition_value_type->createColumnConst(rows, Tuple(partition_value.begin(), partition_value.end())) - ->convertToFullColumnIfConst(); - else - column = partition_value_type->createColumn(); - + ColumnPtr column = partition_value_type->createColumn(); block.insert({column, partition_value_type, name}); } @@ -247,154 +217,55 @@ namespace }; } -/// Adds virtual columns that are not const for all rows -static void injectNonConstVirtualColumns( - size_t rows, - Block & block, - const Names & virtual_columns, - MergeTreeReadTask * task) +void MergeTreeSelectProcessor::injectVirtualColumns(Block & block, const DataTypePtr & partition_value_type, const Names & virtual_columns) { VirtualColumnsInserter inserter(block); + + /// add virtual columns + /// Except _sample_factor, which is added from the outside. for (const auto & virtual_column_name : virtual_columns) { if (virtual_column_name == "_part_offset") { - if (!rows) - { - inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name); - } - else - { - if (!inserter.columnExists(virtual_column_name)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Column {} must have been filled part reader", - virtual_column_name); - } + inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name); } - - if (virtual_column_name == LightweightDeleteDescription::FILTER_COLUMN.name) + else if (virtual_column_name == LightweightDeleteDescription::FILTER_COLUMN.name) { - /// If _row_exists column isn't present in the part then fill it here with 1s - ColumnPtr column; - if (rows) - column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumnConst(rows, 1)->convertToFullColumnIfConst(); - else - column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumn(); - - inserter.insertUInt8Column(column, virtual_column_name); + ColumnPtr column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumn(); + inserter.insertUInt8Column(column, virtual_column_name); } - - if (virtual_column_name == BlockNumberColumn::name) + else if (virtual_column_name == BlockNumberColumn::name) { - ColumnPtr column; - if (rows) - { - size_t value = 0; - if (task) - { - value = task->getInfo().data_part ? task->getInfo().data_part->info.min_block : 0; - } - column = BlockNumberColumn::type->createColumnConst(rows, value)->convertToFullColumnIfConst(); - } - else - column = BlockNumberColumn::type->createColumn(); - + ColumnPtr column = BlockNumberColumn::type->createColumn(); inserter.insertUInt64Column(column, virtual_column_name); } - } -} - -/// Adds virtual columns that are const for the whole part -static void injectPartConstVirtualColumns( - size_t rows, - Block & block, - MergeTreeReadTask * task, - const DataTypePtr & partition_value_type, - const Names & virtual_columns) -{ - VirtualColumnsInserter inserter(block); - /// add virtual columns - /// Except _sample_factor, which is added from the outside. - if (!virtual_columns.empty()) - { - if (unlikely(rows && !task)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert virtual columns to non-empty chunk without specified task."); - - const IMergeTreeDataPart * part = nullptr; - - if (rows) + else if (virtual_column_name == "_part") { - part = task->getInfo().data_part.get(); - if (part->isProjectionPart()) - part = part->getParentPart(); + ColumnPtr column = DataTypeLowCardinality{std::make_shared()}.createColumn(); + inserter.insertLowCardinalityColumn(column, virtual_column_name); } - - for (const auto & virtual_column_name : virtual_columns) + else if (virtual_column_name == "_part_index") { - if (virtual_column_name == "_part") - { - ColumnPtr column; - if (rows) - column = DataTypeLowCardinality{std::make_shared()} - .createColumnConst(rows, part->name) - ->convertToFullColumnIfConst(); - else - column = DataTypeLowCardinality{std::make_shared()}.createColumn(); - - inserter.insertLowCardinalityColumn(column, virtual_column_name); - } - else if (virtual_column_name == "_part_index") - { - ColumnPtr column; - if (rows) - column = DataTypeUInt64().createColumnConst(rows, task->getInfo().part_index_in_query)->convertToFullColumnIfConst(); - else - column = DataTypeUInt64().createColumn(); - - inserter.insertUInt64Column(column, virtual_column_name); - } - else if (virtual_column_name == "_part_uuid") - { - ColumnPtr column; - if (rows) - column = DataTypeUUID().createColumnConst(rows, part->uuid)->convertToFullColumnIfConst(); - else - column = DataTypeUUID().createColumn(); - - inserter.insertUUIDColumn(column, virtual_column_name); - } - else if (virtual_column_name == "_partition_id") - { - ColumnPtr column; - if (rows) - column = DataTypeLowCardinality{std::make_shared()} - .createColumnConst(rows, part->info.partition_id) - ->convertToFullColumnIfConst(); - else - column = DataTypeLowCardinality{std::make_shared()}.createColumn(); - - inserter.insertLowCardinalityColumn(column, virtual_column_name); - } - else if (virtual_column_name == "_partition_value") - { - if (rows) - inserter.insertPartitionValueColumn(rows, part->partition.value, partition_value_type, virtual_column_name); - else - inserter.insertPartitionValueColumn(rows, {}, partition_value_type, virtual_column_name); - } + ColumnPtr column = DataTypeUInt64().createColumn(); + inserter.insertUInt64Column(column, virtual_column_name); + } + else if (virtual_column_name == "_part_uuid") + { + ColumnPtr column = DataTypeUUID().createColumn(); + inserter.insertUUIDColumn(column, virtual_column_name); + } + else if (virtual_column_name == "_partition_id") + { + ColumnPtr column = DataTypeLowCardinality{std::make_shared()}.createColumn(); + inserter.insertLowCardinalityColumn(column, virtual_column_name); + } + else if (virtual_column_name == "_partition_value") + { + inserter.insertPartitionValueColumn(partition_value_type, virtual_column_name); } } } -void MergeTreeSelectProcessor::injectVirtualColumns( - Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns) -{ - /// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves. - /// Note that the order is important: virtual columns filled by the range reader must go first - injectNonConstVirtualColumns(row_count, block, virtual_columns,task); - injectPartConstVirtualColumns(row_count, block, task, partition_value_type, virtual_columns); -} - Block MergeTreeSelectProcessor::applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info) { if (prewhere_info) @@ -449,8 +320,8 @@ Block MergeTreeSelectProcessor::applyPrewhereActions(Block block, const Prewhere Block MergeTreeSelectProcessor::transformHeader( Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns) { + injectVirtualColumns(block, partition_value_type, virtual_columns); auto transformed = applyPrewhereActions(std::move(block), prewhere_info); - injectVirtualColumns(transformed, 0, nullptr, partition_value_type, virtual_columns); return transformed; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index cf1a6313b51..1d3618e94c2 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -82,7 +82,7 @@ private: }; /// Used for filling header with no rows as well as block with data - static void injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); + static void injectVirtualColumns(Block & block, const DataTypePtr & partition_value_type, const Names & virtual_columns); static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info); /// Sets up range readers corresponding to data readers @@ -104,10 +104,6 @@ private: MergeTreeReadTaskPtr task; /// This step is added when the part has lightweight delete mask PrewhereExprStepPtr lightweight_delete_filter_step; - /// These columns will be filled by the merge tree range reader - Names non_const_virtual_column_names; - /// This header is used for chunks from readFromPart(). - Block header_without_const_virtual_columns; /// A result of getHeader(). A chunk which this header is returned from read(). Block result_header; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 82e9f8fd2db..7281c0ddc82 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -179,9 +179,16 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( mark_ranges.emplace(MarkRanges{MarkRange(0, data_part->getMarksCount())}); reader = data_part->getReader( - columns_for_reader, storage_snapshot, - *mark_ranges, /* uncompressed_cache = */ nullptr, - mark_cache.get(), alter_conversions, reader_settings, {}, {}); + columns_for_reader, + storage_snapshot, + *mark_ranges, + /* read_task_info = */ nullptr, + /* uncompressed_cache = */ nullptr, + mark_cache.get(), + alter_conversions, + reader_settings, + {}, + {}); } Chunk MergeTreeSequentialSource::generate() diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 43e3b0c505a..0621966a813 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -232,6 +232,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction /// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns /// TODO: not sorting for now because the conditions are already sorted by Where Optimizer + /// TODO(amos): _part_offset must come first /// 4. Group conditions with the same set of columns into a single read/compute step std::vector> condition_groups; From b2e6a0d69a6abb5d9a11a0c5311c09324042af87 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Jan 2024 12:48:07 +0800 Subject: [PATCH 002/197] Style fix --- src/Storages/MergeTree/MergeTreeSelectProcessor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 264ceae3655..ad8b280a7e6 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -20,7 +20,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; - extern const int LOGICAL_ERROR; extern const int QUERY_WAS_CANCELLED; } From 1ea48239b211bd080bec47444c4e505176ea8e90 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Jan 2024 16:06:24 +0800 Subject: [PATCH 003/197] Fix test --- .../02235_add_part_offset_virtual_column.reference | 2 ++ .../0_stateless/02235_add_part_offset_virtual_column.sql | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02235_add_part_offset_virtual_column.reference b/tests/queries/0_stateless/02235_add_part_offset_virtual_column.reference index 2455f50b7f2..e88abb35ab4 100644 --- a/tests/queries/0_stateless/02235_add_part_offset_virtual_column.reference +++ b/tests/queries/0_stateless/02235_add_part_offset_virtual_column.reference @@ -50,6 +50,8 @@ SOME GRANULES FILTERED OUT 100002 foo PREWHERE 301408 164953047376 164953047376 +335872 166463369216 166463369216 +301407 164952947376 164952947376 42 10042 20042 diff --git a/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql b/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql index dc8fceddc52..5af6565c03d 100644 --- a/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql +++ b/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql @@ -52,7 +52,7 @@ SELECT _part_offset, foo FROM t_1 where granule == 0 AND _part_offset >= 100000 SELECT 'PREWHERE'; SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere granule == 0 where _part_offset >= 100000; -SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part != '' where granule == 0; -- { serverError 10 } -SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part_offset > 100000 where granule == 0; -- { serverError 10 } +SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part != '' where granule == 0; +SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part_offset > 100000 where granule == 0; SELECT _part_offset FROM t_1 PREWHERE order_0 % 10000 == 42 ORDER BY order_0 LIMIT 3; SELECT _part_offset, foo FROM t_1 PREWHERE order_0 % 10000 == 42 ORDER BY order_0 LIMIT 3; From e0e2e35b9cdeda91f5757f3592d764104bd9ff3f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Jan 2024 20:02:22 +0800 Subject: [PATCH 004/197] Fix some comments --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 81263085f75..968738b4341 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -362,7 +362,7 @@ void MergeTreeRangeReader::ReadResult::shrink(Columns & old_columns, const NumRo } } -/// The main invariant of the data in the read result is that he number of rows is +/// The main invariant of the data in the read result is that the number of rows is /// either equal to total_rows_per_granule (if filter has not been applied) or to the number of /// 1s in the filter (if filter has been applied). void MergeTreeRangeReader::ReadResult::checkInternalConsistency() const @@ -1200,7 +1200,7 @@ Columns MergeTreeRangeReader::continueReadingChain(const ReadResult & result, si if (result.rows_per_granule.empty()) { - /// If zero rows were read on prev step, than there is no more rows to read. + /// If zero rows were read on prev step, there is no more rows to read. /// Last granule may have less rows than index_granularity, so finish reading manually. stream.finish(); return columns; From c3bb97dfdf62b6954d334a0f957343c9e7a7fae5 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Jan 2024 20:02:36 +0800 Subject: [PATCH 005/197] Add tests --- .../03000_virtual_columns_in_prewhere.reference | 1 + .../0_stateless/03000_virtual_columns_in_prewhere.sql | 10 ++++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03000_virtual_columns_in_prewhere.reference create mode 100644 tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql diff --git a/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.reference b/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.reference new file mode 100644 index 00000000000..06b63ea6c2f --- /dev/null +++ b/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.reference @@ -0,0 +1 @@ +0 0 0 diff --git a/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql b/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql new file mode 100644 index 00000000000..9a24f797c13 --- /dev/null +++ b/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql @@ -0,0 +1,10 @@ +drop table if exists x; + +create table x (i int, j int, k int) engine MergeTree order by tuple() settings index_granularity=8192, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; + +insert into x select number, number * 2, number * 3 from numbers(100000); + +-- One granule, (_part_offset (8 bytes) + (4 bytes)) * 8192 + (8 bytes) * 1 = 98312 +select * from x prewhere _part_offset = 0 settings max_bytes_to_read = 98312; + +drop table x; From ce3242170bf492d4f4e61cdd6e4498b0f6cdef57 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Jan 2024 20:08:23 +0800 Subject: [PATCH 006/197] Remove comment --- src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 0621966a813..43e3b0c505a 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -232,7 +232,6 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction /// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns /// TODO: not sorting for now because the conditions are already sorted by Where Optimizer - /// TODO(amos): _part_offset must come first /// 4. Group conditions with the same set of columns into a single read/compute step std::vector> condition_groups; From 18d84696675699b60f4dba6545df892a92f86f78 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Jan 2024 20:09:20 +0800 Subject: [PATCH 007/197] Remove another comment --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 968738b4341..ff86ec01efa 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1041,7 +1041,6 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar return read_result; { - /// Physical columns go first and then some virtual columns follow size_t columns_count = merge_tree_reader->getColumns().size(); Columns columns(read_result.columns.begin(), read_result.columns.begin() + columns_count); merge_tree_reader->fillVirtualColumns(columns, read_result.num_rows); From c0eeeb26a97543e1147b848e2534acb36d4e90d2 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 Jan 2024 00:27:01 +0800 Subject: [PATCH 008/197] Disable sparse column in test --- tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql b/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql index 9a24f797c13..d57db9151b9 100644 --- a/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql +++ b/tests/queries/0_stateless/03000_virtual_columns_in_prewhere.sql @@ -1,6 +1,6 @@ drop table if exists x; -create table x (i int, j int, k int) engine MergeTree order by tuple() settings index_granularity=8192, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; +create table x (i int, j int, k int) engine MergeTree order by tuple() settings index_granularity=8192, index_granularity_bytes = '10Mi', min_bytes_for_wide_part=0, min_rows_for_wide_part=0, ratio_of_defaults_for_sparse_serialization=1; insert into x select number, number * 2, number * 3 from numbers(100000); From c4d84ff9fd0e2b6d41225d2b8fac658f64729013 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 7 Feb 2024 12:08:17 +0800 Subject: [PATCH 009/197] trivial opt on filter --- src/Processors/Transforms/FilterTransform.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index ea4dcff4808..94281301704 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -265,6 +265,7 @@ void FilterTransform::doTransform(Chunk & chunk) { size_t num_rows_before_filtration = chunk.getNumRows(); auto columns = chunk.detachColumns(); + DataTypes types; auto select_final_indices_info = getSelectByFinalIndices(chunk); { @@ -275,6 +276,7 @@ void FilterTransform::doTransform(Chunk & chunk) expression->execute(block, num_rows_before_filtration); columns = block.getColumns(); + types = block.getDataTypes(); } if (constant_filter_description.always_true || on_totals) @@ -325,7 +327,8 @@ void FilterTransform::doTransform(Chunk & chunk) size_t first_non_constant_column = num_columns; for (size_t i = 0; i < num_columns; ++i) { - if (i != filter_column_position && !isColumnConst(*columns[i])) + if (i != filter_column_position && !isColumnConst(*columns[i]) + && removeNullableOrLowCardinalityNullable(types[i])->isValueRepresentedByNumber()) { first_non_constant_column = i; break; From 4921776cc5ba4b342f357deb80d4789d7ddc759f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 19 Feb 2024 11:43:49 +0800 Subject: [PATCH 010/197] change as request --- src/Processors/Transforms/FilterTransform.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 94281301704..4591177850b 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -325,12 +325,18 @@ void FilterTransform::doTransform(Chunk & chunk) * or calculate number of set bytes in the filter. */ size_t first_non_constant_column = num_columns; + size_t min_size_in_memory = std::numeric_limits::max(); for (size_t i = 0; i < num_columns; ++i) { - if (i != filter_column_position && !isColumnConst(*columns[i]) - && removeNullableOrLowCardinalityNullable(types[i])->isValueRepresentedByNumber()) + DataTypePtr type_not_null = removeNullableOrLowCardinalityNullable(types[i]); + if (i != filter_column_position && !isColumnConst(*columns[i]) && type_not_null->isValueRepresentedByNumber()) { - first_non_constant_column = i; + size_t size_in_memory = type_not_null->getSizeOfValueInMemory() + (isNullableOrLowCardinalityNullable(types[i]) ? 1 : 0); + if (size_in_memory < min_size_in_memory) + { + min_size_in_memory = size_in_memory; + first_non_constant_column = i; + } break; } } From a2fa67c2804049f7e82d81bb72733aaa753e84ec Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 20 Feb 2024 10:54:33 +0100 Subject: [PATCH 011/197] Allow local type for local_blob_storage --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 7 +++++-- tests/config/config.d/storage_conf.xml | 6 ++++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index b3626135177..1e8eee1d29c 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -254,7 +254,7 @@ void registerWebObjectStorage(ObjectStorageFactory & factory) void registerLocalObjectStorage(ObjectStorageFactory & factory) { - factory.registerObjectStorageType("local_blob_storage", []( + auto creator = []( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, @@ -267,7 +267,10 @@ void registerLocalObjectStorage(ObjectStorageFactory & factory) /// keys are mapped to the fs, object_key_prefix is a directory also fs::create_directories(object_key_prefix); return std::make_shared(object_key_prefix); - }); + }; + + factory.registerObjectStorageType("local_blob_storage", creator); + factory.registerObjectStorageType("local", creator); } #endif diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 1429dfff724..00d8cb3aea5 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -37,11 +37,13 @@ - local_blob_storage + object_storage + local local_disk/ - local_blob_storage + object_storage + local_blob_storage local_disk_2/ From 2a30d6b9d4371439a6b0f9b440e1def439fa74be Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 20 Feb 2024 10:57:32 +0100 Subject: [PATCH 012/197] Update ObjectStorageFactory.cpp --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 1e8eee1d29c..28e75053b1e 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -206,7 +206,7 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory) #if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) void registerAzureObjectStorage(ObjectStorageFactory & factory) { - factory.registerObjectStorageType("azure_blob_storage", []( + auto creator = []( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, @@ -220,7 +220,9 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) getAzureBlobStorageSettings(config, config_prefix, context), container_name); - }); + }; + factory.registerObjectStorageType("azure_blob_storage", creator); + factory.registerObjectStorageType("azure", creator); } #endif From dd1a25fa35496b145eac43bfc2cc9733e224d4d9 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 20 Feb 2024 10:59:18 +0100 Subject: [PATCH 013/197] Update azure config --- tests/config/config.d/azure_storage_conf.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/config/config.d/azure_storage_conf.xml b/tests/config/config.d/azure_storage_conf.xml index f42bb8e3cf8..412d40111a7 100644 --- a/tests/config/config.d/azure_storage_conf.xml +++ b/tests/config/config.d/azure_storage_conf.xml @@ -2,7 +2,8 @@ - azure_blob_storage + object_storage + azure http://localhost:10000/devstoreaccount1 cont false From 2405115484dcce032e6984e9d63f1ad8075eca5b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 20 Feb 2024 18:05:48 +0100 Subject: [PATCH 014/197] Fix upgrade check --- docker/test/upgrade/run.sh | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index aaba5cc6a8c..d8ba48909b6 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -103,6 +103,16 @@ sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ > /etc/clickhouse-server/config.d/keeper_port.xml.tmp sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml +sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \ + | sed "s|azure>|>azure_blob_storage>|" \ + > /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp +sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml + +sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ + | sed "s|local>|>local_blob_storage>|" \ + > /etc/clickhouse-server/config.d/storage_conf.xml.tmp +sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml + # async_replication setting doesn't exist on some older versions remove_keeper_config "async_replication" "1" From 23f300b581bb904931d4445c562fd0e4c0540ed2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 21 Feb 2024 00:57:35 +0000 Subject: [PATCH 015/197] refactoring of virtual columns --- src/Interpreters/InterpreterCreateQuery.cpp | 54 ++++--- src/Interpreters/InterpreterDeleteQuery.cpp | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 6 +- src/Interpreters/inplaceBlockConversions.cpp | 15 +- src/Interpreters/inplaceBlockConversions.h | 2 +- .../Algorithms/SummingSortedAlgorithm.cpp | 2 +- .../optimizeUseAggregateProjection.cpp | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 15 +- .../Transforms/buildPushingToViewsChain.cpp | 2 +- src/Storages/AlterCommands.cpp | 2 +- src/Storages/BlockNumberColumn.cpp | 23 --- src/Storages/BlockNumberColumn.h | 16 -- src/Storages/ColumnsDescription.cpp | 38 +---- src/Storages/ColumnsDescription.h | 34 ++-- src/Storages/IStorage.cpp | 12 +- src/Storages/IStorage.h | 5 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 85 +++------- src/Storages/MergeTree/IMergeTreeReader.h | 6 +- src/Storages/MergeTree/MergeTask.h | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 15 +- src/Storages/MergeTree/MergeTreeData.cpp | 128 ++++++--------- src/Storages/MergeTree/MergeTreeData.h | 10 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 10 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 15 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 54 ++----- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 - .../MergeTree/MergeTreePrefetchedReadPool.h | 1 - src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 - src/Storages/MergeTree/MergeTreeReadPool.h | 1 - .../MergeTree/MergeTreeReadPoolBase.cpp | 3 - .../MergeTree/MergeTreeReadPoolBase.h | 2 - .../MergeTree/MergeTreeReadPoolInOrder.cpp | 2 - .../MergeTree/MergeTreeReadPoolInOrder.h | 1 - .../MergeTreeReadPoolParallelReplicas.cpp | 2 - .../MergeTreeReadPoolParallelReplicas.h | 1 - ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 2 - ...MergeTreeReadPoolParallelReplicasInOrder.h | 1 - src/Storages/MergeTree/MergeTreeReadTask.h | 2 - .../MergeTree/MergeTreeSelectProcessor.cpp | 108 +++---------- .../MergeTree/MergeTreeSelectProcessor.h | 9 +- .../MergeTree/MergeTreeSequentialSource.cpp | 48 ++++-- src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/MergeTreeVirtualColumns.cpp | 62 ++++++++ src/Storages/MergeTreeVirtualColumns.h | 33 ++++ src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 2 +- src/Storages/StorageInMemoryMetadata.h | 2 +- src/Storages/StorageLog.cpp | 15 +- src/Storages/StorageSnapshot.cpp | 147 +++++++++++------- src/Storages/StorageSnapshot.h | 38 ++--- src/Storages/System/IStorageSystemOneBlock.h | 2 +- src/Storages/System/StorageSystemJemalloc.cpp | 2 +- .../System/StorageSystemZooKeeper.cpp | 2 +- src/Storages/VirtualColumnsDescription.cpp | 96 ++++++++++++ src/Storages/VirtualColumnsDescription.h | 68 ++++++++ 57 files changed, 632 insertions(+), 586 deletions(-) create mode 100644 src/Storages/MergeTreeVirtualColumns.cpp create mode 100644 src/Storages/MergeTreeVirtualColumns.h create mode 100644 src/Storages/VirtualColumnsDescription.cpp create mode 100644 src/Storages/VirtualColumnsDescription.h diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index c491ee30321..c48d025a78f 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -35,7 +35,6 @@ #include #include #include -#include #include #include @@ -890,24 +889,6 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Column {} already exists", backQuoteIfNeed(column.name)); } - /// Check if _row_exists for lightweight delete column in column_lists for merge tree family. - if (create.storage && create.storage->engine && endsWith(create.storage->engine->name, "MergeTree")) - { - auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name); - if (search != all_columns.end()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Cannot create table with column '{}' for *MergeTree engines because it " - "is reserved for lightweight delete feature", - LightweightDeleteDescription::FILTER_COLUMN.name); - - auto search_block_number = all_columns.find(BlockNumberColumn::name); - if (search_block_number != all_columns.end()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Cannot create table with column '{}' for *MergeTree engines because it " - "is reserved for storing block number", - BlockNumberColumn::name); - } - const auto & settings = getContext()->getSettingsRef(); /// Check low cardinality types in creating table if it was not allowed in setting @@ -973,9 +954,24 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } +void validateVirtualColumns(const IStorage & storage) +{ + const auto & virtual_columns = storage.getVirtualsDescription(); + for (const auto & storage_column : storage.getInMemoryMetadataPtr()->getColumns()) + { + auto virtual_desc = virtual_columns.tryGetDescription(storage_column.name); + if (virtual_desc && virtual_desc->kind == VirtualsKind::Persistent) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' for {} engines because it is reserved for persistent virtual column", + storage_column.name, storage.getName()); + } + } +} + namespace { - void checkTemporaryTableEngineName(const String& name) + void checkTemporaryTableEngineName(const String & name) { if (name.starts_with("Replicated") || name.starts_with("Shared") || name == "KeeperMap") throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated, Shared or KeeperMap table engines"); @@ -1549,6 +1545,16 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, addColumnsDescriptionToCreateQueryIfNecessary(query_ptr->as(), res); } + validateVirtualColumns(*res); + + if (!res->supportsDynamicSubcolumns() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns())) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column of type Object, " + "because storage {} doesn't support dynamic subcolumns", + res->getName()); + } + if (!create.attach && getContext()->getSettingsRef().database_replicated_allow_only_replicated_engine) { bool is_replicated_storage = typeid_cast(res.get()) != nullptr; @@ -1598,14 +1604,6 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, /// we can safely destroy the object without a call to "shutdown", because there is guarantee /// that no background threads/similar resources remain after exception from "startup". - if (!res->supportsDynamicSubcolumns() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns())) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Cannot create table with column of type Object, " - "because storage {} doesn't support dynamic subcolumns", - res->getName()); - } - res->startup(); return true; } diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 5c13a1145d1..97ae9649ae8 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -15,7 +15,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 724cfca6a80..3acb8e9e16f 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -133,7 +133,7 @@ Block InterpreterInsertQuery::getSampleBlock( if (auto * window_view = dynamic_cast(table.get())) return window_view->getInputHeader(); else if (no_destination) - return metadata_snapshot->getSampleBlockWithVirtuals(table->getVirtuals()); + return metadata_snapshot->getSampleBlockwithVirtuals(table->getVirtuals()); else return metadata_snapshot->getSampleBlockNonMaterialized(); } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 502b961ced8..4063dadea5e 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -352,7 +352,7 @@ bool MutationsInterpreter::Source::isCompactPart() const return part && part->getType() == MergeTreeDataPartType::Compact; } -static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) +static Names getAvailableColumnswithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) { auto all_columns = metadata_snapshot->getColumns().getNamesOfPhysical(); for (const auto & column : storage.getVirtuals()) @@ -369,7 +369,7 @@ MutationsInterpreter::MutationsInterpreter( : MutationsInterpreter( Source(storage_), metadata_snapshot_, std::move(commands_), - getAvailableColumnsWithVirtuals(metadata_snapshot_, *storage_), + getAvailableColumnswithVirtuals(metadata_snapshot_, *storage_), std::move(context_), std::move(settings_)) { if (settings.can_execute && dynamic_cast(source.getStorage().get())) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index fd8f5b154c4..061156c56db 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include namespace DB @@ -280,7 +280,7 @@ void fillMissingColumns( const NamesAndTypesList & requested_columns, const NamesAndTypesList & available_columns, const NameSet & partially_read_columns, - StorageMetadataPtr metadata_snapshot, size_t block_number) + StorageMetadataPtr metadata_snapshot) { size_t num_columns = requested_columns.size(); if (num_columns != res_columns.size()) @@ -359,14 +359,9 @@ void fillMissingColumns( } else { - if (requested_column->name == BlockNumberColumn::name) - res_columns[i] = type->createColumnConst(num_rows, block_number)->convertToFullColumnIfConst(); - else - /// We must turn a constant column into a full column because the interpreter could infer - /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. - res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); - - + /// We must turn a constant column into a full column because the interpreter could infer + /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. + res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); } } } diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index 7a13a75ec8b..bea44bf6db9 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -46,6 +46,6 @@ void fillMissingColumns( const NamesAndTypesList & requested_columns, const NamesAndTypesList & available_columns, const NameSet & partially_read_columns, - StorageMetadataPtr metadata_snapshot, size_t block_number = 0); + StorageMetadataPtr metadata_snapshot); } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 845cf561968..6253d3058aa 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 534716cc60e..75d8bddcf11 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -431,7 +431,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( { const auto & keys = aggregating.getParams().keys; const auto & aggregates = aggregating.getParams().aggregates; - Block key_virtual_columns = reading.getMergeTreeData().getSampleBlockWithVirtualColumns(); + Block key_virtual_columns = reading.getMergeTreeData().getHeaderWithVirtualsForFilter(); AggregateProjectionCandidates candidates; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index aed053909d7..e2ce36264dd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -280,8 +280,8 @@ ReadFromMergeTree::ReadFromMergeTree( bool enable_parallel_reading) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( storage_snapshot_->getSampleBlockForColumns(real_column_names_), + storage_snapshot_, query_info_.prewhere_info, - data_.getPartitionValueType(), virt_column_names_)}) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) , prepared_parts(std::move(parts_)) @@ -381,7 +381,6 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( reader_settings, required_columns, virt_column_names, - data.getPartitionValueType(), pool_settings, context); @@ -395,7 +394,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( auto algorithm = std::make_unique(i); auto processor = std::make_unique( - pool, std::move(algorithm), data, prewhere_info, + pool, std::move(algorithm), storage_snapshot, prewhere_info, actions_settings, block_size_copy, reader_settings, virt_column_names); auto source = std::make_shared(std::move(processor)); @@ -463,7 +462,6 @@ Pipe ReadFromMergeTree::readFromPool( reader_settings, required_columns, virt_column_names, - data.getPartitionValueType(), pool_settings, context); } @@ -477,7 +475,6 @@ Pipe ReadFromMergeTree::readFromPool( reader_settings, required_columns, virt_column_names, - data.getPartitionValueType(), pool_settings, context); } @@ -496,7 +493,7 @@ Pipe ReadFromMergeTree::readFromPool( auto algorithm = std::make_unique(i); auto processor = std::make_unique( - pool, std::move(algorithm), data, prewhere_info, + pool, std::move(algorithm), storage_snapshot, prewhere_info, actions_settings, block_size_copy, reader_settings, virt_column_names); auto source = std::make_shared(std::move(processor)); @@ -554,7 +551,6 @@ Pipe ReadFromMergeTree::readInOrder( reader_settings, required_columns, virt_column_names, - data.getPartitionValueType(), pool_settings, context); } @@ -570,7 +566,6 @@ Pipe ReadFromMergeTree::readInOrder( reader_settings, required_columns, virt_column_names, - data.getPartitionValueType(), pool_settings, context); } @@ -604,7 +599,7 @@ Pipe ReadFromMergeTree::readInOrder( algorithm = std::make_unique(i); auto processor = std::make_unique( - pool, std::move(algorithm), data, prewhere_info, + pool, std::move(algorithm), storage_snapshot, prewhere_info, actions_settings, block_size, reader_settings, virt_column_names); processor->addPartLevelToChunk(isQueryWithFinal()); @@ -1730,8 +1725,8 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info output_stream = DataStream{.header = MergeTreeSelectProcessor::transformHeader( storage_snapshot->getSampleBlockForColumns(real_column_names), + storage_snapshot, prewhere_info_value, - data.getPartitionValueType(), virt_column_names)}; updateSortDescriptionForOutputStream( diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 91bbf04f327..c10dc4378e8 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -214,7 +214,7 @@ Chain buildPushingToViewsChain( /// If we don't write directly to the destination /// then expect that we're inserting with precalculated virtual columns - auto storage_header = no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()) + auto storage_header = no_destination ? metadata_snapshot->getSampleBlockwithVirtuals(storage->getVirtuals()) : metadata_snapshot->getSampleBlock(); /** TODO This is a very important line. At any insertion into the table one of chains should own lock. diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 766863ed9f9..792f942fcf1 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -32,7 +32,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/BlockNumberColumn.cpp b/src/Storages/BlockNumberColumn.cpp index 8c9e1fd902a..e69de29bb2d 100644 --- a/src/Storages/BlockNumberColumn.cpp +++ b/src/Storages/BlockNumberColumn.cpp @@ -1,23 +0,0 @@ -#include -#include - -namespace DB -{ - -CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); - -CompressionCodecPtr getCompressionCodecForBlockNumberColumn() -{ - std::vector codecs; - codecs.reserve(2); - auto data_bytes_size = BlockNumberColumn::type->getSizeOfValueInMemory(); - codecs.emplace_back(getCompressionCodecDelta(data_bytes_size)); - codecs.emplace_back(CompressionCodecFactory::instance().get("LZ4", {})); - return std::make_shared(codecs); -} - -const String BlockNumberColumn::name = "_block_number"; -const DataTypePtr BlockNumberColumn::type = std::make_shared(); -const CompressionCodecPtr BlockNumberColumn::compression_codec = getCompressionCodecForBlockNumberColumn(); - -} diff --git a/src/Storages/BlockNumberColumn.h b/src/Storages/BlockNumberColumn.h index fffa68bfd49..e69de29bb2d 100644 --- a/src/Storages/BlockNumberColumn.h +++ b/src/Storages/BlockNumberColumn.h @@ -1,16 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -struct BlockNumberColumn -{ - static const String name; - static const DataTypePtr type; - static const CompressionCodecPtr compression_codec; -}; - -} diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 40d657f3deb..6ee87297cab 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -31,15 +31,12 @@ #include #include #include -#include +#include namespace DB { -CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); - - namespace ErrorCodes { extern const int NO_SUCH_COLUMN_IN_TABLE; @@ -563,6 +560,12 @@ const ColumnDescription & ColumnsDescription::get(const String & column_name) co return *it; } +const ColumnDescription * ColumnsDescription::tryGet(const String & column_name) const +{ + auto it = columns.get<1>().find(column_name); + return it == columns.get<1>().end() ? nullptr : &(*it); +} + static GetColumnsOptions::Kind defaultKindToGetKind(ColumnDefaultKind kind) { switch (kind) @@ -789,33 +792,6 @@ bool ColumnsDescription::hasCompressionCodec(const String & column_name) const return it != columns.get<1>().end() && it->codec != nullptr; } -CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const -{ - const auto it = columns.get<1>().find(column_name); - - if (it == columns.get<1>().end() || !it->codec) - return default_codec; - - return CompressionCodecFactory::instance().get(it->codec, it->type, default_codec); -} - -CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name) const -{ - assert (column_name != BlockNumberColumn::name); - return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec()); -} - -ASTPtr ColumnsDescription::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const -{ - assert (column_name != BlockNumberColumn::name); - const auto it = columns.get<1>().find(column_name); - - if (it == columns.get<1>().end() || !it->codec) - return default_codec->getFullCodecDesc(); - - return it->codec; -} - ColumnsDescription::ColumnTTLs ColumnsDescription::getColumnTTLs() const { ColumnTTLs ret; diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 5f3e75dc38e..cc6c7d0e1b9 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -29,6 +29,14 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +enum class VirtualsKind : UInt8 +{ + None = 0, + Ephemeral = 1, + Persistent = 2, + All = Ephemeral | Persistent, +}; + struct GetColumnsOptions { enum Kind : UInt8 @@ -52,9 +60,15 @@ struct GetColumnsOptions return *this; } - GetColumnsOptions & withVirtuals(bool value = true) + GetColumnsOptions & withVirtuals(VirtualsKind value = VirtualsKind::All) { - with_virtuals = value; + virtuals_kind = value; + return *this; + } + + GetColumnsOptions & withPersistentVirtuals(bool value = true) + { + with_persistent_virtuals = value; return *this; } @@ -64,17 +78,12 @@ struct GetColumnsOptions return *this; } - GetColumnsOptions & withSystemColumns(bool value = true) - { - with_system_columns = value; - return *this; - } - Kind kind; + VirtualsKind virtuals_kind = VirtualsKind::None; + bool with_subcolumns = false; - bool with_virtuals = false; + bool with_persistent_virtuals = false; bool with_extended_objects = false; - bool with_system_columns = false; }; /// Description of a single table column (in CREATE TABLE for example). @@ -161,6 +170,7 @@ public: bool hasNested(const String & column_name) const; bool hasSubcolumn(const String & column_name) const; const ColumnDescription & get(const String & column_name) const; + const ColumnDescription * tryGet(const String & column_name) const; template void modify(const String & column_name, F && f) @@ -214,9 +224,6 @@ public: /// Does column has non default specified compression codec bool hasCompressionCodec(const String & column_name) const; - CompressionCodecPtr getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; - CompressionCodecPtr getCodecOrDefault(const String & column_name) const; - ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; String toString() const; static ColumnsDescription parse(const String & str); @@ -270,4 +277,5 @@ private: /// don't have strange constructions in default expression like SELECT query or /// arrayJoin function. Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context); + } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 85ef6a0bb35..957051cd409 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -237,11 +237,21 @@ void IStorage::renameInMemory(const StorageID & new_table_id) storage_id = new_table_id; } -NamesAndTypesList IStorage::getVirtuals() const +VirtualColumnsDescription IStorage::getVirtualsDescription() const { return {}; } +NamesAndTypesList IStorage::getVirtuals() const +{ + return getVirtualsDescription().getNamesAndTypesList(); +} + +Block IStorage::getVirtualsHeader() const +{ + return getVirtualsDescription().getSampleBlock(); +} + Names IStorage::getAllRegisteredNames() const { Names result; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4fa6bfdd617..94b089ef613 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -17,6 +17,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #include @@ -226,7 +227,9 @@ public: /// virtual column will be overridden and inaccessible. /// /// By default return empty list of columns. - virtual NamesAndTypesList getVirtuals() const; + virtual VirtualColumnsDescription getVirtualsDescription() const; + virtual NamesAndTypesList getVirtuals() const; /// TODO: make non virtual. + Block getVirtualsHeader() const; Names getAllRegisteredNames() const override; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 7ff69f7cc4b..f774d3a387a 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -1,6 +1,7 @@ #include #include -#include +#include +#include #include #include #include @@ -22,9 +23,9 @@ namespace namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NO_SUCH_COLUMN_IN_TABLE; } - IMergeTreeReader::IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, @@ -56,16 +57,10 @@ IMergeTreeReader::IMergeTreeReader( columns_to_read.reserve(requested_columns.size()); serializations.reserve(requested_columns.size()); - size_t pos = 0; for (const auto & column : requested_columns) { columns_to_read.emplace_back(getColumnInPart(column)); serializations.emplace_back(getSerializationInPart(column)); - - if (read_task_info && read_task_info->virt_column_names.contains(column.name)) - virt_column_pos_to_name.emplace(pos, column.name); - - ++pos; } } @@ -76,72 +71,36 @@ const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints() void IMergeTreeReader::fillVirtualColumns(Columns & columns, size_t rows) const { - if (std::all_of( - virt_column_pos_to_name.begin(), - virt_column_pos_to_name.end(), - [&columns](auto & elem) - { - chassert(elem.first < columns.size()); - return columns[elem.first] != nullptr; - })) - return; - chassert(read_task_info != nullptr); const IMergeTreeDataPart * part = read_task_info->data_part.get(); if (part->isProjectionPart()) part = part->getParentPart(); - for (auto [pos, name] : virt_column_pos_to_name) - { - auto & column = columns[pos]; + const auto & storage_columns = storage_snapshot->getMetadataForQuery()->getColumns(); + const auto & virtual_columns = storage_snapshot->virtual_columns; - if (column != nullptr) + auto it = requested_columns.begin(); + for (size_t pos = 0; pos < columns.size(); ++pos, ++it) + { + if (columns[pos] || storage_columns.has(it->name)) continue; - if (name == "_part_offset") - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} must have been filled by part reader", name); - } - else if (name == LightweightDeleteDescription::FILTER_COLUMN.name) - { - /// If _row_exists column isn't present in the part then fill it here with 1s - column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumnConst(rows, 1)->convertToFullColumnIfConst(); - } - else if (name == BlockNumberColumn::name) - { - column = BlockNumberColumn::type->createColumnConst(rows, part->info.min_block)->convertToFullColumnIfConst(); - } - else if (name == "_part") - { - column = DataTypeLowCardinality{std::make_shared()} - .createColumnConst(rows, part->name) - ->convertToFullColumnIfConst(); - } - else if (name == "_part_index") - { - column = DataTypeUInt64().createColumnConst(rows, read_task_info->part_index_in_query)->convertToFullColumnIfConst(); - } - else if (name == "_part_uuid") - { - column = DataTypeUUID().createColumnConst(rows, part->uuid)->convertToFullColumnIfConst(); - } - else if (name == "_partition_id") - { - column = DataTypeLowCardinality{std::make_shared()} - .createColumnConst(rows, part->info.partition_id) - ->convertToFullColumnIfConst(); - } - else if (name == "_partition_value") - { - column = read_task_info->partition_value_type - ->createColumnConst(rows, Tuple(part->partition.value.begin(), part->partition.value.end())) - ->convertToFullColumnIfConst(); - } + auto virtual_column = virtual_columns.tryGet(it->name); + if (!virtual_column) + continue; + + if (!it->type->equals(*virtual_column->type)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Data type for virtual column {} mismatched. Requested type: {}, Virtual column type: {}", + it->name, it->type->getName(), virtual_column->type->getName()); + + auto field = getFieldForConstVirtualColumn(it->name, *part, read_task_info->part_index_in_query); + columns[pos] = virtual_column->type->createColumnConst(rows, field)->convertToFullColumnIfConst(); } } -void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number) const +void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const { try { @@ -150,7 +109,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e res_columns, num_rows, Nested::convertToSubcolumns(requested_columns), Nested::convertToSubcolumns(available_columns), - partially_read_columns, storage_snapshot->metadata, block_number); + partially_read_columns, storage_snapshot->metadata); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index cd4417265fa..3fe9853fced 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -2,6 +2,7 @@ #include #include +#include "DataTypes/Serializations/ISerialization.h" #include #include #include @@ -49,7 +50,7 @@ public: /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// num_rows is needed in case if all res_columns are nullptr. - void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number = 0) const; + void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const; /// Evaluate defaulted columns if necessary. void evaluateMissingDefaults(Block additional_columns, Columns & res_columns) const; @@ -120,9 +121,6 @@ private: /// Shared information required for reading. MergeTreeReadTaskInfoPtr read_task_info; - - /// Map of positions in requested_columns which are virtual columns to their names. - std::map virt_column_pos_to_name; }; } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 7fb4797e482..28a3c671914 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -15,7 +15,7 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index fc6599b2851..8eb714c7e24 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "Storages/ColumnsDescription.h" #include #include #include @@ -106,10 +107,8 @@ NameSet injectRequiredColumns( auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) .withExtendedObjects() - .withSystemColumns(); - - if (with_subcolumns) - options.withSubcolumns(); + .withVirtuals(VirtualsKind::Persistent) + .withSubcolumns(with_subcolumns); auto virtuals_options = GetColumnsOptions(GetColumnsOptions::None).withVirtuals(); @@ -283,12 +282,8 @@ MergeTreeReadTaskColumns getReadTaskColumns( MergeTreeReadTaskColumns result; auto options = GetColumnsOptions(GetColumnsOptions::All) .withExtendedObjects() - .withSystemColumns(); - - if (with_subcolumns) - options.withSubcolumns(); - - options.withVirtuals(); + .withVirtuals() + .withSubcolumns(with_subcolumns); bool has_part_offset = std::find(required_columns.begin(), required_columns.end(), "_part_offset") != required_columns.end(); NameSet columns_from_previous_steps; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c8262914702..a3a4cb9619f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,6 +8,7 @@ #include #include #include +#include "Common/logger_useful.h" #include #include #include @@ -22,6 +23,7 @@ #include #include #include +#include "Storages/ProjectionsDescription.h" #include #include #include @@ -67,7 +69,7 @@ #include #include #include -#include +#include #include #include #include @@ -1001,73 +1003,38 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat /// TODO Checks for Graphite mode. } +const Names MergeTreeData::virtuals_useful_for_filter = {"_part", "_partition_id", "_part_uuid", "_partition_value"}; -DataTypePtr MergeTreeData::getPartitionValueType() const +Block MergeTreeData::getHeaderWithVirtualsForFilter() const { - DataTypePtr partition_value_type; - auto partition_types = getInMemoryMetadataPtr()->partition_key.sample_block.getDataTypes(); - if (partition_types.empty()) - partition_value_type = std::make_shared(); - else - partition_value_type = std::make_shared(std::move(partition_types)); - return partition_value_type; + Block header; + const auto & virtuals_desc = getVirtualsDescription(); + for (const auto & name : virtuals_useful_for_filter) + if (auto column = virtuals_desc.tryGet(name)) + header.insert({column->type->createColumn(), column->type, name}); + return header; } - -Block MergeTreeData::getSampleBlockWithVirtualColumns() const +Block MergeTreeData::getBlockWithVirtualsForFilter(const MergeTreeData::DataPartsVector & parts, bool ignore_empty) const { - DataTypePtr partition_value_type = getPartitionValueType(); - return { - ColumnWithTypeAndName( - DataTypeLowCardinality{std::make_shared()}.createColumn(), - std::make_shared(std::make_shared()), - "_part"), - ColumnWithTypeAndName( - DataTypeLowCardinality{std::make_shared()}.createColumn(), - std::make_shared(std::make_shared()), - "_partition_id"), - ColumnWithTypeAndName(ColumnUUID::create(), std::make_shared(), "_part_uuid"), - ColumnWithTypeAndName(partition_value_type->createColumn(), partition_value_type, "_partition_value")}; -} + auto block = getHeaderWithVirtualsForFilter(); - -Block MergeTreeData::getBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, bool one_part, bool ignore_empty) const -{ - auto block = getSampleBlockWithVirtualColumns(); - MutableColumns columns = block.mutateColumns(); - - auto & part_column = columns[0]; - auto & partition_id_column = columns[1]; - auto & part_uuid_column = columns[2]; - auto & partition_value_column = columns[3]; - - bool has_partition_value = typeid_cast(partition_value_column.get()); for (const auto & part_or_projection : parts) { if (ignore_empty && part_or_projection->isEmpty()) continue; - const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); - part_column->insert(part->name); - partition_id_column->insert(part->info.partition_id); - part_uuid_column->insert(part->uuid); - Tuple tuple(part->partition.value.begin(), part->partition.value.end()); - if (has_partition_value) - partition_value_column->insert(tuple); - if (one_part) + const auto * part = part_or_projection->isProjectionPart() + ? part_or_projection->getParentPart() + : part_or_projection.get(); + + for (auto & column : block) { - part_column = ColumnConst::create(std::move(part_column), 1); - partition_id_column = ColumnConst::create(std::move(partition_id_column), 1); - part_uuid_column = ColumnConst::create(std::move(part_uuid_column), 1); - if (has_partition_value) - partition_value_column = ColumnConst::create(std::move(partition_value_column), 1); - break; + auto field = getFieldForConstVirtualColumn(column.name, *part, 0); + column.column->assumeMutableRef().insert(field); } } - block.setColumns(std::move(columns)); - if (!has_partition_value) - block.erase("_partition_value"); return block; } @@ -1076,13 +1043,14 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const { if (parts.empty()) - return 0u; + return 0; + auto metadata_snapshot = getInMemoryMetadataPtr(); - Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, true /* one_part */); + auto virtual_columns_block = getBlockWithVirtualsForFilter({parts[0]}); auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr); - // Generate valid expressions for filtering + /// Generate valid expressions for filtering bool valid = true; for (const auto * input : filter_dag->getInputs()) if (!virtual_columns_block.has(input->result_name)) @@ -1095,7 +1063,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( std::unordered_set part_values; if (valid) { - virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */); + virtual_columns_block = getBlockWithVirtualsForFilter(parts); VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) @@ -6653,14 +6621,6 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( const auto & primary_key_max_column_name = metadata_snapshot->minmax_count_projection->primary_key_max_column_name; NameSet required_columns_set(required_columns.begin(), required_columns.end()); - if (required_columns_set.contains("_partition_value") && !typeid_cast(getPartitionValueType().get())) - { - throw Exception( - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, - "Missing column `_partition_value` because there is no partition column in table {}", - getStorageID().getTableName()); - } - if (!primary_key_max_column_name.empty()) need_primary_key_max_column = required_columns_set.contains(primary_key_max_column_name); @@ -6686,11 +6646,11 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( }; Block virtual_columns_block; - auto virtual_block = getSampleBlockWithVirtualColumns(); + auto virtual_block = getHeaderWithVirtualsForFilter(); bool has_virtual_column = std::any_of(required_columns.begin(), required_columns.end(), [&](const auto & name) { return virtual_block.has(name); }); if (has_virtual_column || filter_dag) { - virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */); + virtual_columns_block = getBlockWithVirtualsForFilter(parts, /*ignore_empty=*/ true); if (virtual_columns_block.rows() == 0) return {}; } @@ -7960,19 +7920,29 @@ AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(MergeTreeDataPartP return result; } -NamesAndTypesList MergeTreeData::getVirtuals() const +VirtualColumnsDescription MergeTreeData::getVirtualsDescription() const { - return NamesAndTypesList{ - NameAndTypePair("_part", std::make_shared(std::make_shared())), - NameAndTypePair("_part_index", std::make_shared()), - NameAndTypePair("_part_uuid", std::make_shared()), - NameAndTypePair("_partition_id", std::make_shared(std::make_shared())), - NameAndTypePair("_partition_value", getPartitionValueType()), - NameAndTypePair("_sample_factor", std::make_shared()), - NameAndTypePair("_part_offset", std::make_shared()), - LightweightDeleteDescription::FILTER_COLUMN, - NameAndTypePair(BlockNumberColumn::name, BlockNumberColumn::type), - }; + VirtualColumnsDescription desc; + auto low_cardinality_type = std::make_shared(std::make_shared()); + auto metadata_snapshot = getInMemoryMetadataPtr(); + + desc.addEphemeral("_part", low_cardinality_type, ""); + desc.addEphemeral("_part_index", std::make_shared(), ""); + desc.addEphemeral("_part_uuid", std::make_shared(), ""); + desc.addEphemeral("_partition_id", low_cardinality_type, ""); + desc.addEphemeral("_sample_factor", std::make_shared(), ""); + desc.addEphemeral("_part_offset", std::make_shared(), ""); + + if (metadata_snapshot->hasPartitionKey()) + { + auto partition_types = getInMemoryMetadataPtr()->partition_key.sample_block.getDataTypes(); + desc.addEphemeral("_partition_value", std::make_shared(std::move(partition_types)), ""); + } + + desc.addPersistent(LightweightDeleteDescription::FILTER_COLUMN.name, LightweightDeleteDescription::FILTER_COLUMN.type, nullptr, ""); + desc.addPersistent(BlockNumberColumn::name, BlockNumberColumn::type, BlockNumberColumn::codec, ""); + + return desc; } size_t MergeTreeData::getTotalMergesWithTTLInMergeList() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4ad440dae00..c3c1312d429 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -444,7 +444,7 @@ public: bool supportsTrivialCountOptimization() const override { return !hasLightweightDeletedMask(); } - NamesAndTypesList getVirtuals() const override; + VirtualColumnsDescription getVirtualsDescription() const override; /// Snapshot for MergeTree contains the current set of data parts /// at the moment of the start of query. @@ -993,15 +993,13 @@ public: void removeQueryId(const String & query_id) const; void removeQueryIdNoLock(const String & query_id) const TSA_REQUIRES(query_id_set_mutex); - /// Return the partition expression types as a Tuple type. Return DataTypeUInt8 if partition expression is empty. - DataTypePtr getPartitionValueType() const; + static const Names virtuals_useful_for_filter; /// Construct a sample block of virtual columns. - Block getSampleBlockWithVirtualColumns() const; + Block getHeaderWithVirtualsForFilter() const; /// Construct a block consisting only of possible virtual columns for part pruning. - /// If one_part is true, fill in at most one part. - Block getBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, bool one_part, bool ignore_empty = false) const; + Block getBlockWithVirtualsForFilter(const MergeTreeData::DataPartsVector & parts, bool ignore_empty = false) const; /// In merge tree we do inserts with several steps. One of them: /// X. write part to temporary directory with some temp name diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 7baba26d15c..ee9fa30d98d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index b05b4584259..46c4338ab90 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -5,8 +5,6 @@ namespace DB { - CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -55,14 +53,10 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( marks_source_hashing = std::make_unique(*marks_compressor); } - const auto & storage_columns = metadata_snapshot->getColumns(); + auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { - ASTPtr compression; - if (column.name == BlockNumberColumn::name) - compression = BlockNumberColumn::compression_codec->getFullCodecDesc(); - else - compression = storage_columns.getCodecDescOrDefault(column.name, default_codec); + auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); addStreams(column, compression); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 9d373504473..3a646e0b85d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -6,12 +6,11 @@ #include #include #include -#include +#include #include namespace DB { - CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); namespace ErrorCodes { @@ -91,15 +90,11 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( indices_to_recalc_, stats_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) { - const auto & columns = metadata_snapshot->getColumns(); - for (const auto & it : columns_list) + auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); + for (const auto & column : columns_list) { - ASTPtr compression; - if (it.name == BlockNumberColumn::name) - compression = BlockNumberColumn::compression_codec->getFullCodecDesc(); - else - compression = columns.getCodecDescOrDefault(it.name, default_codec); - addStreams(it, compression); + auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + addStreams(column, compression); } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a76d370d057..df1176a9ea5 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -46,7 +46,7 @@ #include #include -#include +#include #include namespace CurrentMetrics @@ -483,12 +483,13 @@ std::optional> MergeTreeDataSelectExecutor::filterPar { if (!filter_dag) return {}; - auto sample = data.getSampleBlockWithVirtualColumns(); + + auto sample = data.getHeaderWithVirtualsForFilter(); auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample); if (!dag) return {}; - auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); + auto virtual_columns_block = data.getBlockWithVirtualsForFilter(parts); VirtualColumnUtils::filterBlockWithDAG(dag, virtual_columns_block, context); return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); } @@ -876,54 +877,17 @@ static void selectColumnNames( bool & sample_factor_column_queried) { sample_factor_column_queried = false; + const auto & virtual_columns = data.getVirtualsDescription(); - for (const String & name : column_names_to_return) + for (const auto & name : column_names_to_return) { - if (name == "_part") + if (virtual_columns.has(name)) { - virt_column_names.push_back(name); - } - else if (name == "_part_index") - { - virt_column_names.push_back(name); - } - else if (name == "_partition_id") - { - virt_column_names.push_back(name); - } - else if (name == "_part_offset") - { - virt_column_names.push_back(name); - } - else if (name == LightweightDeleteDescription::FILTER_COLUMN.name) - { - virt_column_names.push_back(name); - } - else if (name == BlockNumberColumn::name) - { - virt_column_names.push_back(name); - } - else if (name == "_part_uuid") - { - virt_column_names.push_back(name); - } - else if (name == "_partition_value") - { - if (!typeid_cast(data.getPartitionValueType().get())) - { - throw Exception( - ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, - "Missing column `_partition_value` because there is no partition column in table {}", - data.getStorageID().getTableName()); - } + if (name == "_sample_factor") + sample_factor_column_queried = true; virt_column_names.push_back(name); } - else if (name == "_sample_factor") - { - sample_factor_column_queried = true; - virt_column_names.push_back(name); - } else { real_column_names.push_back(name); diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 701b8fbf528..47c2fe07bb4 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -114,7 +114,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -125,7 +124,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( reader_settings_, column_names_, virtual_column_names_, - partition_value_type_, settings_, context_) , WithContext(context_) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 1a700abb009..b1335fd2774 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -24,7 +24,6 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index dbd27aebc21..8ed7a9d8707 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -40,7 +40,6 @@ MergeTreeReadPool::MergeTreeReadPool( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -51,7 +50,6 @@ MergeTreeReadPool::MergeTreeReadPool( reader_settings_, column_names_, virtual_column_names_, - partition_value_type_, settings_, context_) , min_marks_for_concurrent_read(pool_settings.min_marks_for_concurrent_read) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 9da75648386..e45ccad912f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -32,7 +32,6 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index d5811c33f00..95e42d39cf7 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -13,7 +13,6 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & pool_settings_, const ContextPtr & context_) : parts_ranges(std::move(parts_)) @@ -23,7 +22,6 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( , reader_settings(reader_settings_) , column_names(column_names_) , virtual_column_names(virtual_column_names_) - , partition_value_type(partition_value_type_) , pool_settings(pool_settings_) , owned_mark_cache(context_->getGlobalContext()->getMarkCache()) , owned_uncompressed_cache(pool_settings_.use_uncompressed_cache ? context_->getGlobalContext()->getUncompressedCache() : nullptr) @@ -69,7 +67,6 @@ void MergeTreeReadPoolBase::fillPerPartInfos() /*with_subcolumns=*/true); read_task_info.virt_column_names = {virtual_column_names.begin(), virtual_column_names.end()}; - read_task_info.partition_value_type = partition_value_type; if (pool_settings.preferred_block_size_bytes > 0) { diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 3aa9eb8670e..a0c6a30fb63 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -29,7 +29,6 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); @@ -44,7 +43,6 @@ protected: const MergeTreeReaderSettings reader_settings; const Names column_names; const Names virtual_column_names; - const DataTypePtr partition_value_type; const PoolSettings pool_settings; const MarkCachePtr owned_mark_cache; const UncompressedCachePtr owned_uncompressed_cache; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index 692e45993c7..1b621ad5055 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -18,7 +18,6 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -29,7 +28,6 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( reader_settings_, column_names_, virtual_column_names_, - partition_value_type_, settings_, context_) , has_limit_below_one_block(has_limit_below_one_block_) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index de7457dfab8..d9cc1ba4984 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -17,7 +17,6 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index fb14dfe9a6e..47436ed1407 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -19,7 +19,6 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -30,7 +29,6 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( reader_settings_, column_names_, virtual_column_names_, - partition_value_type_, settings_, context_) , extension(std::move(extension_)) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index 9a8b0e50c04..6a548dffe37 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -17,7 +17,6 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index d242d1e81fe..a822a517933 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -18,7 +18,6 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -29,7 +28,6 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd reader_settings_, column_names_, virtual_column_names_, - partition_value_type_, settings_, context_) , extension(std::move(extension_)) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 09935e1be2f..3e5f8f5dfba 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -18,7 +18,6 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, - const DataTypePtr & partition_value_type_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadTask.h b/src/Storages/MergeTree/MergeTreeReadTask.h index 7e935f5d28d..21ec19ee033 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.h +++ b/src/Storages/MergeTree/MergeTreeReadTask.h @@ -62,8 +62,6 @@ struct MergeTreeReadTaskInfo MergeTreeReadTaskColumns task_columns; /// Virtual column names to read NameSet virt_column_names; - /// For `partition_value` virtual column - DataTypePtr partition_value_type; /// Shared initialized size predictor. It is copied for each new task. MergeTreeBlockSizePredictorPtr shared_size_predictor; }; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index ad8b280a7e6..8e089741cab 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -6,12 +6,13 @@ #include #include #include +#include "Storages/StorageSnapshot.h" #include #include #include #include #include -#include +#include #include namespace DB @@ -21,12 +22,13 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; extern const int QUERY_WAS_CANCELLED; + extern const int NO_SUCH_COLUMN_IN_TABLE; } MergeTreeSelectProcessor::MergeTreeSelectProcessor( MergeTreeReadPoolPtr pool_, MergeTreeSelectAlgorithmPtr algorithm_, - const MergeTreeData & storage_, + const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, const MergeTreeReadTask::BlockSizeParams & block_size_params_, @@ -34,13 +36,13 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( const Names & virt_column_names_) : pool(std::move(pool_)) , algorithm(std::move(algorithm_)) + , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) , actions_settings(actions_settings_) , prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps)) , reader_settings(reader_settings_) , block_size_params(block_size_params_) , virt_column_names(virt_column_names_) - , partition_value_type(storage_.getPartitionValueType()) { if (reader_settings.apply_deleted_mask) { @@ -58,7 +60,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( } result_header = pool->getHeader(); - injectVirtualColumns(result_header, partition_value_type, virt_column_names); + injectVirtualColumns(result_header, storage_snapshot, virt_column_names); result_header = applyPrewhereActions(result_header, prewhere_info); if (!prewhere_actions.steps.empty()) @@ -179,89 +181,20 @@ void MergeTreeSelectProcessor::initializeRangeReaders() task->initializeRangeReaders(all_prewhere_actions); } - -namespace +void MergeTreeSelectProcessor::injectVirtualColumns( + Block & block, + const StorageSnapshotPtr & storage_snapshot, + const Names & virtual_columns) { - struct VirtualColumnsInserter - { - explicit VirtualColumnsInserter(Block & block_) : block(block_) {} - - void insertUInt8Column(const ColumnPtr & column, const String & name) - { - block.insert({column, std::make_shared(), name}); - } - - void insertUInt64Column(const ColumnPtr & column, const String & name) - { - block.insert({column, std::make_shared(), name}); - } - - void insertUUIDColumn(const ColumnPtr & column, const String & name) - { - block.insert({column, std::make_shared(), name}); - } - - void insertLowCardinalityColumn(const ColumnPtr & column, const String & name) - { - block.insert({column, std::make_shared(std::make_shared()), name}); - } - - void insertPartitionValueColumn(const DataTypePtr & partition_value_type, const String & name) - { - ColumnPtr column = partition_value_type->createColumn(); - block.insert({column, partition_value_type, name}); - } - - Block & block; - }; -} - -void MergeTreeSelectProcessor::injectVirtualColumns(Block & block, const DataTypePtr & partition_value_type, const Names & virtual_columns) -{ - VirtualColumnsInserter inserter(block); - - /// add virtual columns - /// Except _sample_factor, which is added from the outside. for (const auto & virtual_column_name : virtual_columns) { - if (virtual_column_name == "_part_offset") - { - inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name); - } - else if (virtual_column_name == LightweightDeleteDescription::FILTER_COLUMN.name) - { - ColumnPtr column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumn(); - inserter.insertUInt8Column(column, virtual_column_name); - } - else if (virtual_column_name == BlockNumberColumn::name) - { - ColumnPtr column = BlockNumberColumn::type->createColumn(); - inserter.insertUInt64Column(column, virtual_column_name); - } - else if (virtual_column_name == "_part") - { - ColumnPtr column = DataTypeLowCardinality{std::make_shared()}.createColumn(); - inserter.insertLowCardinalityColumn(column, virtual_column_name); - } - else if (virtual_column_name == "_part_index") - { - ColumnPtr column = DataTypeUInt64().createColumn(); - inserter.insertUInt64Column(column, virtual_column_name); - } - else if (virtual_column_name == "_part_uuid") - { - ColumnPtr column = DataTypeUUID().createColumn(); - inserter.insertUUIDColumn(column, virtual_column_name); - } - else if (virtual_column_name == "_partition_id") - { - ColumnPtr column = DataTypeLowCardinality{std::make_shared()}.createColumn(); - inserter.insertLowCardinalityColumn(column, virtual_column_name); - } - else if (virtual_column_name == "_partition_value") - { - inserter.insertPartitionValueColumn(partition_value_type, virtual_column_name); - } + auto column = storage_snapshot->virtual_columns.tryGet(virtual_column_name); + if (!column) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no virtual column {} in table {}", + virtual_column_name, storage_snapshot->storage.getStorageID().getNameForLogs()); + + block.insert({column->type->createColumn(), column->type, column->name}); } } @@ -317,9 +250,12 @@ Block MergeTreeSelectProcessor::applyPrewhereActions(Block block, const Prewhere } Block MergeTreeSelectProcessor::transformHeader( - Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns) + Block block, + const StorageSnapshotPtr & storage_snapshot, + const PrewhereInfoPtr & prewhere_info, + const Names & virtual_columns) { - injectVirtualColumns(block, partition_value_type, virtual_columns); + injectVirtualColumns(block, storage_snapshot, virtual_columns); auto transformed = applyPrewhereActions(std::move(block), prewhere_info); return transformed; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index a3cfec1774d..9b399003909 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -5,6 +5,7 @@ #include #include #include +#include "Storages/StorageSnapshot.h" namespace DB @@ -41,7 +42,7 @@ public: MergeTreeSelectProcessor( MergeTreeReadPoolPtr pool_, MergeTreeSelectAlgorithmPtr algorithm_, - const MergeTreeData & storage_, + const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, const MergeTreeReadTask::BlockSizeParams & block_size_params_, @@ -52,8 +53,8 @@ public: static Block transformHeader( Block block, + const StorageSnapshotPtr & storage_snapshot, const PrewhereInfoPtr & prewhere_info, - const DataTypePtr & partition_value_type, const Names & virtual_columns); Block getHeader() const { return result_header; } @@ -82,7 +83,7 @@ private: }; /// Used for filling header with no rows as well as block with data - static void injectVirtualColumns(Block & block, const DataTypePtr & partition_value_type, const Names & virtual_columns); + static void injectVirtualColumns(Block & block, const StorageSnapshotPtr & storage_snapshot, const Names & virtual_columns); static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info); /// Sets up range readers corresponding to data readers @@ -90,6 +91,7 @@ private: const MergeTreeReadPoolPtr pool; const MergeTreeSelectAlgorithmPtr algorithm; + const StorageSnapshotPtr storage_snapshot; const PrewhereInfoPtr prewhere_info; const ExpressionActionsSettings actions_settings; @@ -98,7 +100,6 @@ private: const MergeTreeReaderSettings reader_settings; const MergeTreeReadTask::BlockSizeParams block_size_params; const Names virt_column_names; - const DataTypePtr partition_value_type; /// Current task to read from. MergeTreeReadTaskPtr task; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 8a326d64fc7..73689e15cd4 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -16,6 +17,7 @@ namespace DB { + namespace ErrorCodes { extern const int MEMORY_LIMIT_EXCEEDED; @@ -55,7 +57,6 @@ protected: Chunk generate() override; private: - const MergeTreeData & storage; StorageSnapshotPtr storage_snapshot; @@ -86,7 +87,6 @@ private: void finish(); }; - MergeTreeSequentialSource::MergeTreeSequentialSource( MergeTreeSequentialSourceType type, const MergeTreeData & storage_, @@ -136,10 +136,8 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( { auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) .withExtendedObjects() - .withSystemColumns(); - - if (storage.supportsSubcolumns()) - options.withSubcolumns(); + .withVirtuals(VirtualsKind::Persistent) + .withSubcolumns(storage.supportsSubcolumns()); columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read); } @@ -193,6 +191,38 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( {}); } +static void fillBlockNumberColumns( + Columns & res_columns, + const NamesAndTypesList & columns_list, + UInt64 block_number, + UInt64 block_offset, + UInt64 num_rows) +{ + chassert(res_columns.size() == columns_list.size()); + + auto it = columns_list.begin(); + for (size_t i = 0; i < res_columns.size(); ++i, ++it) + { + if (res_columns[i]) + continue; + + if (it->name == BlockNumberColumn::name) + { + res_columns[i] = BlockNumberColumn::type->createColumnConst(num_rows, block_number)->convertToFullColumnIfConst(); + } + else if (it->name == BlockOffsetColumn::name) + { + auto column = BlockOffsetColumn::type->createColumn(); + auto & block_offset_data = assert_cast(*column).getData(); + + block_offset_data.resize(num_rows); + std::iota(block_offset_data.begin(), block_offset_data.end(), block_offset); + + res_columns[i] = std::move(column); + } + } +} + Chunk MergeTreeSequentialSource::generate() try { @@ -211,16 +241,16 @@ try if (rows_read) { + fillBlockNumberColumns(columns, sample, data_part->info.min_block, current_row, rows_read); + current_row += rows_read; current_mark += (rows_to_read == rows_read); bool should_evaluate_missing_defaults = false; - reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read, data_part->info.min_block); + reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); if (should_evaluate_missing_defaults) - { reader->evaluateMissingDefaults({}, columns); - } reader->performRequiredConversions(columns); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9959688d889..0ec8fb5c9ad 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTreeVirtualColumns.cpp b/src/Storages/MergeTreeVirtualColumns.cpp new file mode 100644 index 00000000000..0936c933f9e --- /dev/null +++ b/src/Storages/MergeTreeVirtualColumns.cpp @@ -0,0 +1,62 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NO_SUCH_COLUMN_IN_TABLE; +} + +static ASTPtr getCompressionCodecDeltaLZ4() +{ + return makeASTFunction("CODEC", + std::make_shared("Delta"), + std::make_shared("LZ4")); +} + +const String RowExistsColumn::name = "_row_exists"; +const DataTypePtr RowExistsColumn::type = std::make_shared(); + +const String BlockNumberColumn::name = "_block_number"; +const DataTypePtr BlockNumberColumn::type = std::make_shared(); +const ASTPtr BlockNumberColumn::codec = getCompressionCodecDeltaLZ4(); + +const String BlockOffsetColumn::name = "_block_offset"; +const DataTypePtr BlockOffsetColumn::type = std::make_shared(); +const ASTPtr BlockOffsetColumn::codec = getCompressionCodecDeltaLZ4(); + +Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part, UInt64 part_index) +{ + if (column_name == "_part_offset" || column_name == BlockOffsetColumn::name) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} is not const and must be filled by range reader", column_name); + + if (column_name == RowExistsColumn::name) + return 1ULL; + + if (column_name == BlockNumberColumn::name) + return part.info.min_block; + + if (column_name == "_part") + return part.name; + + if (column_name == "_part_index") + return part_index; + + if (column_name == "_part_uuid") + return part.uuid; + + if (column_name == "_partition_id") + return part.info.partition_id; + + if (column_name == "_partition_value") + return Tuple(part.partition.value.begin(), part.partition.value.end()); + + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Unexpected virtual column name: {}", column_name); +} + +} diff --git a/src/Storages/MergeTreeVirtualColumns.h b/src/Storages/MergeTreeVirtualColumns.h new file mode 100644 index 00000000000..3ee22028d1f --- /dev/null +++ b/src/Storages/MergeTreeVirtualColumns.h @@ -0,0 +1,33 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class IMergeTreeDataPart; + +struct RowExistsColumn +{ + static const String name; + static const DataTypePtr type; +}; + +struct BlockNumberColumn +{ + static const String name; + static const DataTypePtr type; + static const ASTPtr codec; +}; + +struct BlockOffsetColumn +{ + static const String name; + static const DataTypePtr type; + static const ASTPtr codec; +}; + +Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part, UInt64 part_index); + +} diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 86ed1d03b94..8c13031de3f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -104,7 +104,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 64ff224fc10..bda058c5635 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -349,7 +349,7 @@ Block StorageInMemoryMetadata::getSampleBlockNonMaterialized() const return res; } -Block StorageInMemoryMetadata::getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const +Block StorageInMemoryMetadata::getSampleBlockwithVirtuals(const NamesAndTypesList & virtuals) const { auto res = getSampleBlock(); diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index ecc30f7b756..1fa30084fad 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -174,7 +174,7 @@ struct StorageInMemoryMetadata /// Block with ordinary + materialized + virtuals. Virtuals have to be /// explicitly specified, because they are part of Storage type, not /// Storage metadata. - Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; + Block getSampleBlockwithVirtuals(const NamesAndTypesList & virtuals) const; /// Returns structure with partition key. const KeyDescription & getPartitionKey() const; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index c7b0a9d0644..a31e95025b6 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -22,6 +22,7 @@ #include #include "StorageLogSettings.h" +#include "Storages/StorageSnapshot.h" #include #include #include @@ -35,7 +36,7 @@ #include #include #include -#include +#include #include #include @@ -48,8 +49,6 @@ namespace DB { - CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); - namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; @@ -299,6 +298,7 @@ public: : SinkToStorage(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) + , storage_snapshot(std::make_shared(storage, metadata_snapshot)) , lock(std::move(lock_)) { if (!lock) @@ -343,6 +343,7 @@ public: private: StorageLog & storage; StorageMetadataPtr metadata_snapshot; + StorageSnapshotPtr storage_snapshot; WriteLock lock; bool done = false; @@ -476,13 +477,7 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no information about file {} in StorageLog", data_file_name); const auto & data_file = *data_file_it->second; - const auto & columns = metadata_snapshot->getColumns(); - - CompressionCodecPtr compression; - if (name_and_type.name == BlockNumberColumn::name) - compression = BlockNumberColumn::compression_codec; - else - compression = columns.getCodecOrDefault(name_and_type.name); + auto compression = storage_snapshot->getCodecOrDefault(name_and_type.name); it = streams.try_emplace(data_file.name, storage.disk, data_file.path, storage.file_checker.getFileSize(data_file.path), diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 34c092c7208..d1ccbaf02dc 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -1,11 +1,12 @@ #include #include -#include +#include #include #include #include #include #include +#include "Storages/ColumnsDescription.h" namespace DB { @@ -18,6 +19,39 @@ namespace ErrorCodes extern const int COLUMN_QUERIED_MORE_THAN_ONCE; } +StorageSnapshot::StorageSnapshot( + const IStorage & storage_, + StorageMetadataPtr metadata_) + : storage(storage_) + , metadata(std::move(metadata_)) + , virtual_columns(storage_.getVirtualsDescription()) +{ +} + +StorageSnapshot::StorageSnapshot( + const IStorage & storage_, + StorageMetadataPtr metadata_, + ColumnsDescription object_columns_) + : storage(storage_) + , metadata(std::move(metadata_)) + , virtual_columns(storage_.getVirtualsDescription()) + , object_columns(std::move(object_columns_)) +{ +} + +StorageSnapshot::StorageSnapshot( + const IStorage & storage_, + StorageMetadataPtr metadata_, + ColumnsDescription object_columns_, + DataPtr data_) + : storage(storage_) + , metadata(std::move(metadata_)) + , virtual_columns(storage_.getVirtualsDescription()) + , object_columns(std::move(object_columns_)) + , data(std::move(data_)) +{ +} + std::shared_ptr StorageSnapshot::clone(DataPtr data_) const { auto res = std::make_shared(storage, metadata, object_columns); @@ -28,17 +62,6 @@ std::shared_ptr StorageSnapshot::clone(DataPtr data_) const return res; } -void StorageSnapshot::init() -{ - for (const auto & [name, type] : storage.getVirtuals()) - virtual_columns[name] = type; - - if (storage.hasLightweightDeletedMask()) - system_columns[LightweightDeleteDescription::FILTER_COLUMN.name] = LightweightDeleteDescription::FILTER_COLUMN.type; - - system_columns[BlockNumberColumn::name] = BlockNumberColumn::type; -} - NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const { auto all_columns = getMetadataForQuery()->getColumns().get(options); @@ -46,35 +69,22 @@ NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) if (options.with_extended_objects) extendObjectColumns(all_columns, object_columns, options.with_subcolumns); - NameSet column_names; - if (options.with_virtuals) + if (options.virtuals_kind != VirtualsKind::None && !virtual_columns.empty()) { - /// Virtual columns must be appended after ordinary, - /// because user can override them. - if (!virtual_columns.empty()) + NameSet column_names; + for (const auto & column : all_columns) + column_names.insert(column.name); + + auto virtuals_list = virtual_columns.get(options.virtuals_kind); + for (const auto & column : virtuals_list) { - for (const auto & column : all_columns) - column_names.insert(column.name); + if (column_names.contains(column.name)) + continue; - for (const auto & [name, type] : virtual_columns) - if (!column_names.contains(name)) - all_columns.emplace_back(name, type); + all_columns.emplace_back(column.name, column.type); } } - if (options.with_system_columns) - { - if (!system_columns.empty() && column_names.empty()) - { - for (const auto & column : all_columns) - column_names.insert(column.name); - } - - for (const auto & [name, type] : system_columns) - if (!column_names.contains(name)) - all_columns.emplace_back(name, type); - } - return all_columns; } @@ -100,18 +110,11 @@ std::optional StorageSnapshot::tryGetColumn(const GetColumnsOpt return object_column; } - if (options.with_virtuals) + if (options.virtuals_kind != VirtualsKind::None) { - auto it = virtual_columns.find(column_name); - if (it != virtual_columns.end()) - return NameAndTypePair(column_name, it->second); - } - - if (options.with_system_columns) - { - auto it = system_columns.find(column_name); - if (it != system_columns.end()) - return NameAndTypePair(column_name, it->second); + auto virtual_column = virtual_columns.tryGet(column_name, options.virtuals_kind); + if (virtual_column) + return NameAndTypePair{virtual_column->name, virtual_column->type}; } return {}; @@ -126,6 +129,47 @@ NameAndTypePair StorageSnapshot::getColumn(const GetColumnsOptions & options, co return *column; } +CompressionCodecPtr StorageSnapshot::getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const +{ + auto get_codec_or_default = [&](const auto & column_desc) + { + return column_desc.codec + ? CompressionCodecFactory::instance().get(column_desc.codec, column_desc.type, default_codec) + : default_codec; + }; + + const auto & columns = metadata->getColumns(); + if (const auto * column_desc = columns.tryGet(column_name)) + return get_codec_or_default(*column_desc); + + if (const auto virtual_desc = virtual_columns.tryGetDescription(column_name)) + return get_codec_or_default(*virtual_desc); + + return default_codec; +} + +CompressionCodecPtr StorageSnapshot::getCodecOrDefault(const String & column_name) const +{ + return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec()); +} + +ASTPtr StorageSnapshot::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const +{ + auto get_codec_or_default = [&](const auto & column_desc) + { + return column_desc.codec ? column_desc.codec : default_codec->getFullCodecDesc(); + }; + + const auto & columns = metadata->getColumns(); + if (const auto * column_desc = columns.tryGet(column_name)) + return get_codec_or_default(*column_desc); + + if (const auto virtual_desc = virtual_columns.tryGetDescription(column_name)) + return get_codec_or_default(*virtual_desc); + + return default_codec->getFullCodecDesc(); +} + Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const { Block res; @@ -143,11 +187,11 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons { res.insert({object_column->type->createColumn(), object_column->type, column_name}); } - else if (auto it = virtual_columns.find(column_name); it != virtual_columns.end()) + else if (auto virtual_column = virtual_columns.tryGet(column_name)) { /// Virtual columns must be appended after ordinary, because user can /// override them. - const auto & type = it->second; + const auto & type = virtual_column->type; res.insert({type->createColumn(), type, column_name}); } else @@ -175,12 +219,11 @@ ColumnsDescription StorageSnapshot::getDescriptionForColumns(const Names & colum { res.add(*object_column, "", false, false); } - else if (auto it = virtual_columns.find(name); it != virtual_columns.end()) + else if (auto virtual_column = virtual_columns.tryGet(name)) { /// Virtual columns must be appended after ordinary, because user can /// override them. - const auto & type = it->second; - res.add({name, type}); + res.add({name, virtual_column->type}); } else { @@ -216,7 +259,7 @@ void StorageSnapshot::check(const Names & column_names) const { bool has_column = columns.hasColumnOrSubcolumn(GetColumnsOptions::AllPhysical, name) || object_columns.hasColumnOrSubcolumn(GetColumnsOptions::AllPhysical, name) - || virtual_columns.contains(name); + || virtual_columns.has(name); if (!has_column) { diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index d62e118e1f2..cabc4f6e2b4 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -1,10 +1,14 @@ #pragma once #include +#include namespace DB { class IStorage; +class ICompressionCodec; + +using CompressionCodecPtr = std::shared_ptr; /// Snapshot of storage that fixes set columns that can be read in query. /// There are 3 sources of columns: regular columns from metadata, @@ -13,6 +17,7 @@ struct StorageSnapshot { const IStorage & storage; const StorageMetadataPtr metadata; + const VirtualColumnsDescription virtual_columns; const ColumnsDescription object_columns; /// Additional data, on which set of columns may depend. @@ -30,35 +35,18 @@ struct StorageSnapshot StorageSnapshot( const IStorage & storage_, - StorageMetadataPtr metadata_) - : storage(storage_), metadata(std::move(metadata_)) - { - init(); - } + StorageMetadataPtr metadata_); StorageSnapshot( const IStorage & storage_, StorageMetadataPtr metadata_, - ColumnsDescription object_columns_) - : storage(storage_) - , metadata(std::move(metadata_)) - , object_columns(std::move(object_columns_)) - { - init(); - } + ColumnsDescription object_columns_); StorageSnapshot( const IStorage & storage_, StorageMetadataPtr metadata_, ColumnsDescription object_columns_, - DataPtr data_) - : storage(storage_) - , metadata(std::move(metadata_)) - , object_columns(std::move(object_columns_)) - , data(std::move(data_)) - { - init(); - } + DataPtr data_); std::shared_ptr clone(DataPtr data_) const; @@ -72,6 +60,10 @@ struct StorageSnapshot std::optional tryGetColumn(const GetColumnsOptions & options, const String & column_name) const; NameAndTypePair getColumn(const GetColumnsOptions & options, const String & column_name) const; + CompressionCodecPtr getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; + CompressionCodecPtr getCodecOrDefault(const String & column_name) const; + ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; + /// Block with ordinary + materialized + aliases + virtuals + subcolumns. Block getSampleBlockForColumns(const Names & column_names) const; @@ -90,12 +82,6 @@ struct StorageSnapshot private: void init(); - - std::unordered_map virtual_columns; - - /// System columns are not visible in the schema but might be persisted in the data. - /// One example of such column is lightweight delete mask '_row_exists'. - std::unordered_map system_columns; }; using StorageSnapshotPtr = std::shared_ptr; diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 3b2807965a4..540cd6b9a6b 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -52,7 +52,7 @@ public: size_t /*num_streams*/) override { storage_snapshot->check(column_names); - Block sample_block = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + Block sample_block = storage_snapshot->metadata->getSampleBlockwithVirtuals(getVirtuals()); if (supportsColumnsMask()) { diff --git a/src/Storages/System/StorageSystemJemalloc.cpp b/src/Storages/System/StorageSystemJemalloc.cpp index 15543208dd9..bc868b93e9c 100644 --- a/src/Storages/System/StorageSystemJemalloc.cpp +++ b/src/Storages/System/StorageSystemJemalloc.cpp @@ -115,7 +115,7 @@ Pipe StorageSystemJemallocBins::read( { storage_snapshot->check(column_names); - auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + auto header = storage_snapshot->metadata->getSampleBlockwithVirtuals(getVirtuals()); MutableColumns res_columns = header.cloneEmptyColumns(); fillJemallocBins(res_columns); diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index abf93bf1ac0..42ed1e3deb9 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -243,7 +243,7 @@ void StorageSystemZooKeeper::read( size_t max_block_size, size_t /*num_streams*/) { - auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + auto header = storage_snapshot->metadata->getSampleBlockwithVirtuals(getVirtuals()); auto read_step = std::make_unique(header, query_info, max_block_size, context); query_plan.addStep(std::move(read_step)); } diff --git a/src/Storages/VirtualColumnsDescription.cpp b/src/Storages/VirtualColumnsDescription.cpp new file mode 100644 index 00000000000..65223071dfe --- /dev/null +++ b/src/Storages/VirtualColumnsDescription.cpp @@ -0,0 +1,96 @@ +#include "Core/NamesAndTypes.h" +#include "DataTypes/Serializations/ISerialization.h" +#include "base/types.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DUPLICATE_COLUMN; + extern const int NO_SUCH_COLUMN_IN_TABLE; +} + +VirtualColumnDescription::VirtualColumnDescription( + String name_, DataTypePtr type_, ASTPtr codec_, String comment_, VirtualsKind kind_) + : ColumnDescription(std::move(name_), std::move(type_), std::move(codec_), std::move(comment_)) + , kind(kind_) +{ +} + +void VirtualColumnsDescription::add(VirtualColumnDescription desc_) +{ + auto [it, inserted] = container.emplace(std::move(desc_)); + if (!inserted) + throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Virtual column {} already exists", it->name); +} + +void VirtualColumnsDescription::addEphemeral(String name, DataTypePtr type, String comment) +{ + add({std::move(name), std::move(type), nullptr, std::move(comment), VirtualsKind::Ephemeral}); +} + +void VirtualColumnsDescription::addPersistent(String name, DataTypePtr type, ASTPtr codec, String comment) +{ + add({std::move(name), std::move(type), std::move(codec), std::move(comment), VirtualsKind::Persistent}); +} + +NamesAndTypesList VirtualColumnsDescription::get(VirtualsKind kind) const +{ + NamesAndTypesList result; + for (const auto & column : container) + if (static_cast(column.kind) & static_cast(kind)) + result.emplace_back(column.name, column.type); + return result; +} + +std::optional VirtualColumnsDescription::tryGet(const String & name, VirtualsKind kind) const +{ + auto it = container.find(name); + if (it != container.end() && (static_cast(it->kind) & static_cast(kind))) + return NameAndTypePair{it->name, it->type}; + return {}; +} + +NameAndTypePair VirtualColumnsDescription::get(const String & name, VirtualsKind kind) const +{ + auto column = tryGet(name, kind); + if (!column) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no virtual column {}", name); + return *column; +} + +std::optional VirtualColumnsDescription::tryGetDescription(const String & name, VirtualsKind kind) const +{ + auto it = container.find(name); + if (it != container.end() && (static_cast(it->kind) & static_cast(kind))) + return *it; + return {}; +} + +VirtualColumnDescription VirtualColumnsDescription::getDescription(const String & name, VirtualsKind kind) const +{ + auto column = tryGetDescription(name, kind); + if (!column) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no virtual column {}", name); + return *column; +} + +NamesAndTypesList VirtualColumnsDescription::getNamesAndTypesList() const +{ + NamesAndTypesList result; + for (const auto & desc : container) + result.emplace_back(desc.name, desc.type); + return result; +} + +Block VirtualColumnsDescription::getSampleBlock() const +{ + Block result; + for (const auto & desc : container) + result.insert({desc.type->createColumn(), desc.type, desc.name}); + return result; +} + +} diff --git a/src/Storages/VirtualColumnsDescription.h b/src/Storages/VirtualColumnsDescription.h new file mode 100644 index 00000000000..b93bf29635a --- /dev/null +++ b/src/Storages/VirtualColumnsDescription.h @@ -0,0 +1,68 @@ +#pragma once +#include + +namespace DB +{ + +struct VirtualColumnDescription : public ColumnDescription +{ +public: + using Self = VirtualColumnDescription; + VirtualsKind kind; + + VirtualColumnDescription() = default; + VirtualColumnDescription(String name_, DataTypePtr type_, ASTPtr codec_, String comment_, VirtualsKind kind_); + + bool isEphemeral() const { return kind == VirtualsKind::Ephemeral; } + bool isPersistent() const { return kind == VirtualsKind::Persistent; } + + struct Comparator + { + using is_transparent = void; + bool operator()(const Self & lhs, const Self & rhs) const { return lhs.name < rhs.name; } + bool operator()(const Self & lhs, const String & rhs) const { return lhs.name < rhs; } + bool operator()(const String & lhs, const Self & rhs) const { return lhs < rhs.name; } + }; +}; + +class VirtualColumnsDescription +{ +public: + using Container = std::set; + using const_iterator = Container::const_iterator; + + const_iterator begin() const { return container.begin(); } + const_iterator end() const { return container.end(); } + + VirtualColumnsDescription() = default; + + void add(VirtualColumnDescription desc); + void addEphemeral(String name, DataTypePtr type, String comment); + void addPersistent(String name, DataTypePtr type, ASTPtr codec, String comment); + + bool empty() const { return container.empty(); } + bool has(const String & name) const { return container.contains(name); } + + NameAndTypePair get(const String & name, VirtualsKind kind) const; + std::optional tryGet(const String & name, VirtualsKind kind) const; + + NameAndTypePair get(const String & name) const { return get(name, VirtualsKind::All); } + std::optional tryGet(const String & name) const { return tryGet(name, VirtualsKind::All); } + + std::optional tryGetDescription(const String & name, VirtualsKind kind) const; + VirtualColumnDescription getDescription(const String & name, VirtualsKind kind) const; + + std::optional tryGetDescription(const String & name) const { return tryGetDescription(name, VirtualsKind::All); } + VirtualColumnDescription getDescription(const String & name) const { return getDescription(name, VirtualsKind::All); } + + NamesAndTypesList get(VirtualsKind kind) const; + NamesAndTypesList getNamesAndTypesList() const; + + Block getSampleBlock() const; + Block getSampleBlock(const Names & names) const; + +private: + Container container; +}; + +} From 74399253d9a45402218a8d54835e3cbc6be9bd4e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 21 Feb 2024 02:28:30 +0000 Subject: [PATCH 016/197] refactoring of virtual columns --- .../QueryPlan/ReadFromMergeTree.cpp | 47 ++++++---------- src/Processors/QueryPlan/ReadFromMergeTree.h | 16 ++---- src/Storages/BlockNumberColumn.cpp | 0 src/Storages/BlockNumberColumn.h | 0 src/Storages/MergeTree/IMergeTreeReader.cpp | 1 - .../MergeTreeDataPartWriterCompact.cpp | 1 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 53 ++----------------- .../MergeTree/MergeTreeDataSelectExecutor.h | 1 - .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 - .../MergeTree/MergeTreePrefetchedReadPool.h | 1 - src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 - src/Storages/MergeTree/MergeTreeReadPool.h | 1 - .../MergeTree/MergeTreeReadPoolBase.cpp | 11 +--- .../MergeTree/MergeTreeReadPoolBase.h | 2 - .../MergeTree/MergeTreeReadPoolInOrder.cpp | 2 - .../MergeTree/MergeTreeReadPoolInOrder.h | 1 - .../MergeTreeReadPoolParallelReplicas.cpp | 2 - .../MergeTreeReadPoolParallelReplicas.h | 1 - ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 2 - ...MergeTreeReadPoolParallelReplicasInOrder.h | 1 - src/Storages/MergeTree/MergeTreeReadTask.h | 2 - .../MergeTree/MergeTreeSelectProcessor.cpp | 36 ++----------- .../MergeTree/MergeTreeSelectProcessor.h | 12 +---- src/Storages/MergeTreeVirtualColumns.cpp | 1 + 24 files changed, 31 insertions(+), 167 deletions(-) delete mode 100644 src/Storages/BlockNumberColumn.cpp delete mode 100644 src/Storages/BlockNumberColumn.h diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e2ce36264dd..c095dc06c3a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -265,29 +265,24 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c ReadFromMergeTree::ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, std::vector alter_conversions_, - Names real_column_names_, - Names virt_column_names_, + Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, StorageSnapshotPtr storage_snapshot_, ContextPtr context_, size_t max_block_size_, size_t num_streams_, - bool sample_factor_column_queried_, std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( - storage_snapshot_->getSampleBlockForColumns(real_column_names_), - storage_snapshot_, - query_info_.prewhere_info, - virt_column_names_)}) + storage_snapshot_->getSampleBlockForColumns(all_column_names_), + query_info_.prewhere_info)}) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) , prepared_parts(std::move(parts_)) , alter_conversions_for_parts(std::move(alter_conversions_)) - , real_column_names(std::move(real_column_names_)) - , virt_column_names(std::move(virt_column_names_)) + , all_column_names(std::move(all_column_names_)) , data(data_) , query_info(query_info_) , prewhere_info(query_info_.prewhere_info) @@ -300,7 +295,7 @@ ReadFromMergeTree::ReadFromMergeTree( .preferred_block_size_bytes = context->getSettingsRef().preferred_block_size_bytes, .preferred_max_column_in_block_size_bytes = context->getSettingsRef().preferred_max_column_in_block_size_bytes} , requested_num_streams(num_streams_) - , sample_factor_column_queried(sample_factor_column_queried_) + , sample_factor_column_queried(false) /// TODO: kek , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(std::move(log_)) , analyzed_result_ptr(analyzed_result_ptr_) @@ -380,7 +375,6 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( actions_settings, reader_settings, required_columns, - virt_column_names, pool_settings, context); @@ -395,7 +389,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( auto processor = std::make_unique( pool, std::move(algorithm), storage_snapshot, prewhere_info, - actions_settings, block_size_copy, reader_settings, virt_column_names); + actions_settings, block_size_copy, reader_settings); auto source = std::make_shared(std::move(processor)); pipes.emplace_back(std::move(source)); @@ -461,7 +455,6 @@ Pipe ReadFromMergeTree::readFromPool( actions_settings, reader_settings, required_columns, - virt_column_names, pool_settings, context); } @@ -474,7 +467,6 @@ Pipe ReadFromMergeTree::readFromPool( actions_settings, reader_settings, required_columns, - virt_column_names, pool_settings, context); } @@ -494,7 +486,7 @@ Pipe ReadFromMergeTree::readFromPool( auto processor = std::make_unique( pool, std::move(algorithm), storage_snapshot, prewhere_info, - actions_settings, block_size_copy, reader_settings, virt_column_names); + actions_settings, block_size_copy, reader_settings); auto source = std::make_shared(std::move(processor)); @@ -550,7 +542,6 @@ Pipe ReadFromMergeTree::readInOrder( actions_settings, reader_settings, required_columns, - virt_column_names, pool_settings, context); } @@ -565,7 +556,6 @@ Pipe ReadFromMergeTree::readInOrder( actions_settings, reader_settings, required_columns, - virt_column_names, pool_settings, context); } @@ -600,7 +590,7 @@ Pipe ReadFromMergeTree::readInOrder( auto processor = std::make_unique( pool, std::move(algorithm), storage_snapshot, prewhere_info, - actions_settings, block_size, reader_settings, virt_column_names); + actions_settings, block_size, reader_settings); processor->addPartLevelToChunk(isQueryWithFinal()); @@ -1311,8 +1301,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( requested_num_streams, max_block_numbers_to_read, data, - real_column_names, - sample_factor_column_queried, + all_column_names, log, indexes); } @@ -1506,8 +1495,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( size_t num_streams, std::shared_ptr max_block_numbers_to_read, const MergeTreeData & data, - const Names & real_column_names, - bool sample_factor_column_queried, + const Names & all_column_names, LoggerPtr log, std::optional & indexes) { @@ -1523,8 +1511,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( num_streams, max_block_numbers_to_read, data, - real_column_names, - sample_factor_column_queried, + all_column_names, log, indexes); } @@ -1538,8 +1525,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( size_t num_streams, std::shared_ptr max_block_numbers_to_read, const MergeTreeData & data, - const Names & real_column_names, - bool sample_factor_column_queried, + const Names & all_column_names, LoggerPtr log, std::optional & indexes) { @@ -1548,7 +1534,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( size_t total_parts = parts.size(); - result.column_names_to_read = real_column_names; + result.column_names_to_read = all_column_names; /// If there are only virtual columns in the query, you must request at least one non-virtual one. if (result.column_names_to_read.empty()) @@ -1607,7 +1593,6 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( data, metadata_snapshot, context, - sample_factor_column_queried, log); if (result.sampling.read_nothing) @@ -1724,10 +1709,8 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info prewhere_info = prewhere_info_value; output_stream = DataStream{.header = MergeTreeSelectProcessor::transformHeader( - storage_snapshot->getSampleBlockForColumns(real_column_names), - storage_snapshot, - prewhere_info_value, - virt_column_names)}; + storage_snapshot->getSampleBlockForColumns(all_column_names), + prewhere_info_value)}; updateSortDescriptionForOutputStream( *output_stream, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index fdeaff57279..4ecaa5e262f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -110,15 +110,13 @@ public: ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, std::vector alter_conversions_, - Names real_column_names_, - Names virt_column_names_, + Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, StorageSnapshotPtr storage_snapshot, ContextPtr context_, size_t max_block_size_, size_t num_streams_, - bool sample_factor_column_queried_, std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, @@ -135,8 +133,7 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeIndexes(JSONBuilder::JSONMap & map) const override; - const Names & getRealColumnNames() const { return real_column_names; } - const Names & getVirtualColumnNames() const { return virt_column_names; } + const Names & getRealColumnNames() const { return all_column_names; } StorageID getStorageID() const { return data.getStorageID(); } const StorageSnapshotPtr & getStorageSnapshot() const { return storage_snapshot; } @@ -166,8 +163,7 @@ public: size_t num_streams, std::shared_ptr max_block_numbers_to_read, const MergeTreeData & data, - const Names & real_column_names, - bool sample_factor_column_queried, + const Names & all_column_names, LoggerPtr log, std::optional & indexes); @@ -215,8 +211,7 @@ private: size_t num_streams, std::shared_ptr max_block_numbers_to_read, const MergeTreeData & data, - const Names & real_column_names, - bool sample_factor_column_queried, + const Names & all_column_names, LoggerPtr log, std::optional & indexes); @@ -233,8 +228,7 @@ private: MergeTreeData::DataPartsVector prepared_parts; std::vector alter_conversions_for_parts; - Names real_column_names; - Names virt_column_names; + Names all_column_names; const MergeTreeData & data; SelectQueryInfo query_info; diff --git a/src/Storages/BlockNumberColumn.cpp b/src/Storages/BlockNumberColumn.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Storages/BlockNumberColumn.h b/src/Storages/BlockNumberColumn.h deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index f774d3a387a..7e5bb0aaea4 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -23,7 +23,6 @@ namespace namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int NO_SUCH_COLUMN_IN_TABLE; } IMergeTreeReader::IMergeTreeReader( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 46c4338ab90..1721fd15b8d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index df1176a9ea5..6b09f85a74d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -69,7 +69,6 @@ namespace ErrorCodes extern const int CANNOT_PARSE_TEXT; extern const int TOO_MANY_PARTITIONS; extern const int DUPLICATED_PART_UUIDS; - extern const int NO_SUCH_COLUMN_IN_TABLE; } @@ -166,7 +165,6 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - bool sample_factor_column_queried, LoggerPtr log) { const Settings & settings = context->getSettingsRef(); @@ -296,7 +294,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (sampling.use_sampling) { - if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) + if (relative_sample_size != RelativeSize(0)) sampling.used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; @@ -869,32 +867,6 @@ std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( return nullptr; } -static void selectColumnNames( - const Names & column_names_to_return, - const MergeTreeData & data, - Names & real_column_names, - Names & virt_column_names, - bool & sample_factor_column_queried) -{ - sample_factor_column_queried = false; - const auto & virtual_columns = data.getVirtualsDescription(); - - for (const auto & name : column_names_to_return) - { - if (virtual_columns.has(name)) - { - if (name == "_sample_factor") - sample_factor_column_queried = true; - - virt_column_names.push_back(name); - } - else - { - real_column_names.push_back(name); - } - } -} - ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, const PrewhereInfoPtr & prewhere_info, @@ -910,14 +882,6 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar if (total_parts == 0) return std::make_shared(); - Names real_column_names; - Names virt_column_names; - /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. - /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. - bool sample_factor_column_queried = false; - - selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - std::optional indexes; /// NOTE: We don't need alter_conversions because the returned analysis_result is only used for: /// 1. estimate the number of rows to read; 2. projection reading, which doesn't have alter_conversions. @@ -932,8 +896,7 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar num_streams, max_block_numbers_to_read, data, - real_column_names, - sample_factor_column_queried, + column_names_to_return, log, indexes); } @@ -960,14 +923,6 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( else if (parts.empty()) return {}; - Names real_column_names; - Names virt_column_names; - /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. - /// The virtual column `_sample_factor` (which is equal to 1 / used sample rate) can be requested in the query. - bool sample_factor_column_queried = false; - - selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - /// Do not keep data parts in snapshot. /// They are stored separately, and some could be released after PK analysis. auto storage_snapshot_copy = storage_snapshot->clone(std::make_unique()); @@ -975,15 +930,13 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( return std::make_unique( std::move(parts), std::move(alter_conversions), - real_column_names, - virt_column_names, + column_names_to_return, data, query_info, storage_snapshot_copy, context, max_block_size, num_streams, - sample_factor_column_queried, max_block_numbers_to_read, log, merge_tree_select_result_ptr, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 17975354187..79f936ae0a8 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -215,7 +215,6 @@ public: const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - bool sample_factor_column_queried, LoggerPtr log); /// Check query limits: max_partitions_to_read, max_concurrent_queries. diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 47c2fe07bb4..07c4b367b2c 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -113,7 +113,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -123,7 +122,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( actions_settings_, reader_settings_, column_names_, - virtual_column_names_, settings_, context_) , WithContext(context_) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index b1335fd2774..250631c6264 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -23,7 +23,6 @@ public: const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 8ed7a9d8707..e0f5eb22950 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -39,7 +39,6 @@ MergeTreeReadPool::MergeTreeReadPool( const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -49,7 +48,6 @@ MergeTreeReadPool::MergeTreeReadPool( actions_settings_, reader_settings_, column_names_, - virtual_column_names_, settings_, context_) , min_marks_for_concurrent_read(pool_settings.min_marks_for_concurrent_read) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index e45ccad912f..44726f3877f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -31,7 +31,6 @@ public: const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 95e42d39cf7..a1d5ddd8729 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -12,7 +12,6 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & pool_settings_, const ContextPtr & context_) : parts_ranges(std::move(parts_)) @@ -21,7 +20,6 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( , actions_settings(actions_settings_) , reader_settings(reader_settings_) , column_names(column_names_) - , virtual_column_names(virtual_column_names_) , pool_settings(pool_settings_) , owned_mark_cache(context_->getGlobalContext()->getMarkCache()) , owned_uncompressed_cache(pool_settings_.use_uncompressed_cache ? context_->getGlobalContext()->getUncompressedCache() : nullptr) @@ -52,22 +50,15 @@ void MergeTreeReadPoolBase::fillPerPartInfos() LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, part_with_ranges.alter_conversions); - Names column_and_virtual_column_names; - column_and_virtual_column_names.reserve(column_names.size() + virtual_column_names.size()); - column_and_virtual_column_names.insert(column_and_virtual_column_names.end(), column_names.begin(), column_names.end()); - column_and_virtual_column_names.insert( - column_and_virtual_column_names.end(), virtual_column_names.begin(), virtual_column_names.end()); read_task_info.task_columns = getReadTaskColumns( part_info, storage_snapshot, - column_and_virtual_column_names, + column_names, prewhere_info, actions_settings, reader_settings, /*with_subcolumns=*/true); - read_task_info.virt_column_names = {virtual_column_names.begin(), virtual_column_names.end()}; - if (pool_settings.preferred_block_size_bytes > 0) { const auto & result_column_names = read_task_info.task_columns.columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index a0c6a30fb63..5ec2cf454c3 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -28,7 +28,6 @@ public: const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_); @@ -42,7 +41,6 @@ protected: const ExpressionActionsSettings actions_settings; const MergeTreeReaderSettings reader_settings; const Names column_names; - const Names virtual_column_names; const PoolSettings pool_settings; const MarkCachePtr owned_mark_cache; const UncompressedCachePtr owned_uncompressed_cache; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index 1b621ad5055..dd341f6f750 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -17,7 +17,6 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -27,7 +26,6 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( actions_settings_, reader_settings_, column_names_, - virtual_column_names_, settings_, context_) , has_limit_below_one_block(has_limit_below_one_block_) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index d9cc1ba4984..1b846fdb22a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -16,7 +16,6 @@ public: const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 47436ed1407..24c0e2525ff 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -18,7 +18,6 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -28,7 +27,6 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( actions_settings_, reader_settings_, column_names_, - virtual_column_names_, settings_, context_) , extension(std::move(extension_)) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index 6a548dffe37..d9895b3e14b 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -16,7 +16,6 @@ public: const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index a822a517933..a34bd05ae63 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -17,7 +17,6 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_) : MergeTreeReadPoolBase( @@ -27,7 +26,6 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd actions_settings_, reader_settings_, column_names_, - virtual_column_names_, settings_, context_) , extension(std::move(extension_)) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 3e5f8f5dfba..207dfb7899a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -17,7 +17,6 @@ public: const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, - const Names & virtual_column_names_, const PoolSettings & settings_, const ContextPtr & context_); diff --git a/src/Storages/MergeTree/MergeTreeReadTask.h b/src/Storages/MergeTree/MergeTreeReadTask.h index 21ec19ee033..b69abac5451 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.h +++ b/src/Storages/MergeTree/MergeTreeReadTask.h @@ -60,8 +60,6 @@ struct MergeTreeReadTaskInfo AlterConversionsPtr alter_conversions; /// Column names to read during PREWHERE and WHERE MergeTreeReadTaskColumns task_columns; - /// Virtual column names to read - NameSet virt_column_names; /// Shared initialized size predictor. It is copied for each new task. MergeTreeBlockSizePredictorPtr shared_size_predictor; }; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 8e089741cab..d94d4506ee6 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -32,8 +32,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, const MergeTreeReadTask::BlockSizeParams & block_size_params_, - const MergeTreeReaderSettings & reader_settings_, - const Names & virt_column_names_) + const MergeTreeReaderSettings & reader_settings_) : pool(std::move(pool_)) , algorithm(std::move(algorithm_)) , storage_snapshot(storage_snapshot_) @@ -42,7 +41,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( , prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps)) , reader_settings(reader_settings_) , block_size_params(block_size_params_) - , virt_column_names(virt_column_names_) + , result_header(applyPrewhereActions(pool->getHeader(), prewhere_info)) { if (reader_settings.apply_deleted_mask) { @@ -59,10 +58,6 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( lightweight_delete_filter_step = std::make_shared(std::move(step)); } - result_header = pool->getHeader(); - injectVirtualColumns(result_header, storage_snapshot, virt_column_names); - result_header = applyPrewhereActions(result_header, prewhere_info); - if (!prewhere_actions.steps.empty()) LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions.steps.size(), prewhere_actions.dumpConditions()); @@ -181,23 +176,6 @@ void MergeTreeSelectProcessor::initializeRangeReaders() task->initializeRangeReaders(all_prewhere_actions); } -void MergeTreeSelectProcessor::injectVirtualColumns( - Block & block, - const StorageSnapshotPtr & storage_snapshot, - const Names & virtual_columns) -{ - for (const auto & virtual_column_name : virtual_columns) - { - auto column = storage_snapshot->virtual_columns.tryGet(virtual_column_name); - if (!column) - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, - "There is no virtual column {} in table {}", - virtual_column_name, storage_snapshot->storage.getStorageID().getNameForLogs()); - - block.insert({column->type->createColumn(), column->type, column->name}); - } -} - Block MergeTreeSelectProcessor::applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info) { if (prewhere_info) @@ -249,15 +227,9 @@ Block MergeTreeSelectProcessor::applyPrewhereActions(Block block, const Prewhere return block; } -Block MergeTreeSelectProcessor::transformHeader( - Block block, - const StorageSnapshotPtr & storage_snapshot, - const PrewhereInfoPtr & prewhere_info, - const Names & virtual_columns) +Block MergeTreeSelectProcessor::transformHeader(Block block, const PrewhereInfoPtr & prewhere_info) { - injectVirtualColumns(block, storage_snapshot, virtual_columns); - auto transformed = applyPrewhereActions(std::move(block), prewhere_info); - return transformed; + return applyPrewhereActions(std::move(block), prewhere_info); } } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 9b399003909..3afb4f3fac0 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -46,17 +46,11 @@ public: const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, const MergeTreeReadTask::BlockSizeParams & block_size_params_, - const MergeTreeReaderSettings & reader_settings_, - const Names & virt_column_names_); + const MergeTreeReaderSettings & reader_settings_); String getName() const; - static Block transformHeader( - Block block, - const StorageSnapshotPtr & storage_snapshot, - const PrewhereInfoPtr & prewhere_info, - const Names & virtual_columns); - + static Block transformHeader(Block block, const PrewhereInfoPtr & prewhere_info); Block getHeader() const { return result_header; } ChunkAndProgress read(); @@ -83,7 +77,6 @@ private: }; /// Used for filling header with no rows as well as block with data - static void injectVirtualColumns(Block & block, const StorageSnapshotPtr & storage_snapshot, const Names & virtual_columns); static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info); /// Sets up range readers corresponding to data readers @@ -99,7 +92,6 @@ private: const MergeTreeReaderSettings reader_settings; const MergeTreeReadTask::BlockSizeParams block_size_params; - const Names virt_column_names; /// Current task to read from. MergeTreeReadTaskPtr task; diff --git a/src/Storages/MergeTreeVirtualColumns.cpp b/src/Storages/MergeTreeVirtualColumns.cpp index 0936c933f9e..8c41ab2d16e 100644 --- a/src/Storages/MergeTreeVirtualColumns.cpp +++ b/src/Storages/MergeTreeVirtualColumns.cpp @@ -10,6 +10,7 @@ namespace DB namespace ErrorCodes { extern const int NO_SUCH_COLUMN_IN_TABLE; + extern const int LOGICAL_ERROR; } static ASTPtr getCompressionCodecDeltaLZ4() From 462f597c6b2c5e015c934e47810e06e75dd9fa0c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 21 Feb 2024 11:02:05 +0100 Subject: [PATCH 017/197] Update run.sh --- docker/test/upgrade/run.sh | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index d8ba48909b6..262334cf7ee 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -77,6 +77,18 @@ remove_keeper_config "async_replication" "1" # create_if_not_exists feature flag doesn't exist on some older versions remove_keeper_config "create_if_not_exists" "[01]" +#todo: remove these after 24.3 released. +sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \ + | sed "s|azure>|>azure_blob_storage>|" \ + > /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp +sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml + +#todo: remove these after 24.3 released. +sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ + | sed "s|local>|>local_blob_storage>|" \ + > /etc/clickhouse-server/config.d/storage_conf.xml.tmp +sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml + # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml @@ -103,11 +115,13 @@ sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ > /etc/clickhouse-server/config.d/keeper_port.xml.tmp sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml +#todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \ | sed "s|azure>|>azure_blob_storage>|" \ > /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +#todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ | sed "s|local>|>local_blob_storage>|" \ > /etc/clickhouse-server/config.d/storage_conf.xml.tmp From 3d57237fe8c8b1c9379f62a78cf6efcec02d6727 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 21 Feb 2024 16:40:27 +0000 Subject: [PATCH 018/197] refactoring of virtual columns --- .../QueryPlan/ReadFromMergeTree.cpp | 27 ++++------------- src/Processors/QueryPlan/ReadFromMergeTree.h | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 4 ++- src/Storages/MergeTree/IMergeTreeReader.cpp | 29 +++++++++++++------ src/Storages/MergeTree/IMergeTreeReader.h | 10 +++---- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 4 +-- .../MergeTree/MergeTreeDataPartCompact.h | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +-- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 4 +-- .../MergeTree/MergeTreeDataPartWide.h | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 ++ .../MergeTree/MergeTreePrefetchedReadPool.h | 1 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 ++ src/Storages/MergeTree/MergeTreeReadPool.h | 1 + .../MergeTree/MergeTreeReadPoolBase.cpp | 5 ++++ .../MergeTree/MergeTreeReadPoolBase.h | 2 ++ .../MergeTree/MergeTreeReadPoolInOrder.cpp | 2 ++ .../MergeTree/MergeTreeReadPoolInOrder.h | 1 + .../MergeTreeReadPoolParallelReplicas.cpp | 2 ++ .../MergeTreeReadPoolParallelReplicas.h | 1 + ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 2 ++ ...MergeTreeReadPoolParallelReplicasInOrder.h | 1 + src/Storages/MergeTree/MergeTreeReadTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadTask.h | 10 ++++++- .../MergeTree/MergeTreeReaderCompact.cpp | 4 +-- .../MergeTree/MergeTreeReaderCompact.h | 2 +- .../MergeTree/MergeTreeReaderInMemory.cpp | 4 +-- .../MergeTree/MergeTreeReaderInMemory.h | 2 +- .../MergeTree/MergeTreeReaderWide.cpp | 4 +-- src/Storages/MergeTree/MergeTreeReaderWide.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 1 - .../MergeTree/MergeTreeSequentialSource.cpp | 4 +-- src/Storages/MergeTreeVirtualColumns.cpp | 10 ++----- src/Storages/MergeTreeVirtualColumns.h | 2 +- 36 files changed, 91 insertions(+), 71 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index c095dc06c3a..22ccd623196 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -295,20 +295,11 @@ ReadFromMergeTree::ReadFromMergeTree( .preferred_block_size_bytes = context->getSettingsRef().preferred_block_size_bytes, .preferred_max_column_in_block_size_bytes = context->getSettingsRef().preferred_max_column_in_block_size_bytes} , requested_num_streams(num_streams_) - , sample_factor_column_queried(false) /// TODO: kek , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(std::move(log_)) , analyzed_result_ptr(analyzed_result_ptr_) , is_parallel_reading_from_replicas(enable_parallel_reading) { - if (sample_factor_column_queried) - { - /// Only _sample_factor virtual column is added by ReadFromMergeTree - /// Other virtual columns are added by MergeTreeSelectProcessor. - auto type = std::make_shared(); - output_stream->header.insert({type->createColumn(), type, "_sample_factor"}); - } - if (is_parallel_reading_from_replicas) { all_ranges_callback = context->getMergeTreeAllRangesCallback(); @@ -370,6 +361,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( auto pool = std::make_shared( std::move(extension), std::move(parts_with_range), + shared_virtual_fields, storage_snapshot, prewhere_info, actions_settings, @@ -450,6 +442,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + shared_virtual_fields, storage_snapshot, prewhere_info, actions_settings, @@ -462,6 +455,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + shared_virtual_fields, storage_snapshot, prewhere_info, actions_settings, @@ -537,6 +531,7 @@ Pipe ReadFromMergeTree::readInOrder( std::move(extension), mode, parts_with_ranges, + shared_virtual_fields, storage_snapshot, prewhere_info, actions_settings, @@ -551,6 +546,7 @@ Pipe ReadFromMergeTree::readInOrder( has_limit_below_one_block, read_type, parts_with_ranges, + shared_virtual_fields, storage_snapshot, prewhere_info, actions_settings, @@ -1904,6 +1900,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons { auto result = getAnalysisResult(); result.checkLimits(context->getSettingsRef(), query_info); + shared_virtual_fields.emplace("_sample_factor", result.sampling.used_sample_factor); LOG_DEBUG( log, @@ -1988,18 +1985,6 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); }; - /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. - if (sample_factor_column_queried) - { - ColumnWithTypeAndName column; - column.name = "_sample_factor"; - column.type = std::make_shared(); - column.column = column.type->createColumnConst(0, Field(result.sampling.used_sample_factor)); - - auto adding_column = ActionsDAG::makeAddingColumnActions(std::move(column)); - append_actions(std::move(adding_column)); - } - if (result_projection) cur_header = result_projection->updateHeader(cur_header); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 4ecaa5e262f..02c6cc3463b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -243,7 +243,6 @@ private: size_t requested_num_streams; size_t output_streams_limit = 0; - const bool sample_factor_column_queried; /// Used for aggregation optimization (see DB::QueryPlanOptimizations::tryAggregateEachPartitionIndependently). bool output_each_partition_through_separate_port = false; @@ -284,7 +283,9 @@ private: RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection); ReadFromMergeTree::AnalysisResult getAnalysisResult() const; + AnalysisResultPtr analyzed_result_ptr; + VirtualFields shared_virtual_fields; bool is_parallel_reading_from_replicas; std::optional all_ranges_callback; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index d5b024164a2..ab6df012195 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -70,6 +71,7 @@ public: using Checksums = MergeTreeDataPartChecksums; using Checksum = MergeTreeDataPartChecksums::Checksum; using ValueSizeMap = std::map; + using VirtualFields = std::unordered_map; using MergeTreeReaderPtr = std::unique_ptr; using MergeTreeWriterPtr = std::unique_ptr; @@ -95,7 +97,7 @@ public: const NamesAndTypesList & columns_, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 7e5bb0aaea4..73ccd71b217 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -23,12 +24,13 @@ namespace namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } IMergeTreeReader::IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -51,7 +53,7 @@ IMergeTreeReader::IMergeTreeReader( , part_columns(data_part_info_for_read->isWidePart() ? data_part_info_for_read->getColumnsDescriptionWithCollectedNested() : data_part_info_for_read->getColumnsDescription()) - , read_task_info(read_task_info_) + , virtual_fields(virtual_fields_) { columns_to_read.reserve(requested_columns.size()); serializations.reserve(requested_columns.size()); @@ -70,12 +72,11 @@ const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints() void IMergeTreeReader::fillVirtualColumns(Columns & columns, size_t rows) const { - chassert(read_task_info != nullptr); - - const IMergeTreeDataPart * part = read_task_info->data_part.get(); - if (part->isProjectionPart()) - part = part->getParentPart(); + const auto * loaded_part_info = typeid_cast(data_part_info_for_read.get()); + if (!loaded_part_info) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Filling of virtual columns is supported only for LoadedMergeTreeDataPartInfoForReader"); + const auto & data_part = loaded_part_info->getDataPart(); const auto & storage_columns = storage_snapshot->getMetadataForQuery()->getColumns(); const auto & virtual_columns = storage_snapshot->virtual_columns; @@ -90,11 +91,21 @@ void IMergeTreeReader::fillVirtualColumns(Columns & columns, size_t rows) const continue; if (!it->type->equals(*virtual_column->type)) + { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Data type for virtual column {} mismatched. Requested type: {}, Virtual column type: {}", + "Data type for virtual column {} mismatched. Requested type: {}, virtual column type: {}", it->name, it->type->getName(), virtual_column->type->getName()); + } + + if (it->name == "_part_offset" || it->name == BlockOffsetColumn::name) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Virtual column {} must be filled by range reader", it->name); + + Field field; + if (auto field_it = virtual_fields.find(it->name); field_it != virtual_fields.end()) + field = field_it->second; + else + field = getFieldForConstVirtualColumn(it->name, *data_part); - auto field = getFieldForConstVirtualColumn(it->name, *part, read_task_info->part_index_in_query); columns[pos] = virtual_column->type->createColumnConst(rows, field)->convertToFullColumnIfConst(); } } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 3fe9853fced..32877cd88eb 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -2,7 +2,6 @@ #include #include -#include "DataTypes/Serializations/ISerialization.h" #include #include #include @@ -10,8 +9,6 @@ namespace DB { -class IDataType; - /// Reads the data between pairs of marks in the same part. When reading consecutive ranges, avoids unnecessary seeks. /// When ranges are almost consecutive, seeks are fast because they are performed inside the buffer. /// Avoids loading the marks file if it is not needed (e.g. when reading the whole part). @@ -19,12 +16,13 @@ class IMergeTreeReader : private boost::noncopyable { public: using ValueSizeMap = std::map; + using VirtualFields = std::unordered_map; using DeserializeBinaryBulkStateMap = std::map; IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -119,8 +117,8 @@ private: /// Actual columns description in part. const ColumnsDescription & part_columns; - /// Shared information required for reading. - MergeTreeReadTaskInfoPtr read_task_info; + /// TODO: + VirtualFields virtual_fields; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a3a4cb9619f..d5e2aa9f78b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1030,7 +1030,7 @@ Block MergeTreeData::getBlockWithVirtualsForFilter(const MergeTreeData::DataPart for (auto & column : block) { - auto field = getFieldForConstVirtualColumn(column.name, *part, 0); + auto field = getFieldForConstVirtualColumn(column.name, *part); column.column->assumeMutableRef().insert(field); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index ee9fa30d98d..8e3f2e07684 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -33,7 +33,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const NamesAndTypesList & columns_to_read, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, @@ -48,7 +48,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( return std::make_unique( read_info, columns_to_read, - read_task_info_, + virtual_fields, storage_snapshot, uncompressed_cache, mark_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 479cb23c3e0..a97d15a08f3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -32,7 +32,7 @@ public: const NamesAndTypesList & columns, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index d4d04bd55c2..e023ae9be0e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -33,7 +33,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( const NamesAndTypesList & columns_to_read, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields, UncompressedCache * /* uncompressed_cache */, MarkCache * /* mark_cache */, const AlterConversionsPtr & alter_conversions, @@ -48,7 +48,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( read_info, ptr, columns_to_read, - read_task_info_, + virtual_fields, storage_snapshot, mark_ranges, reader_settings); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index d3e6a9a5b27..90b4b0e3471 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -21,7 +21,7 @@ public: const NamesAndTypesList & columns, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 5c8b3f9e357..018b8a35534 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -31,7 +31,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( const NamesAndTypesList & columns_to_read, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, @@ -43,7 +43,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( return std::make_unique( read_info, columns_to_read, - read_task_info_, + virtual_fields, storage_snapshot, uncompressed_cache, mark_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 7bc4fe8c777..a8710dad679 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -27,7 +27,7 @@ public: const NamesAndTypesList & columns, const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const AlterConversionsPtr & alter_conversions, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 07c4b367b2c..63d8288eacf 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -108,6 +108,7 @@ MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetchedReaders::get() MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, @@ -117,6 +118,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 250631c6264..0c8a6716d40 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -18,6 +18,7 @@ class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithCo public: MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index e0f5eb22950..ea75943adf9 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -34,6 +34,7 @@ size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & column MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, @@ -43,6 +44,7 @@ MergeTreeReadPool::MergeTreeReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 44726f3877f..cb0e8a9657f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -26,6 +26,7 @@ public: MergeTreeReadPool( RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index a1d5ddd8729..b63be82e4ca 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -7,6 +7,7 @@ namespace DB MergeTreeReadPoolBase::MergeTreeReadPoolBase( RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, @@ -15,6 +16,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const PoolSettings & pool_settings_, const ContextPtr & context_) : parts_ranges(std::move(parts_)) + , shared_virtual_fields(std::move(shared_virtual_fields_)) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) , actions_settings(actions_settings_) @@ -59,6 +61,9 @@ void MergeTreeReadPoolBase::fillPerPartInfos() reader_settings, /*with_subcolumns=*/true); + read_task_info.const_virtual_fields = shared_virtual_fields; + read_task_info.const_virtual_fields.emplace("_part_index", read_task_info.part_index_in_query); + if (pool_settings.preferred_block_size_bytes > 0) { const auto & result_column_names = read_task_info.task_columns.columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 5ec2cf454c3..1b5bfec5898 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -23,6 +23,7 @@ public: MergeTreeReadPoolBase( RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, @@ -36,6 +37,7 @@ public: protected: /// Initialized in constructor const RangesInDataParts parts_ranges; + const VirtualFields shared_virtual_fields; const StorageSnapshotPtr storage_snapshot; const PrewhereInfoPtr prewhere_info; const ExpressionActionsSettings actions_settings; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index dd341f6f750..4c0391ffa57 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -12,6 +12,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, @@ -21,6 +22,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index 1b846fdb22a..9fedf396a6b 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -11,6 +11,7 @@ public: bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 24c0e2525ff..38035d97f56 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, @@ -22,6 +23,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index d9895b3e14b..ca159edb91c 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -11,6 +11,7 @@ public: MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index a34bd05ae63..01c0a9f91be 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -12,6 +12,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, @@ -21,6 +22,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 207dfb7899a..4fe3f7a699c 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -12,6 +12,7 @@ public: ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, diff --git a/src/Storages/MergeTree/MergeTreeReadTask.cpp b/src/Storages/MergeTree/MergeTreeReadTask.cpp index f70c8f973cf..7bbabf6a18d 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.cpp +++ b/src/Storages/MergeTree/MergeTreeReadTask.cpp @@ -44,7 +44,7 @@ MergeTreeReadTask::Readers MergeTreeReadTask::createReaders( columns_to_read, extras.storage_snapshot, ranges, - read_info, + read_info->const_virtual_fields, extras.uncompressed_cache, extras.mark_cache, read_info->alter_conversions, diff --git a/src/Storages/MergeTree/MergeTreeReadTask.h b/src/Storages/MergeTree/MergeTreeReadTask.h index b69abac5451..509cbccd83e 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.h +++ b/src/Storages/MergeTree/MergeTreeReadTask.h @@ -20,6 +20,8 @@ using MergeTreeBlockSizePredictorPtr = std::shared_ptr; using MergeTreeReaderPtr = std::unique_ptr; +using VirtualFields = std::unordered_map; + enum class MergeTreeReadType { @@ -62,6 +64,8 @@ struct MergeTreeReadTaskInfo MergeTreeReadTaskColumns task_columns; /// Shared initialized size predictor. It is copied for each new task. MergeTreeBlockSizePredictorPtr shared_size_predictor; + /// TODO: comment + VirtualFields const_virtual_fields; }; using MergeTreeReadTaskInfoPtr = std::shared_ptr; @@ -116,7 +120,11 @@ public: }; MergeTreeReadTask( - MergeTreeReadTaskInfoPtr info_, Readers readers_, MarkRanges mark_ranges_, MergeTreeBlockSizePredictorPtr size_predictor_); + MergeTreeReadTaskInfoPtr info_, + Readers readers_, + MarkRanges mark_ranges_, + + MergeTreeBlockSizePredictorPtr size_predictor_); void initializeRangeReaders(const PrewhereExprInfo & prewhere_actions); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 65b578e065d..63824366722 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes MergeTreeReaderCompact::MergeTreeReaderCompact( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -30,7 +30,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( : IMergeTreeReader( data_part_info_for_read_, columns_, - read_task_info_, + virtual_fields_, storage_snapshot_, uncompressed_cache_, mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index c87e4889d26..769e6a08be4 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -21,7 +21,7 @@ public: MergeTreeReaderCompact( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index e628bd76cb4..91fc8966a7a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -19,14 +19,14 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_) : IMergeTreeReader( data_part_info_for_read_, columns_, - read_task_info_, + virtual_fields_, storage_snapshot_, nullptr, nullptr, diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h index 161b615a511..cc1e2e9e4e2 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.h +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -18,7 +18,7 @@ public: MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 8270f2452c2..d34a58a25b0 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -24,7 +24,7 @@ namespace MergeTreeReaderWide::MergeTreeReaderWide( MergeTreeDataPartInfoForReaderPtr data_part_info_, NamesAndTypesList columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, @@ -36,7 +36,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( : IMergeTreeReader( data_part_info_, columns_, - read_task_info_, + virtual_fields_, storage_snapshot_, uncompressed_cache_, mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index ecfaa43a3f8..a9a5526dd65 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -17,7 +17,7 @@ public: MergeTreeReaderWide( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, - const MergeTreeReadTaskInfoPtr & read_task_info_, + const VirtualFields & virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index d94d4506ee6..8d0264cd16e 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; extern const int QUERY_WAS_CANCELLED; - extern const int NO_SUCH_COLUMN_IN_TABLE; } MergeTreeSelectProcessor::MergeTreeSelectProcessor( diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 73689e15cd4..35f5782b95a 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -182,8 +182,8 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( columns_for_reader, storage_snapshot, *mark_ranges, - /* read_task_info = */ nullptr, - /* uncompressed_cache = */ nullptr, + /*virtual_fields=*/ {}, + /*uncompressed_cache=*/{}, mark_cache.get(), alter_conversions, reader_settings, diff --git a/src/Storages/MergeTreeVirtualColumns.cpp b/src/Storages/MergeTreeVirtualColumns.cpp index 8c41ab2d16e..94168ccd95a 100644 --- a/src/Storages/MergeTreeVirtualColumns.cpp +++ b/src/Storages/MergeTreeVirtualColumns.cpp @@ -31,11 +31,8 @@ const String BlockOffsetColumn::name = "_block_offset"; const DataTypePtr BlockOffsetColumn::type = std::make_shared(); const ASTPtr BlockOffsetColumn::codec = getCompressionCodecDeltaLZ4(); -Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part, UInt64 part_index) +Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part) { - if (column_name == "_part_offset" || column_name == BlockOffsetColumn::name) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} is not const and must be filled by range reader", column_name); - if (column_name == RowExistsColumn::name) return 1ULL; @@ -45,9 +42,6 @@ Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTree if (column_name == "_part") return part.name; - if (column_name == "_part_index") - return part_index; - if (column_name == "_part_uuid") return part.uuid; @@ -57,7 +51,7 @@ Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTree if (column_name == "_partition_value") return Tuple(part.partition.value.begin(), part.partition.value.end()); - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Unexpected virtual column name: {}", column_name); + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Unexpected const virtual column: {}", column_name); } } diff --git a/src/Storages/MergeTreeVirtualColumns.h b/src/Storages/MergeTreeVirtualColumns.h index 3ee22028d1f..cd9fe544ed8 100644 --- a/src/Storages/MergeTreeVirtualColumns.h +++ b/src/Storages/MergeTreeVirtualColumns.h @@ -28,6 +28,6 @@ struct BlockOffsetColumn static const ASTPtr codec; }; -Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part, UInt64 part_index); +Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part); } From 6656cdb9fbd8aef2d7efaef501c61be37b9f1be4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 21 Feb 2024 17:24:12 +0000 Subject: [PATCH 019/197] refactoring of virtual columns --- .../MergeTree/MergeTreeBlockReadUtils.cpp | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 8eb714c7e24..f97e07751e0 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -7,6 +7,7 @@ #include #include #include "Storages/ColumnsDescription.h" +#include "Storages/MergeTreeVirtualColumns.h" #include #include #include @@ -107,21 +108,14 @@ NameSet injectRequiredColumns( auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) .withExtendedObjects() - .withVirtuals(VirtualsKind::Persistent) + .withVirtuals() .withSubcolumns(with_subcolumns); - auto virtuals_options = GetColumnsOptions(GetColumnsOptions::None).withVirtuals(); - for (size_t i = 0; i < columns.size(); ++i) { /// We are going to fetch physical columns and system columns first if (!storage_snapshot->tryGetColumn(options, columns[i])) - { - if (storage_snapshot->tryGetColumn(virtuals_options, columns[i])) - continue; - else - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column or subcolumn {} in table", columns[i]); - } + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column or subcolumn {} in table", columns[i]); have_at_least_one_physical_column |= injectRequiredColumnsRecursively( columns[i], storage_snapshot, alter_conversions, @@ -285,12 +279,20 @@ MergeTreeReadTaskColumns getReadTaskColumns( .withVirtuals() .withSubcolumns(with_subcolumns); - bool has_part_offset = std::find(required_columns.begin(), required_columns.end(), "_part_offset") != required_columns.end(); + static const NameSet columns_to_read_at_first_step = {"_part_offset", BlockOffsetColumn::name}; + NameSet columns_from_previous_steps; auto add_step = [&](const PrewhereExprStep & step) { Names step_column_names; + if (columns_from_previous_steps.empty()) + { + for (const auto & required_column : required_columns) + if (columns_to_read_at_first_step.contains(required_column)) + step_column_names.push_back(required_column); + } + /// Computation results from previous steps might be used in the current step as well. In such a case these /// computed columns will be present in the current step inputs. They don't need to be read from the disk so /// exclude them from the list of columns to read. This filtering must be done before injecting required @@ -301,13 +303,6 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (!columns_from_previous_steps.contains(name)) step_column_names.push_back(name); - /// Make sure _part_offset is read in STEP 0 - if (columns_from_previous_steps.empty() && has_part_offset) - { - if (std::find(step_column_names.begin(), step_column_names.end(), "_part_offset") == step_column_names.end()) - step_column_names.push_back("_part_offset"); - } - if (!step_column_names.empty()) injectRequiredColumns( data_part_info_for_reader, storage_snapshot, From 2836d0bb55ff5a5eae3f397e3964020e84320b5f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 21 Feb 2024 18:46:42 +0100 Subject: [PATCH 020/197] Update run.sh --- docker/test/upgrade/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 262334cf7ee..9fd54995ab4 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -79,13 +79,13 @@ remove_keeper_config "create_if_not_exists" "[01]" #todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \ - | sed "s|azure>|>azure_blob_storage>|" \ + | sed "s|azure|azure_blob_storage|" \ > /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml #todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ - | sed "s|local>|>local_blob_storage>|" \ + | sed "s|local|local_blob_storage|" \ > /etc/clickhouse-server/config.d/storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml @@ -117,13 +117,13 @@ sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-serv #todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \ - | sed "s|azure>|>azure_blob_storage>|" \ + | sed "s|azure|azure_blob_storage|" \ > /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml #todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ - | sed "s|local>|>local_blob_storage>|" \ + | sed "s|local|local_blob_storage|" \ > /etc/clickhouse-server/config.d/storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml From c624e2fa8bb78e186fa00ded925cf5bd24590d7c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 21 Feb 2024 18:56:28 +0000 Subject: [PATCH 021/197] fix style check --- src/Storages/MergeTreeVirtualColumns.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTreeVirtualColumns.cpp b/src/Storages/MergeTreeVirtualColumns.cpp index 94168ccd95a..1a576bef017 100644 --- a/src/Storages/MergeTreeVirtualColumns.cpp +++ b/src/Storages/MergeTreeVirtualColumns.cpp @@ -10,7 +10,6 @@ namespace DB namespace ErrorCodes { extern const int NO_SUCH_COLUMN_IN_TABLE; - extern const int LOGICAL_ERROR; } static ASTPtr getCompressionCodecDeltaLZ4() From d3c6761dcf11d2cf8cbc6f88c26dd0e45af891d2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 23 Feb 2024 12:06:17 +0100 Subject: [PATCH 022/197] Update run.sh --- docker/test/upgrade/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index fe0c1212b4e..79224c581af 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -104,6 +104,8 @@ rm /etc/clickhouse-server/users.d/nonconst_timezone.xml rm /etc/clickhouse-server/users.d/s3_cache_new.xml rm /etc/clickhouse-server/users.d/replicated_ddl_entry.xml +cat /etc/clickhouse-server/config.d/storage_conf.xml + start stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log From 3c5d6c4df042942f5bbc5d5b7612c3ae5f6b4ba8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 25 Feb 2024 15:00:26 +0100 Subject: [PATCH 023/197] Revert "Merge pull request #60373 from azat/tests/detect-io_uring" This reverts commit 648e6ec5a63f777c5c44405d45574aaebf77142b, reversing changes made to 273b4835a4cac50fb5fa57809ea78a343bf2b125. --- src/Storages/StorageFile.cpp | 19 ------------------- tests/clickhouse-test | 27 +++++---------------------- 2 files changed, 5 insertions(+), 41 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7d674fea9ca..595573b566d 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -26,8 +26,6 @@ #include #include #include -#include -#include #include #include @@ -94,7 +92,6 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; extern const int CANNOT_DETECT_FORMAT; extern const int CANNOT_COMPILE_REGEXP; - extern const int UNSUPPORTED_METHOD; } namespace @@ -279,22 +276,6 @@ std::unique_ptr selectReadBuffer( ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); } - else if (read_method == LocalFSReadMethod::io_uring && !use_table_fd) - { -#if USE_LIBURING - auto & reader = context->getIOURingReader(); - if (!reader.isSupported()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "io_uring is not supported by this system"); - - res = std::make_unique( - reader, - Priority{}, - current_path, - context->getSettingsRef().max_read_buffer_size); -#else - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Read method io_uring is only supported in Linux"); -#endif - } else { if (use_table_fd) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f438c6f4f31..9c21f1fd2a2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -13,6 +13,7 @@ import sys import os import os.path import glob +import platform import signal import re import copy @@ -573,27 +574,6 @@ def get_localzone(): return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) -def supports_io_uring(): - return not subprocess.call( - [ - args.binary, - "-q", - "select * from file('/dev/null', 'LineAsString')", - "--storage_file_read_method", - "io_uring", - ], - stdout=subprocess.DEVNULL, - stderr=subprocess.DEVNULL, - ) - - -def get_local_filesystem_methods(): - methods = ["read", "pread", "mmap", "pread_threadpool"] - if supports_io_uring(): - methods.append("io_uring") - return methods - - class SettingsRandomizer: settings = { "max_insert_threads": lambda: 0 @@ -634,7 +614,10 @@ class SettingsRandomizer: 0.2, 0.5, 1, 10 * 1024 * 1024 * 1024 ), "local_filesystem_read_method": lambda: random.choice( - get_local_filesystem_methods() + # Allow to use uring only when running on Linux + ["read", "pread", "mmap", "pread_threadpool", "io_uring"] + if platform.system().lower() == "linux" + else ["read", "pread", "mmap", "pread_threadpool"] ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), From 584ede2fad6ea475be4e9401841cb0d3fc2cb90f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 25 Feb 2024 23:54:11 +0800 Subject: [PATCH 024/197] opt for multiIf with decimal --- src/Functions/multiIf.cpp | 111 ++++++++++++++++++++++++-------------- 1 file changed, 72 insertions(+), 39 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index cb946b55c73..9cb70570762 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -3,12 +3,20 @@ #include #include #include +#include #include #include #include #include #include #include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -242,8 +250,9 @@ public: const auto & settings = context->getSettingsRef(); const WhichDataType which(removeNullable(result_type)); - bool execute_multiif_columnar - = settings.allow_execute_multiif_columnar && !contains_short && (which.isInt() || which.isUInt() || which.isFloat()); + bool execute_multiif_columnar = settings.allow_execute_multiif_columnar && !contains_short && instructions.size() <= std::numeric_limits::max() + && (which.isInt() || which.isUInt() || which.isFloat() || which.isDecimal() || which.isDateOrDate32OrDateTimeOrDateTime64() + || which.isEnum() || which.isIPv4() || which.isIPv6()); size_t rows = input_rows_count; if (!execute_multiif_columnar) @@ -253,36 +262,58 @@ public: return std::move(res); } -#define EXECUTE_INSTRUCTIONS_COLUMNAR(TYPE, INDEX) \ +#define EXECUTE_INSTRUCTIONS_COLUMNAR(TYPE, FIELD, INDEX) \ if (which.is##TYPE()) \ { \ - MutableColumnPtr res = ColumnVector::create(rows); \ - MutableColumnPtr null_map = result_type->isNullable() ? ColumnUInt8::create(rows) : nullptr; \ - executeInstructionsColumnar(instructions, rows, res, null_map, result_type->isNullable()); \ - if (!result_type->isNullable()) \ - return std::move(res); \ + MutableColumnPtr res = result_type->createColumn(); \ + res->reserve(rows); \ + if (result_type->isNullable()) \ + { \ + auto & res_nullable = assert_cast(*res); \ + auto & res_data = assert_cast &>(res_nullable.getNestedColumn()).getData(); \ + auto & res_null_map = res_nullable.getNullMapData(); \ + executeInstructionsColumnar(instructions, rows, res_data, &res_null_map); \ + } \ else \ - return ColumnNullable::create(std::move(res), std::move(null_map)); \ + { \ + auto & res_data = assert_cast &>(*res).getData(); \ + executeInstructionsColumnar(instructions, rows, res_data, nullptr); \ + } \ + return std::move(res); \ } #define ENUMERATE_NUMERIC_TYPES(M, INDEX) \ - M(UInt8, INDEX) \ - M(UInt16, INDEX) \ - M(UInt32, INDEX) \ - M(UInt64, INDEX) \ - M(Int8, INDEX) \ - M(Int16, INDEX) \ - M(Int32, INDEX) \ - M(Int64, INDEX) \ - M(UInt128, INDEX) \ - M(UInt256, INDEX) \ - M(Int128, INDEX) \ - M(Int256, INDEX) \ - M(Float32, INDEX) \ - M(Float64, INDEX) \ + M(UInt8, UInt8, INDEX) \ + M(UInt16, UInt16, INDEX) \ + M(UInt32, UInt32, INDEX) \ + M(UInt64, UInt64, INDEX) \ + M(Int8, Int8, INDEX) \ + M(Int16, Int16, INDEX) \ + M(Int32, Int32, INDEX) \ + M(Int64, Int64, INDEX) \ + M(Float32, Float32, INDEX) \ + M(Float64, Float64, INDEX) \ + M(UInt128, UInt128, INDEX) \ + M(UInt256, UInt256, INDEX) \ + M(Int128, Int128, INDEX) \ + M(Int256, Int256, INDEX) \ + M(Decimal32, Decimal32, INDEX) \ + M(Decimal64, Decimal64, INDEX) \ + M(Decimal128, Decimal128, INDEX) \ + M(Decimal256, Decimal256, INDEX) \ + M(Date, UInt16, INDEX) \ + M(Date32, Int32, INDEX) \ + M(DateTime, UInt32, INDEX) \ + M(DateTime64, DateTime64, INDEX) \ + M(Enum8, Int8, INDEX) \ + M(Enum16, Int16, INDEX) \ + M(IPv4, IPv4, INDEX) \ + M(IPv6, IPv6, INDEX) \ throw Exception( \ ErrorCodes::NOT_IMPLEMENTED, "Columnar execution of function {} not implemented for type {}", getName(), result_type->getName()); + ENUMERATE_NUMERIC_TYPES(EXECUTE_INSTRUCTIONS_COLUMNAR, UInt8) + /* size_t num_instructions = instructions.size(); if (num_instructions <= std::numeric_limits::max()) { @@ -299,7 +330,10 @@ public: else throw Exception( ErrorCodes::LOGICAL_ERROR, "Instruction size({}) of function {} is out of range", getName(), result_type->getName()); + */ } +#undef ENUMERATE_NUMERIC_TYPES +#undef EXECUTE_INSTRUCTIONS_COLUMNAR private: @@ -382,13 +416,16 @@ private: } template - static void executeInstructionsColumnar(std::vector & instructions, size_t rows, const MutableColumnPtr & res, const MutableColumnPtr & null_map, bool nullable) + static void executeInstructionsColumnar( + std::vector & instructions, + size_t rows, + PaddedPODArray & res_data, + PaddedPODArray * res_null_map = nullptr) { PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); - PaddedPODArray & res_data = assert_cast &>(*res).getData(); - if (!nullable) + if (!res_null_map) { for (size_t row_i = 0; row_i < rows; ++row_i) { @@ -399,10 +436,9 @@ private: } else { - PaddedPODArray & null_map_data = assert_cast(*null_map).getData(); - std::vector data_cols(instructions.size()); + std::vector data_cols(instructions.size()); std::vector null_map_cols(instructions.size()); - ColumnPtr shared_null_map_col = nullptr; + PaddedPODArray shared_null_map(rows, 0); for (size_t i = 0; i < instructions.size(); ++i) { if (instructions[i].source->isNullable()) @@ -416,24 +452,21 @@ private: nullable_col = assert_cast(data_column.get()); } null_map_cols[i] = assert_cast(*nullable_col->getNullMapColumnPtr()).getData().data(); - data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); + data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); } else { - if (!shared_null_map_col) - { - shared_null_map_col = ColumnUInt8::create(rows, 0); - } - null_map_cols[i] = assert_cast(*shared_null_map_col).getData().data(); - data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); + null_map_cols[i] = shared_null_map.data(); + data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); } } for (size_t row_i = 0; row_i < rows; ++row_i) { - auto & instruction = instructions[inserts[row_i]]; + S insert = inserts[row_i]; + auto & instruction = instructions[insert]; size_t index = instruction.source_is_constant ? 0 : row_i; - res_data[row_i] = *(data_cols[inserts[row_i]] + index); - null_map_data[row_i] = *(null_map_cols[inserts[row_i]] + index); + res_data[row_i] = *(data_cols[insert] + index); + (*res_null_map)[row_i] = *(null_map_cols[insert] + index); } } } From ab6ab8bfac4c0b5f2b941b678591cc7d318d6bdf Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 25 Feb 2024 17:11:22 +0000 Subject: [PATCH 025/197] replace only secret arguments with [HIDDEN], also hide it in TreeNode dump --- src/Analyzer/ConstantNode.cpp | 4 +- src/Analyzer/IQueryTreeNode.cpp | 2 + src/Analyzer/IQueryTreeNode.h | 6 + src/Analyzer/Passes/QueryAnalysisPass.cpp | 27 +- src/Parsers/ASTFunction.cpp | 503 +------------------ src/Parsers/FunctionSecretArgumentsFinder.h | 514 ++++++++++++++++++++ 6 files changed, 539 insertions(+), 517 deletions(-) create mode 100644 src/Parsers/FunctionSecretArgumentsFinder.h diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 69bed3dbe90..837c05ff5cb 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -45,10 +45,10 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state if (hasAlias()) buffer << ", alias: " << getAlias(); - buffer << ", constant_value: " << constant_value->getValue().dump(); + buffer << ", constant_value: " << (is_masked ? "[HIDDEN]" : constant_value->getValue().dump()); buffer << ", constant_value_type: " << constant_value->getType()->getName(); - if (getSourceExpression()) + if (!is_masked && getSourceExpression()) { buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION" << '\n'; getSourceExpression()->dumpTreeImpl(buffer, format_state, indent + 4); diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index d61cb0ffab1..3ef323c9648 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -45,12 +45,14 @@ const char * toString(QueryTreeNodeType type) } IQueryTreeNode::IQueryTreeNode(size_t children_size, size_t weak_pointers_size) + : is_masked(false) { children.resize(children_size); weak_pointers.resize(weak_pointers_size); } IQueryTreeNode::IQueryTreeNode(size_t children_size) + : is_masked(false) { children.resize(children_size); } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index b07aa2d31b0..c08a8860749 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -251,6 +251,11 @@ public: return children; } + void setMasked(bool masked = true) + { + is_masked = masked; + } + protected: /** Construct query tree node. * Resize children to children size. @@ -281,6 +286,7 @@ protected: QueryTreeNodes children; QueryTreeWeakNodes weak_pointers; + bool is_masked; private: String alias; diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index e25dbf52669..31f07b4ec77 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -1373,7 +1374,7 @@ private: ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); + ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const std::pair & secrets = std::pair()); ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope); @@ -5111,22 +5112,17 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } /// Resolve function arguments - + FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinder{function_node_ptr->toAST()->as()}.getResult(); bool allow_table_expressions = is_special_function_in; auto arguments_projection_names = resolveExpressionNodeList(function_node_ptr->getArgumentsNode(), scope, true /*allow_lambda_expression*/, - allow_table_expressions /*allow_table_expression*/); + allow_table_expressions /*allow_table_expression*/, + {secret_arguments.start, secret_arguments.count}); + + for (size_t n = secret_arguments.start; n < secret_arguments.start + secret_arguments.count; ++n) + arguments_projection_names[n] = "[HIDDEN]"; - if (function_node_ptr->toAST()->hasSecretParts()) - { - for (auto & argument : arguments_projection_names) - { - SipHash hash; - hash.update(argument); - argument = getHexUIntLowercase(hash.get128()); - } - } auto & function_node = *function_node_ptr; /// Replace right IN function argument if it is table or table function with subquery that read ordinary columns @@ -6111,7 +6107,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id * Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table; * Example: SELECT *** FROM system.one; */ -ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) +ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const std::pair & secrets) { auto & node_list_typed = node_list->as(); size_t node_list_size = node_list_typed.getNodes().size(); @@ -6121,10 +6117,13 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node ProjectionNames result_projection_names; + size_t n = 0; for (auto & node : node_list_typed.getNodes()) { auto node_to_resolve = node; auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression); + if (n >= secrets.first && n < secrets.first + secrets.second) + node_to_resolve->setMasked(); size_t expected_projection_names_size = 1; if (auto * expression_list = node_to_resolve->as()) @@ -6146,6 +6145,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node result_projection_names.insert(result_projection_names.end(), expression_node_projection_names.begin(), expression_node_projection_names.end()); expression_node_projection_names.clear(); + + ++n; } node_list_typed.getNodes() = std::move(result_nodes); diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index ba4c7db96e6..7468c41910a 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -36,508 +37,6 @@ namespace ErrorCodes } -namespace -{ - /// Finds arguments of a specified function which should not be displayed for most users for security reasons. - /// That involves passwords and secret keys. - class FunctionSecretArgumentsFinder - { - public: - explicit FunctionSecretArgumentsFinder(const ASTFunction & function_) : function(function_) - { - if (!function.arguments) - return; - - const auto * expr_list = function.arguments->as(); - if (!expr_list) - return; - - arguments = &expr_list->children; - switch (function.kind) - { - case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; - case ASTFunction::Kind::WINDOW_FUNCTION: break; - case ASTFunction::Kind::LAMBDA_FUNCTION: break; - case ASTFunction::Kind::TABLE_ENGINE: findTableEngineSecretArguments(); break; - case ASTFunction::Kind::DATABASE_ENGINE: findDatabaseEngineSecretArguments(); break; - case ASTFunction::Kind::BACKUP_NAME: findBackupNameSecretArguments(); break; - } - } - - struct Result - { - /// Result constructed by default means no arguments will be hidden. - size_t start = static_cast(-1); - size_t count = 0; /// Mostly it's either 0 or 1. There are only a few cases where `count` can be greater than 1 (e.g. see `encrypt`). - /// In all known cases secret arguments are consecutive - bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. - /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` - std::vector nested_maps; - - bool hasSecrets() const - { - return count != 0 || !nested_maps.empty(); - } - }; - - Result getResult() const { return result; } - - private: - const ASTFunction & function; - const ASTs * arguments = nullptr; - Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments->size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findOrdinaryFunctionSecretArguments() - { - if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || - (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((function.name == "remote") || (function.name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((function.name == "encrypt") || (function.name == "decrypt") || - (function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") || - (function.name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (function.name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - size_t count = arguments->size(); - while (count > 0) - { - const ASTFunction * f = arguments->at(count - 1)->as(); - if (!f) - break; - if (f->name == "headers") - result.nested_maps.push_back(f->name); - else if (f->name != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments->size()) - return false; - - return tryGetStringFromArgument(*(*arguments)[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const IAST & argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument.as()) - { - if (literal->value.getType() != Field::Types::String) - return false; - if (res) - *res = literal->value.safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument.as()) - { - if (res) - *res = id->name(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments->size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = (*arguments)[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments->empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments->size() - 1; - } - - void findTableEngineSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "ExternalDistributed") - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - findExternalDistributedTableEngineSecretArguments(); - } - else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) - { - /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) - /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || - (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) - { - /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) - findS3TableEngineSecretArguments(); - } - else if (engine_name == "URL") - { - findURLSecretArguments(); - } - } - - void findExternalDistributedTableEngineSecretArguments() - { - if (isNamedCollectionName(1)) - { - /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 2); - } - else - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - markSecretArgument(5); - } - } - - void findS3TableEngineSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'secret_access_key') - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((3 <= count) && (count <= 4)) - { - String second_arg; - if (tryGetStringFromArgument(1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (count == 3) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: S3('url', 'format', ...) - } - } - } - - /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (2 < count) - markSecretArgument(2); - } - - void findDatabaseEngineSecretArguments() - { - const String & engine_name = function.name; - if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || - (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL")) - { - /// MySQL('host:port', 'database', 'user', 'password') - /// PostgreSQL('host:port', 'database', 'user', 'password') - findMySQLDatabaseSecretArguments(); - } - else if (engine_name == "S3") - { - /// S3('url', 'access_key_id', 'secret_access_key') - findS3DatabaseSecretArguments(); - } - } - - void findMySQLDatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// MySQL(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// MySQL('host:port', 'database', 'user', 'password') - markSecretArgument(3); - } - } - - void findS3DatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'password', ...) - findSecretNamedArgument("secret_access_key", 1); - } - else - { - /// S3('url', 'access_key_id', 'secret_access_key') - markSecretArgument(2); - } - } - - void findBackupNameSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "S3") - { - /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) - markSecretArgument(2); - } - } - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments->size() <= arg_idx) - return false; - - const auto * identifier = (*arguments)[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments->size(); ++i) - { - const auto & argument = (*arguments)[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->name != "equals")) - continue; - - const auto * expr_list = equals_func->arguments->as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->children; - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(*equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } - }; -} - - void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const { /// These functions contain some unexpected ASTs in arguments (e.g. SETTINGS or even a SELECT query) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h new file mode 100644 index 00000000000..355dd99a21a --- /dev/null +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -0,0 +1,514 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + + +/// Finds arguments of a specified function which should not be displayed for most users for security reasons. +/// That involves passwords and secret keys. +class FunctionSecretArgumentsFinder +{ +public: + explicit FunctionSecretArgumentsFinder(const ASTFunction & function_) : function(function_) + { + if (!function.arguments) + return; + + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return; + + arguments = &expr_list->children; + switch (function.kind) + { + case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; + case ASTFunction::Kind::WINDOW_FUNCTION: break; + case ASTFunction::Kind::LAMBDA_FUNCTION: break; + case ASTFunction::Kind::TABLE_ENGINE: findTableEngineSecretArguments(); break; + case ASTFunction::Kind::DATABASE_ENGINE: findDatabaseEngineSecretArguments(); break; + case ASTFunction::Kind::BACKUP_NAME: findBackupNameSecretArguments(); break; + } + } + + struct Result + { + /// Result constructed by default means no arguments will be hidden. + size_t start = static_cast(-1); + size_t count = 0; /// Mostly it's either 0 or 1. There are only a few cases where `count` can be greater than 1 (e.g. see `encrypt`). + /// In all known cases secret arguments are consecutive + bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. + /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` + std::vector nested_maps; + + bool hasSecrets() const + { + return count != 0 || !nested_maps.empty(); + } + }; + + Result getResult() const { return result; } + +private: + const ASTFunction & function; + const ASTs * arguments = nullptr; + Result result; + + void markSecretArgument(size_t index, bool argument_is_named = false) + { + if (index >= arguments->size()) + return; + if (!result.count) + { + result.start = index; + result.are_named = argument_is_named; + } + chassert(index >= result.start); /// We always check arguments consecutively + result.count = index + 1 - result.start; + if (!argument_is_named) + result.are_named = false; + } + + void findOrdinaryFunctionSecretArguments() + { + if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb")) + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) + /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || + (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function.name == "s3Cluster") + { + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ true); + } + else if ((function.name == "remote") || (function.name == "remoteSecure")) + { + /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) + findRemoteFunctionSecretArguments(); + } + else if ((function.name == "encrypt") || (function.name == "decrypt") || + (function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") || + (function.name == "tryDecrypt")) + { + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) + findEncryptionFunctionSecretArguments(); + } + else if (function.name == "url") + { + findURLSecretArguments(); + } + } + + void findMySQLFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// mysql(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + markSecretArgument(4); + } + } + + /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should + /// always be at the end). Marks "headers" as secret, if found. + size_t excludeS3OrURLNestedMaps() + { + size_t count = arguments->size(); + while (count > 0) + { + const ASTFunction * f = arguments->at(count - 1)->as(); + if (!f) + break; + if (f->name == "headers") + result.nested_maps.push_back(f->name); + else if (f->name != "extra_credentials") + break; + count -= 1; + } + return count; + } + + void findS3FunctionSecretArguments(bool is_cluster_function) + { + /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (url_arg_idx + 2 < count) + markSecretArgument(url_arg_idx + 2); + } + + void findURLSecretArguments() + { + if (!isNamedCollectionName(0)) + excludeS3OrURLNestedMaps(); + } + + bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const + { + if (arg_idx >= arguments->size()) + return false; + + return tryGetStringFromArgument(*(*arguments)[arg_idx], res, allow_identifier); + } + + static bool tryGetStringFromArgument(const IAST & argument, String * res, bool allow_identifier = true) + { + if (const auto * literal = argument.as()) + { + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument.as()) + { + if (res) + *res = id->name(); + return true; + } + } + + return false; + } + + void findRemoteFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// remote(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + return; + } + + /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: + /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// remote('addresses_expr', db.table) + if (arguments->size() < 3) + return; + + size_t arg_num = 1; + + /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. + const auto * table_function = (*arguments)[arg_num]->as(); + if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) + { + ++arg_num; + } + else + { + std::optional database; + std::optional qualified_table_name; + if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `user`. + markSecretArgument(arg_num + 2); + } + if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `sharding_key`. + markSecretArgument(arg_num + 3); + } + return; + } + + /// Skip the current argument (which is either a database name or a qualified table name). + ++arg_num; + if (database) + { + /// Skip the 'table' argument if the previous argument was a database name. + ++arg_num; + } + } + + /// Skip username. + ++arg_num; + + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); + if (can_be_password) + markSecretArgument(arg_num); + } + + /// Tries to get either a database name or a qualified table name from an argument. + /// Empty string is also allowed (it means the default database). + /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. + bool tryGetDatabaseNameOrQualifiedTableName( + size_t arg_idx, + std::optional & res_database, + std::optional & res_qualified_table_name) const + { + res_database.reset(); + res_qualified_table_name.reset(); + + String str; + if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) + return false; + + if (str.empty()) + { + res_database = ""; + return true; + } + + auto qualified_table_name = QualifiedTableName::tryParseFromString(str); + if (!qualified_table_name) + return false; + + if (qualified_table_name->database.empty()) + res_database = std::move(qualified_table_name->table); + else + res_qualified_table_name = std::move(qualified_table_name); + return true; + } + + void findEncryptionFunctionSecretArguments() + { + if (arguments->empty()) + return; + + /// We replace all arguments after 'mode' with '[HIDDEN]': + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') + result.start = 1; + result.count = arguments->size() - 1; + } + + void findTableEngineSecretArguments() + { + const String & engine_name = function.name; + if (engine_name == "ExternalDistributed") + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + findExternalDistributedTableEngineSecretArguments(); + } + else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) + { + /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) + /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || + (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) + { + /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) + findS3TableEngineSecretArguments(); + } + else if (engine_name == "URL") + { + findURLSecretArguments(); + } + } + + void findExternalDistributedTableEngineSecretArguments() + { + if (isNamedCollectionName(1)) + { + /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 2); + } + else + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + markSecretArgument(5); + } + } + + void findS3TableEngineSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'secret_access_key') + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((3 <= count) && (count <= 4)) + { + String second_arg; + if (tryGetStringFromArgument(1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (count == 3) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: S3('url', 'format', ...) + } + } + } + + /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (2 < count) + markSecretArgument(2); + } + + void findDatabaseEngineSecretArguments() + { + const String & engine_name = function.name; + if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || + (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL")) + { + /// MySQL('host:port', 'database', 'user', 'password') + /// PostgreSQL('host:port', 'database', 'user', 'password') + findMySQLDatabaseSecretArguments(); + } + else if (engine_name == "S3") + { + /// S3('url', 'access_key_id', 'secret_access_key') + findS3DatabaseSecretArguments(); + } + } + + void findMySQLDatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// MySQL(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// MySQL('host:port', 'database', 'user', 'password') + markSecretArgument(3); + } + } + + void findS3DatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'password', ...) + findSecretNamedArgument("secret_access_key", 1); + } + else + { + /// S3('url', 'access_key_id', 'secret_access_key') + markSecretArgument(2); + } + } + + void findBackupNameSecretArguments() + { + const String & engine_name = function.name; + if (engine_name == "S3") + { + /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) + markSecretArgument(2); + } + } + + /// Whether a specified argument can be the name of a named collection? + bool isNamedCollectionName(size_t arg_idx) const + { + if (arguments->size() <= arg_idx) + return false; + + const auto * identifier = (*arguments)[arg_idx]->as(); + return identifier != nullptr; + } + + /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + { + for (size_t i = start; i < arguments->size(); ++i) + { + const auto & argument = (*arguments)[i]; + const auto * equals_func = argument->as(); + if (!equals_func || (equals_func->name != "equals")) + continue; + + const auto * expr_list = equals_func->arguments->as(); + if (!expr_list) + continue; + + const auto & equal_args = expr_list->children; + if (equal_args.size() != 2) + continue; + + String found_key; + if (!tryGetStringFromArgument(*equal_args[0], &found_key)) + continue; + + if (found_key == key) + markSecretArgument(i, /* argument_is_named= */ true); + } + } +}; + +} From 18eb75f4d2243f6c323c342c305f958bb37f4e3c Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Thu, 28 Dec 2023 14:13:52 +0800 Subject: [PATCH 026/197] lazy build join output --- src/Interpreters/HashJoin.cpp | 76 +++++++++++++------ .../performance/storage_join_direct_join.xml | 3 + 2 files changed, 54 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 33dc178ca00..fbd9d522f47 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1034,6 +1034,12 @@ public: } }; + struct LazyOutput + { + PaddedPODArray blocks; + PaddedPODArray row_nums; + }; + AddedColumns( const Block & left_block, const Block & block_with_columns_to_add, @@ -1049,10 +1055,12 @@ public: size_t num_columns_to_add = block_with_columns_to_add.columns(); if (is_asof_join) ++num_columns_to_add; - + has_columns_to_add = num_columns_to_add > 0; columns.reserve(num_columns_to_add); type_name.reserve(num_columns_to_add); right_indexes.reserve(num_columns_to_add); + lazy_output.blocks.reserve(rows_to_add); + lazy_output.row_nums.reserve(rows_to_add); for (const auto & src_column : block_with_columns_to_add) { @@ -1089,6 +1097,34 @@ public: size_t size() const { return columns.size(); } + void buildOutput() + { + for (size_t i = 0; i < this->size(); ++i) + { + auto& col = columns[i]; + for (size_t j = 0; j < lazy_output.blocks.size(); ++j) + { + if (!lazy_output.blocks[j]) + { + type_name[i].type->insertDefaultInto(*col); + continue; + } + const auto & column_from_block = reinterpret_cast(lazy_output.blocks[j])->getByPosition(right_indexes[i]); + /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. + if (is_join_get) + { + if (auto * nullable_col = typeid_cast(col.get()); + nullable_col && !column_from_block.column->isNullable()) + { + nullable_col->insertFromNotNullable(*column_from_block.column, lazy_output.row_nums[j]); + continue; + } + } + col->insertFrom(*column_from_block.column, lazy_output.row_nums[j]); + } + } + } + ColumnWithTypeAndName moveColumn(size_t i) { return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); @@ -1098,9 +1134,6 @@ public: template void appendFromBlock(const Block & block, size_t row_num) { - if constexpr (has_defaults) - applyLazyDefaults(); - #ifndef NDEBUG for (size_t j = 0; j < right_indexes.size(); ++j) { @@ -1126,33 +1159,20 @@ public: demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); } #endif - - if (is_join_get) + if (has_columns_to_add) { - size_t right_indexes_size = right_indexes.size(); - for (size_t j = 0; j < right_indexes_size; ++j) - { - const auto & column_from_block = block.getByPosition(right_indexes[j]); - if (auto * nullable_col = nullable_column_ptrs[j]) - nullable_col->insertFromNotNullable(*column_from_block.column, row_num); - else - columns[j]->insertFrom(*column_from_block.column, row_num); - } - } - else - { - size_t right_indexes_size = right_indexes.size(); - for (size_t j = 0; j < right_indexes_size; ++j) - { - const auto & column_from_block = block.getByPosition(right_indexes[j]); - columns[j]->insertFrom(*column_from_block.column, row_num); - } + lazy_output.blocks.emplace_back(reinterpret_cast(&block)); + lazy_output.row_nums.emplace_back(static_cast(row_num)); } } void appendDefaultRow() { - ++lazy_defaults_count; + if (has_columns_to_add) + { + lazy_output.blocks.emplace_back(0); + lazy_output.row_nums.emplace_back(0); + } } void applyLazyDefaults() @@ -1169,6 +1189,10 @@ public: std::vector join_on_keys; + // The default row is represented by an empty RowRef, so that fixed-size blocks can be generated sequentially, + // default_count cannot represent the position of the row + LazyOutput lazy_output; + size_t max_joined_block_rows = 0; size_t rows_to_add; std::unique_ptr offsets_to_replicate; @@ -1198,6 +1222,7 @@ private: std::vector right_indexes; size_t lazy_defaults_count = 0; + bool has_columns_to_add; /// for ASOF const IColumn * left_asof_key = nullptr; @@ -1702,6 +1727,7 @@ Block HashJoin::joinBlockImpl( added_columns.join_on_keys.clear(); Block remaining_block = sliceBlock(block, num_joined); + added_columns.buildOutput(); for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index 2fc63c2c926..70e55ff82a4 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -14,6 +14,9 @@ toString(number), toString(number) FROM numbers(1000000); SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; + SELECT keys.key, value1, value2, value3 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; + SELECT keys.key, value1, value2, value3, value4, value5 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; + SELECT keys.key, value1, value2, value3, value4, value5, value6, value7, value8, value9, FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS allow_experimental_analyzer=1 \ No newline at end of file From 5c42f2c0411ba9a3604405d41370304a26a6f542 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 29 Dec 2023 15:29:54 +0800 Subject: [PATCH 027/197] optimize default value --- src/Interpreters/HashJoin.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index fbd9d522f47..f994336cd53 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1102,13 +1102,20 @@ public: for (size_t i = 0; i < this->size(); ++i) { auto& col = columns[i]; + size_t default_count = 0; for (size_t j = 0; j < lazy_output.blocks.size(); ++j) { if (!lazy_output.blocks[j]) { - type_name[i].type->insertDefaultInto(*col); +// type_name[i].type->insertDefaultInto(*col); + default_count ++; continue; } + if (default_count > 0) + { + JoinCommon::addDefaultValues(*col, type_name[i].type, default_count); + default_count = 0; + } const auto & column_from_block = reinterpret_cast(lazy_output.blocks[j])->getByPosition(right_indexes[i]); /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. if (is_join_get) @@ -1122,6 +1129,11 @@ public: } col->insertFrom(*column_from_block.column, lazy_output.row_nums[j]); } + if (default_count > 0) + { + JoinCommon::addDefaultValues(*col, type_name[i].type, default_count); + default_count = 0; + } } } From 3a2c9a8793918fb84d25a893f525a82a62d35691 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 2 Jan 2024 10:06:31 +0800 Subject: [PATCH 028/197] update --- src/Interpreters/HashJoin.cpp | 56 +++++++------------ .../performance/storage_join_direct_join.xml | 3 - 2 files changed, 20 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f994336cd53..ca6aa414bc9 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1103,19 +1103,23 @@ public: { auto& col = columns[i]; size_t default_count = 0; - for (size_t j = 0; j < lazy_output.blocks.size(); ++j) + auto apply_default = [&]() { - if (!lazy_output.blocks[j]) - { -// type_name[i].type->insertDefaultInto(*col); - default_count ++; - continue; - } if (default_count > 0) { JoinCommon::addDefaultValues(*col, type_name[i].type, default_count); default_count = 0; } + }; + + for (size_t j = 0; j < lazy_output.blocks.size(); ++j) + { + if (!lazy_output.blocks[j]) + { + default_count ++; + continue; + } + apply_default(); const auto & column_from_block = reinterpret_cast(lazy_output.blocks[j])->getByPosition(right_indexes[i]); /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. if (is_join_get) @@ -1129,11 +1133,7 @@ public: } col->insertFrom(*column_from_block.column, lazy_output.row_nums[j]); } - if (default_count > 0) - { - JoinCommon::addDefaultValues(*col, type_name[i].type, default_count); - default_count = 0; - } + apply_default(); } } @@ -1143,7 +1143,6 @@ public: } - template void appendFromBlock(const Block & block, size_t row_num) { #ifndef NDEBUG @@ -1187,16 +1186,6 @@ public: } } - void applyLazyDefaults() - { - if (lazy_defaults_count) - { - for (size_t j = 0, size = right_indexes.size(); j < size; ++j) - JoinCommon::addDefaultValues(*columns[j], type_name[j].type, lazy_defaults_count); - lazy_defaults_count = 0; - } - } - const IColumn & leftAsofKey() const { return *left_asof_key; } std::vector join_on_keys; @@ -1233,7 +1222,6 @@ private: std::vector nullable_column_ptrs; std::vector right_indexes; - size_t lazy_defaults_count = 0; bool has_columns_to_add; /// for ASOF const IColumn * left_asof_key = nullptr; @@ -1345,7 +1333,7 @@ public: } }; -template +template void addFoundRowAll( const typename Map::mapped_type & mapped, AddedColumns & added, @@ -1353,9 +1341,6 @@ void addFoundRowAll( KnownRowsHolder & known_rows [[maybe_unused]], JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]]) { - if constexpr (add_missing) - added.applyLazyDefaults(); - if constexpr (multiple_disjuncts) { std::unique_ptr::Type>> new_known_rows_ptr; @@ -1364,7 +1349,7 @@ void addFoundRowAll( { if (!known_rows.isKnown(std::make_pair(it->block, it->row_num))) { - added.appendFromBlock(*it->block, it->row_num); + added.appendFromBlock(*it->block, it->row_num); ++current_offset; if (!new_known_rows_ptr) { @@ -1388,7 +1373,7 @@ void addFoundRowAll( { for (auto it = mapped.begin(); it.ok(); ++it) { - added.appendFromBlock(*it->block, it->row_num); + added.appendFromBlock(*it->block, it->row_num); ++current_offset; } } @@ -1477,7 +1462,7 @@ NO_INLINE size_t joinRightColumns( else used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*row_ref.block, row_ref.row_num); + added_columns.appendFromBlock(*row_ref.block, row_ref.row_num); } else addNotFoundRow(added_columns, current_offset); @@ -1487,7 +1472,7 @@ NO_INLINE size_t joinRightColumns( setUsed(added_columns.filter, i); used_flags.template setUsed(find_result); auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) { @@ -1497,7 +1482,7 @@ NO_INLINE size_t joinRightColumns( { auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; setUsed(added_columns.filter, i); - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } } else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) @@ -1508,7 +1493,7 @@ NO_INLINE size_t joinRightColumns( if (used_once) { setUsed(added_columns.filter, i); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); } break; @@ -1526,7 +1511,7 @@ NO_INLINE size_t joinRightColumns( { setUsed(added_columns.filter, i); used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); if (join_features.is_any_or_semi_join) { @@ -1549,7 +1534,6 @@ NO_INLINE size_t joinRightColumns( } } - added_columns.applyLazyDefaults(); return i; } diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index 70e55ff82a4..2fc63c2c926 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -14,9 +14,6 @@ toString(number), toString(number) FROM numbers(1000000); SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; - SELECT keys.key, value1, value2, value3 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; - SELECT keys.key, value1, value2, value3, value4, value5 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; - SELECT keys.key, value1, value2, value3, value4, value5, value6, value7, value8, value9, FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS allow_experimental_analyzer=1 \ No newline at end of file From dc07f62c2c5c924693336c9553059dbb303cf6a7 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 2 Jan 2024 16:26:01 +0800 Subject: [PATCH 029/197] try new case --- tests/performance/storage_join_direct_join.xml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index 2fc63c2c926..d3bc25cd96b 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -8,12 +8,26 @@ value4 String, value5 String, value6 String, value7 String, value8 String, value9 String, value10 String) ENGINE = Join(ANY, LEFT, key); + CREATE TABLE dict2 (key UInt64, value1 UInt64, value2 Float64, value3 String, + value4 String, value5 String, value6 String, value7 String, value8 String, value9 String, + value10 String) ENGINE = MergeTree ORDER BY key; + INSERT INTO keys SELECT rand() FROM numbers(10000000); INSERT INTO dict SELECT rand(), rand()%1000, rand()*0.0001, toString(number), toString(number), toString(number), toString(number), toString(number), toString(number), toString(number), toString(number) FROM numbers(1000000); + INSERT INTO dict2 SELECT rand(), rand()%1000, rand()*0.0001, toString(number), + toString(number), toString(number), toString(number), toString(number), toString(number), + toString(number), toString(number) FROM numbers(1000000); + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict2 AS d ON (keys.key = d.key) FORMAT Null; + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS allow_experimental_analyzer=1 + + DROP TABLE IF EXISTS keys + DROP TABLE IF EXISTS dict + DROP TABLE IF EXISTS dict2 \ No newline at end of file From 93fc7a293fb9157c8c327dd748144b36ee80fde6 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Thu, 4 Jan 2024 10:27:59 +0800 Subject: [PATCH 030/197] add new any left join case --- tests/performance/any_join.xml | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 tests/performance/any_join.xml diff --git a/tests/performance/any_join.xml b/tests/performance/any_join.xml new file mode 100644 index 00000000000..ed473c36d55 --- /dev/null +++ b/tests/performance/any_join.xml @@ -0,0 +1,23 @@ + + + 1 + + + CREATE TABLE keys (key UInt64) ENGINE = MergeTree ORDER BY key; + CREATE TABLE dict (key UInt64, value1 UInt64, value2 Float64, value3 String, + value4 String, value5 String, value6 String, value7 String, value8 String, value9 String, + value10 String) ENGINE = MergeTree ORDER BY key; + + INSERT INTO keys SELECT rand() %500000 FROM numbers(10000000); + INSERT INTO dict SELECT rand() %300000, rand()%1000, rand()*0.0001, toString(number), + toString(number), toString(number), toString(number), toString(number), toString(number), + toString(number), toString(number) FROM numbers(1000000); + + + + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; + SELECT keys.key, value1 FROM keys ALL INNER JOIN dict AS d ON (keys.key = d.key) FORMAT Null; + + DROP TABLE IF EXISTS keys + DROP TABLE IF EXISTS dict + \ No newline at end of file From b0943ab3e81709bb0db0a077cfe22916c518d5e0 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Thu, 22 Feb 2024 15:34:31 +0800 Subject: [PATCH 031/197] add lazyAddedColumns --- src/Interpreters/HashJoin.cpp | 266 ++++++++++++++++++++++++---------- 1 file changed, 186 insertions(+), 80 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index ca6aa414bc9..1ea12955409 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1034,12 +1034,6 @@ public: } }; - struct LazyOutput - { - PaddedPODArray blocks; - PaddedPODArray row_nums; - }; - AddedColumns( const Block & left_block, const Block & block_with_columns_to_add, @@ -1055,12 +1049,9 @@ public: size_t num_columns_to_add = block_with_columns_to_add.columns(); if (is_asof_join) ++num_columns_to_add; - has_columns_to_add = num_columns_to_add > 0; columns.reserve(num_columns_to_add); type_name.reserve(num_columns_to_add); right_indexes.reserve(num_columns_to_add); - lazy_output.blocks.reserve(rows_to_add); - lazy_output.row_nums.reserve(rows_to_add); for (const auto & src_column : block_with_columns_to_add) { @@ -1095,9 +1086,162 @@ public: } } + virtual ~AddedColumns() { } + size_t size() const { return columns.size(); } - void buildOutput() + virtual void buildOutput() + { + } + + ColumnWithTypeAndName moveColumn(size_t i) + { + return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); + } + + virtual void appendFromBlock(const Block & block, size_t row_num, bool has_defaults) + { + if (has_defaults) + applyLazyDefaults(); + +#ifndef NDEBUG + for (size_t j = 0; j < right_indexes.size(); ++j) + { + const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); + const auto * dest_column = columns[j].get(); + if (auto * nullable_col = nullable_column_ptrs[j]) + { + if (!is_join_get) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Columns {} and {} can have different nullability only in joinGetOrNull", + dest_column->getName(), column_from_block->getName()); + dest_column = nullable_col->getNestedColumnPtr().get(); + } + /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, + * because dictionaries can be different, while calling insertFrom on them is safe, for example: + * ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1))) + * and + * ColumnLowCardinality(size = 0, UInt16(size = 0), ColumnUnique(size = 1, String(size = 1))) + */ + if (typeid(*dest_column) != typeid(*column_from_block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", + dest_column->getName(), column_from_block->getName(), + demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); + } +#endif + if (is_join_get) + { + size_t right_indexes_size = right_indexes.size(); + for (size_t j = 0; j < right_indexes_size; ++j) + { + const auto & column_from_block = block.getByPosition(right_indexes[j]); + if (auto * nullable_col = nullable_column_ptrs[j]) + nullable_col->insertFromNotNullable(*column_from_block.column, row_num); + else + columns[j]->insertFrom(*column_from_block.column, row_num); + } + } + else + { + size_t right_indexes_size = right_indexes.size(); + for (size_t j = 0; j < right_indexes_size; ++j) + { + const auto & column_from_block = block.getByPosition(right_indexes[j]); + columns[j]->insertFrom(*column_from_block.column, row_num); + } + } + } + + virtual void appendDefaultRow() + { + ++lazy_defaults_count; + } + + virtual void applyLazyDefaults() + { + if (lazy_defaults_count) + { + for (size_t j = 0, size = right_indexes.size(); j < size; ++j) + JoinCommon::addDefaultValues(*columns[j], type_name[j].type, lazy_defaults_count); + lazy_defaults_count = 0; + } + } + + const IColumn & leftAsofKey() const { return *left_asof_key; } + + std::vector join_on_keys; + + size_t max_joined_block_rows = 0; + size_t rows_to_add; + std::unique_ptr offsets_to_replicate; + bool need_filter = false; + IColumn::Filter filter; + + void reserve(bool need_replicate) + { + if (!max_joined_block_rows) + return; + + /// Do not allow big allocations when user set max_joined_block_rows to huge value + size_t reserve_size = std::min(max_joined_block_rows, DEFAULT_BLOCK_SIZE * 2); + + if (need_replicate) + /// Reserve 10% more space for columns, because some rows can be repeated + reserve_size = static_cast(1.1 * reserve_size); + + for (auto & column : columns) + column->reserve(reserve_size); + } + +protected: + MutableColumns columns; + bool is_join_get; + std::vector right_indexes; + std::vector type_name; + std::vector nullable_column_ptrs; +private: + + + + + size_t lazy_defaults_count = 0; + /// for ASOF + const IColumn * left_asof_key = nullptr; + + + void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) + { + columns.push_back(src_column.column->cloneEmpty()); + columns.back()->reserve(src_column.column->size()); + type_name.emplace_back(src_column.type, src_column.name, qualified_name); + } +}; + +class LazyAddedColumns : public AddedColumns +{ +public: + struct LazyOutput + { + PaddedPODArray blocks; + PaddedPODArray row_nums; + }; + + LazyAddedColumns( + const Block & left_block, + const Block & block_with_columns_to_add, + const Block & saved_block_sample, + const HashJoin & join, + std::vector && join_on_keys_, + bool is_asof_join, + bool is_join_get_) + : AddedColumns(left_block, block_with_columns_to_add, saved_block_sample, join, std::move(join_on_keys_), is_asof_join, is_join_get_) + { + has_columns_to_add = block_with_columns_to_add.columns() > 0; + lazy_output.blocks.reserve(rows_to_add); + lazy_output.row_nums.reserve(rows_to_add); + } + + virtual void buildOutput() override { for (size_t i = 0; i < this->size(); ++i) { @@ -1137,13 +1281,7 @@ public: } } - ColumnWithTypeAndName moveColumn(size_t i) - { - return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); - } - - - void appendFromBlock(const Block & block, size_t row_num) + virtual void appendFromBlock(const Block & block, size_t row_num, bool) override { #ifndef NDEBUG for (size_t j = 0; j < right_indexes.size(); ++j) @@ -1154,8 +1292,8 @@ public: { if (!is_join_get) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Columns {} and {} can have different nullability only in joinGetOrNull", - dest_column->getName(), column_from_block->getName()); + "Columns {} and {} can have different nullability only in joinGetOrNull", + dest_column->getName(), column_from_block->getName()); dest_column = nullable_col->getNestedColumnPtr().get(); } /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, @@ -1166,8 +1304,8 @@ public: */ if (typeid(*dest_column) != typeid(*column_from_block)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", - dest_column->getName(), column_from_block->getName(), - demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); + dest_column->getName(), column_from_block->getName(), + demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); } #endif if (has_columns_to_add) @@ -1177,7 +1315,7 @@ public: } } - void appendDefaultRow() + virtual void appendDefaultRow() override { if (has_columns_to_add) { @@ -1186,54 +1324,14 @@ public: } } - const IColumn & leftAsofKey() const { return *left_asof_key; } - - std::vector join_on_keys; + virtual void applyLazyDefaults() override { } +private : // The default row is represented by an empty RowRef, so that fixed-size blocks can be generated sequentially, // default_count cannot represent the position of the row LazyOutput lazy_output; - - size_t max_joined_block_rows = 0; - size_t rows_to_add; - std::unique_ptr offsets_to_replicate; - bool need_filter = false; - IColumn::Filter filter; - - void reserve(bool need_replicate) - { - if (!max_joined_block_rows) - return; - - /// Do not allow big allocations when user set max_joined_block_rows to huge value - size_t reserve_size = std::min(max_joined_block_rows, DEFAULT_BLOCK_SIZE * 2); - - if (need_replicate) - /// Reserve 10% more space for columns, because some rows can be repeated - reserve_size = static_cast(1.1 * reserve_size); - - for (auto & column : columns) - column->reserve(reserve_size); - } - -private: - std::vector type_name; - MutableColumns columns; - std::vector nullable_column_ptrs; - - std::vector right_indexes; bool has_columns_to_add; - /// for ASOF - const IColumn * left_asof_key = nullptr; - bool is_join_get; - - void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) - { - columns.push_back(src_column.column->cloneEmpty()); - columns.back()->reserve(src_column.column->size()); - type_name.emplace_back(src_column.type, src_column.name, qualified_name); - } }; template @@ -1333,7 +1431,7 @@ public: } }; -template +template void addFoundRowAll( const typename Map::mapped_type & mapped, AddedColumns & added, @@ -1341,6 +1439,9 @@ void addFoundRowAll( KnownRowsHolder & known_rows [[maybe_unused]], JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]]) { + if constexpr (add_missing) + added.applyLazyDefaults(); + if constexpr (multiple_disjuncts) { std::unique_ptr::Type>> new_known_rows_ptr; @@ -1349,7 +1450,7 @@ void addFoundRowAll( { if (!known_rows.isKnown(std::make_pair(it->block, it->row_num))) { - added.appendFromBlock(*it->block, it->row_num); + added.appendFromBlock(*it->block, it->row_num, false); ++current_offset; if (!new_known_rows_ptr) { @@ -1373,7 +1474,7 @@ void addFoundRowAll( { for (auto it = mapped.begin(); it.ok(); ++it) { - added.appendFromBlock(*it->block, it->row_num); + added.appendFromBlock(*it->block, it->row_num, false); ++current_offset; } } @@ -1462,7 +1563,7 @@ NO_INLINE size_t joinRightColumns( else used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*row_ref.block, row_ref.row_num); + added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing); } else addNotFoundRow(added_columns, current_offset); @@ -1472,7 +1573,7 @@ NO_INLINE size_t joinRightColumns( setUsed(added_columns.filter, i); used_flags.template setUsed(find_result); auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) { @@ -1482,7 +1583,7 @@ NO_INLINE size_t joinRightColumns( { auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; setUsed(added_columns.filter, i); - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } } else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) @@ -1493,7 +1594,7 @@ NO_INLINE size_t joinRightColumns( if (used_once) { setUsed(added_columns.filter, i); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); } break; @@ -1511,7 +1612,7 @@ NO_INLINE size_t joinRightColumns( { setUsed(added_columns.filter, i); used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); if (join_features.is_any_or_semi_join) { @@ -1701,14 +1802,19 @@ Block HashJoin::joinBlockImpl( * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. * For ASOF, the last column is used as the ASOF column */ - AddedColumns added_columns( - block, - block_with_columns_to_add, - savedBlockSample(), - *this, - std::move(join_on_keys), - join_features.is_asof_join, - is_join_get); + std::unique_ptr added_columns_ptr; + if (!join_features.is_any_join) + { + added_columns_ptr = std::make_unique( + block, block_with_columns_to_add, savedBlockSample(), *this, std::move(join_on_keys), join_features.is_asof_join, is_join_get); + } + else + { + added_columns_ptr = std::make_unique( + block, block_with_columns_to_add, savedBlockSample(), *this, std::move(join_on_keys), join_features.is_asof_join, is_join_get); + } + + AddedColumns & added_columns = * added_columns_ptr; bool has_required_right_keys = (required_right_keys.columns() != 0); added_columns.need_filter = join_features.need_filter || has_required_right_keys; From ce3f95f71760d63b946934363732c9b625c46ad8 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Thu, 22 Feb 2024 16:13:38 +0800 Subject: [PATCH 032/197] fix style --- src/Interpreters/HashJoin.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 1ea12955409..9a2a6157c15 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1200,10 +1200,6 @@ protected: std::vector type_name; std::vector nullable_column_ptrs; private: - - - - size_t lazy_defaults_count = 0; /// for ASOF const IColumn * left_asof_key = nullptr; From 22a33884c32cc403b3e4d6f6a48559e6cf45e613 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 23 Feb 2024 15:49:36 +0800 Subject: [PATCH 033/197] fix bug --- src/Interpreters/HashJoin.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9a2a6157c15..121181e8475 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1631,6 +1631,7 @@ NO_INLINE size_t joinRightColumns( } } + added_columns.applyLazyDefaults(); return i; } From 2279885c3e039f2aa5278cafb1f3e3a015289366 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 23 Feb 2024 16:59:04 +0800 Subject: [PATCH 034/197] fix clangtidy error --- src/Interpreters/HashJoin.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 121181e8475..ca7589f1b07 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1086,13 +1086,11 @@ public: } } - virtual ~AddedColumns() { } + virtual ~AddedColumns() = default; size_t size() const { return columns.size(); } - virtual void buildOutput() - { - } + virtual void buildOutput() { } ColumnWithTypeAndName moveColumn(size_t i) { @@ -1237,7 +1235,7 @@ public: lazy_output.row_nums.reserve(rows_to_add); } - virtual void buildOutput() override + void buildOutput() override { for (size_t i = 0; i < this->size(); ++i) { @@ -1277,7 +1275,7 @@ public: } } - virtual void appendFromBlock(const Block & block, size_t row_num, bool) override + void appendFromBlock(const Block & block, size_t row_num, bool) override { #ifndef NDEBUG for (size_t j = 0; j < right_indexes.size(); ++j) @@ -1311,7 +1309,7 @@ public: } } - virtual void appendDefaultRow() override + void appendDefaultRow() override { if (has_columns_to_add) { @@ -1320,7 +1318,7 @@ public: } } - virtual void applyLazyDefaults() override { } + void applyLazyDefaults() override { } private : // The default row is represented by an empty RowRef, so that fixed-size blocks can be generated sequentially, From 29c44762167707e0763306fc5cc60c15d1f98e49 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Feb 2024 12:09:07 +0800 Subject: [PATCH 035/197] Ping CI From 94f78ac44b3d36d8cedaf51dc62b9182cd8c8ebf Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 26 Feb 2024 15:34:56 +0800 Subject: [PATCH 036/197] use template class --- src/Interpreters/HashJoin.cpp | 330 ++++++++++++++++------------------ 1 file changed, 158 insertions(+), 172 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index ca7589f1b07..202ef51cea2 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1019,6 +1019,7 @@ struct JoinOnKeyColumns bool isRowFiltered(size_t i) const { return join_mask_column.isRowFiltered(i); } }; +template class AddedColumns { public: @@ -1034,6 +1035,12 @@ public: } }; + struct LazyOutput + { + PaddedPODArray blocks; + PaddedPODArray row_nums; + }; + AddedColumns( const Block & left_block, const Block & block_with_columns_to_add, @@ -1047,6 +1054,14 @@ public: , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); + + if constexpr (lazy) + { + has_columns_to_add = block_with_columns_to_add.columns() > 0; + lazy_output.blocks.reserve(rows_to_add); + lazy_output.row_nums.reserve(rows_to_add); + } + if (is_asof_join) ++num_columns_to_add; columns.reserve(num_columns_to_add); @@ -1086,76 +1101,21 @@ public: } } - virtual ~AddedColumns() = default; - size_t size() const { return columns.size(); } - virtual void buildOutput() { } + void buildOutput(); ColumnWithTypeAndName moveColumn(size_t i) { return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); } - virtual void appendFromBlock(const Block & block, size_t row_num, bool has_defaults) - { - if (has_defaults) - applyLazyDefaults(); + void appendFromBlock(const Block & block, size_t row_num, bool has_default); -#ifndef NDEBUG - for (size_t j = 0; j < right_indexes.size(); ++j) - { - const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); - const auto * dest_column = columns[j].get(); - if (auto * nullable_col = nullable_column_ptrs[j]) - { - if (!is_join_get) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Columns {} and {} can have different nullability only in joinGetOrNull", - dest_column->getName(), column_from_block->getName()); - dest_column = nullable_col->getNestedColumnPtr().get(); - } - /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, - * because dictionaries can be different, while calling insertFrom on them is safe, for example: - * ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1))) - * and - * ColumnLowCardinality(size = 0, UInt16(size = 0), ColumnUnique(size = 1, String(size = 1))) - */ - if (typeid(*dest_column) != typeid(*column_from_block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", - dest_column->getName(), column_from_block->getName(), - demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); - } -#endif - if (is_join_get) - { - size_t right_indexes_size = right_indexes.size(); - for (size_t j = 0; j < right_indexes_size; ++j) - { - const auto & column_from_block = block.getByPosition(right_indexes[j]); - if (auto * nullable_col = nullable_column_ptrs[j]) - nullable_col->insertFromNotNullable(*column_from_block.column, row_num); - else - columns[j]->insertFrom(*column_from_block.column, row_num); - } - } - else - { - size_t right_indexes_size = right_indexes.size(); - for (size_t j = 0; j < right_indexes_size; ++j) - { - const auto & column_from_block = block.getByPosition(right_indexes[j]); - columns[j]->insertFrom(*column_from_block.column, row_num); - } - } - } + void appendDefaultRow(); - virtual void appendDefaultRow() - { - ++lazy_defaults_count; - } - virtual void applyLazyDefaults() + void applyLazyDefaults() { if (lazy_defaults_count) { @@ -1197,6 +1157,12 @@ protected: std::vector right_indexes; std::vector type_name; std::vector nullable_column_ptrs; + + // The default row is represented by an empty RowRef, so that fixed-size blocks can be generated sequentially, + // default_count cannot represent the position of the row + LazyOutput lazy_output; + bool has_columns_to_add; + private: size_t lazy_defaults_count = 0; /// for ASOF @@ -1210,123 +1176,153 @@ private: type_name.emplace_back(src_column.type, src_column.name, qualified_name); } }; - -class LazyAddedColumns : public AddedColumns +template<> void AddedColumns::buildOutput() { -public: - struct LazyOutput +} +template<> +void AddedColumns::buildOutput() +{ + for (size_t i = 0; i < this->size(); ++i) { - PaddedPODArray blocks; - PaddedPODArray row_nums; - }; - - LazyAddedColumns( - const Block & left_block, - const Block & block_with_columns_to_add, - const Block & saved_block_sample, - const HashJoin & join, - std::vector && join_on_keys_, - bool is_asof_join, - bool is_join_get_) - : AddedColumns(left_block, block_with_columns_to_add, saved_block_sample, join, std::move(join_on_keys_), is_asof_join, is_join_get_) - { - has_columns_to_add = block_with_columns_to_add.columns() > 0; - lazy_output.blocks.reserve(rows_to_add); - lazy_output.row_nums.reserve(rows_to_add); - } - - void buildOutput() override - { - for (size_t i = 0; i < this->size(); ++i) + auto& col = columns[i]; + size_t default_count = 0; + auto apply_default = [&]() { - auto& col = columns[i]; - size_t default_count = 0; - auto apply_default = [&]() + if (default_count > 0) { - if (default_count > 0) - { - JoinCommon::addDefaultValues(*col, type_name[i].type, default_count); - default_count = 0; - } - }; + JoinCommon::addDefaultValues(*col, type_name[i].type, default_count); + default_count = 0; + } + }; - for (size_t j = 0; j < lazy_output.blocks.size(); ++j) + for (size_t j = 0; j < lazy_output.blocks.size(); ++j) + { + if (!lazy_output.blocks[j]) { - if (!lazy_output.blocks[j]) - { - default_count ++; - continue; - } - apply_default(); - const auto & column_from_block = reinterpret_cast(lazy_output.blocks[j])->getByPosition(right_indexes[i]); - /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. - if (is_join_get) - { - if (auto * nullable_col = typeid_cast(col.get()); - nullable_col && !column_from_block.column->isNullable()) - { - nullable_col->insertFromNotNullable(*column_from_block.column, lazy_output.row_nums[j]); - continue; - } - } - col->insertFrom(*column_from_block.column, lazy_output.row_nums[j]); + default_count ++; + continue; } apply_default(); - } - } - - void appendFromBlock(const Block & block, size_t row_num, bool) override - { -#ifndef NDEBUG - for (size_t j = 0; j < right_indexes.size(); ++j) - { - const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); - const auto * dest_column = columns[j].get(); - if (auto * nullable_col = nullable_column_ptrs[j]) + const auto & column_from_block = reinterpret_cast(lazy_output.blocks[j])->getByPosition(right_indexes[i]); + /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. + if (is_join_get) { - if (!is_join_get) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Columns {} and {} can have different nullability only in joinGetOrNull", - dest_column->getName(), column_from_block->getName()); - dest_column = nullable_col->getNestedColumnPtr().get(); + if (auto * nullable_col = typeid_cast(col.get()); + nullable_col && !column_from_block.column->isNullable()) + { + nullable_col->insertFromNotNullable(*column_from_block.column, lazy_output.row_nums[j]); + continue; + } } - /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, + col->insertFrom(*column_from_block.column, lazy_output.row_nums[j]); + } + apply_default(); + } +} + +template <> +void AddedColumns::appendFromBlock(const Block & block, size_t row_num,const bool has_defaults) +{ + if (has_defaults) + applyLazyDefaults(); + +#ifndef NDEBUG + for (size_t j = 0; j < right_indexes.size(); ++j) + { + const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); + const auto * dest_column = columns[j].get(); + if (auto * nullable_col = nullable_column_ptrs[j]) + { + if (!is_join_get) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Columns {} and {} can have different nullability only in joinGetOrNull", + dest_column->getName(), column_from_block->getName()); + dest_column = nullable_col->getNestedColumnPtr().get(); + } + /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, * because dictionaries can be different, while calling insertFrom on them is safe, for example: * ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1))) * and * ColumnLowCardinality(size = 0, UInt16(size = 0), ColumnUnique(size = 1, String(size = 1))) */ - if (typeid(*dest_column) != typeid(*column_from_block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", - dest_column->getName(), column_from_block->getName(), - demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); - } + if (typeid(*dest_column) != typeid(*column_from_block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", + dest_column->getName(), column_from_block->getName(), + demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); + } #endif - if (has_columns_to_add) - { - lazy_output.blocks.emplace_back(reinterpret_cast(&block)); - lazy_output.row_nums.emplace_back(static_cast(row_num)); - } - } - - void appendDefaultRow() override + if (is_join_get) { - if (has_columns_to_add) + size_t right_indexes_size = right_indexes.size(); + for (size_t j = 0; j < right_indexes_size; ++j) { - lazy_output.blocks.emplace_back(0); - lazy_output.row_nums.emplace_back(0); + const auto & column_from_block = block.getByPosition(right_indexes[j]); + if (auto * nullable_col = nullable_column_ptrs[j]) + nullable_col->insertFromNotNullable(*column_from_block.column, row_num); + else + columns[j]->insertFrom(*column_from_block.column, row_num); } } + else + { + size_t right_indexes_size = right_indexes.size(); + for (size_t j = 0; j < right_indexes_size; ++j) + { + const auto & column_from_block = block.getByPosition(right_indexes[j]); + columns[j]->insertFrom(*column_from_block.column, row_num); + } + } +} - void applyLazyDefaults() override { } +template <> +void AddedColumns::appendFromBlock(const Block & block, size_t row_num, bool) +{ +#ifndef NDEBUG + for (size_t j = 0; j < right_indexes.size(); ++j) + { + const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); + const auto * dest_column = columns[j].get(); + if (auto * nullable_col = nullable_column_ptrs[j]) + { + if (!is_join_get) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Columns {} and {} can have different nullability only in joinGetOrNull", + dest_column->getName(), column_from_block->getName()); + dest_column = nullable_col->getNestedColumnPtr().get(); + } + /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, + * because dictionaries can be different, while calling insertFrom on them is safe, for example: + * ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1))) + * and + * ColumnLowCardinality(size = 0, UInt16(size = 0), ColumnUnique(size = 1, String(size = 1))) + */ + if (typeid(*dest_column) != typeid(*column_from_block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", + dest_column->getName(), column_from_block->getName(), + demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); + } +#endif + if (has_columns_to_add) + { + lazy_output.blocks.emplace_back(reinterpret_cast(&block)); + lazy_output.row_nums.emplace_back(static_cast(row_num)); + } +} +template<> +void AddedColumns::appendDefaultRow() +{ + ++lazy_defaults_count; +} -private : - // The default row is represented by an empty RowRef, so that fixed-size blocks can be generated sequentially, - // default_count cannot represent the position of the row - LazyOutput lazy_output; - bool has_columns_to_add; - -}; +template<> +void AddedColumns::appendDefaultRow() +{ + if (has_columns_to_add) + { + lazy_output.blocks.emplace_back(0); + lazy_output.row_nums.emplace_back(0); + } +} template struct JoinFeatures @@ -1425,7 +1421,7 @@ public: } }; -template +template void addFoundRowAll( const typename Map::mapped_type & mapped, AddedColumns & added, @@ -1474,7 +1470,7 @@ void addFoundRowAll( } } -template +template void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & current_offset [[maybe_unused]]) { if constexpr (add_missing) @@ -1494,7 +1490,7 @@ void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unuse /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). -template +template NO_INLINE size_t joinRightColumns( std::vector && key_getter_vector, const std::vector & mapv, @@ -1633,7 +1629,7 @@ NO_INLINE size_t joinRightColumns( return i; } -template +template size_t joinRightColumnsSwitchMultipleDisjuncts( std::vector && key_getter_vector, const std::vector & mapv, @@ -1645,7 +1641,7 @@ size_t joinRightColumnsSwitchMultipleDisjuncts( : joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, used_flags); } -template +template size_t joinRightColumnsSwitchNullability( std::vector && key_getter_vector, const std::vector & mapv, @@ -1662,7 +1658,7 @@ size_t joinRightColumnsSwitchNullability( } } -template +template size_t switchJoinRightColumns( const std::vector & mapv, AddedColumns & added_columns, @@ -1797,19 +1793,9 @@ Block HashJoin::joinBlockImpl( * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. * For ASOF, the last column is used as the ASOF column */ - std::unique_ptr added_columns_ptr; - if (!join_features.is_any_join) - { - added_columns_ptr = std::make_unique( - block, block_with_columns_to_add, savedBlockSample(), *this, std::move(join_on_keys), join_features.is_asof_join, is_join_get); - } - else - { - added_columns_ptr = std::make_unique( - block, block_with_columns_to_add, savedBlockSample(), *this, std::move(join_on_keys), join_features.is_asof_join, is_join_get); - } + AddedColumns added_columns( + block, block_with_columns_to_add, savedBlockSample(), *this, std::move(join_on_keys), join_features.is_asof_join, is_join_get); - AddedColumns & added_columns = * added_columns_ptr; bool has_required_right_keys = (required_right_keys.columns() != 0); added_columns.need_filter = join_features.need_filter || has_required_right_keys; From a0f108ec3fd6f55d741c7aa6c162d0199e3385b0 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 26 Feb 2024 16:06:18 +0800 Subject: [PATCH 037/197] fix bug --- src/Interpreters/HashJoin.cpp | 39 +++++++++++++++++++++-------------- 1 file changed, 24 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 202ef51cea2..83162e16151 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1054,16 +1054,16 @@ public: , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); + if (is_asof_join) + ++num_columns_to_add; if constexpr (lazy) { - has_columns_to_add = block_with_columns_to_add.columns() > 0; + has_columns_to_add = num_columns_to_add > 0; lazy_output.blocks.reserve(rows_to_add); lazy_output.row_nums.reserve(rows_to_add); } - if (is_asof_join) - ++num_columns_to_add; columns.reserve(num_columns_to_add); type_name.reserve(num_columns_to_add); right_indexes.reserve(num_columns_to_add); @@ -1114,16 +1114,7 @@ public: void appendDefaultRow(); - - void applyLazyDefaults() - { - if (lazy_defaults_count) - { - for (size_t j = 0, size = right_indexes.size(); j < size; ++j) - JoinCommon::addDefaultValues(*columns[j], type_name[j].type, lazy_defaults_count); - lazy_defaults_count = 0; - } - } + void applyLazyDefaults(); const IColumn & leftAsofKey() const { return *left_asof_key; } @@ -1220,6 +1211,22 @@ void AddedColumns::buildOutput() } } +template<> +void AddedColumns::applyLazyDefaults() +{ + if (lazy_defaults_count) + { + for (size_t j = 0, size = right_indexes.size(); j < size; ++j) + JoinCommon::addDefaultValues(*columns[j], type_name[j].type, lazy_defaults_count); + lazy_defaults_count = 0; + } +} + +template<> +void AddedColumns::applyLazyDefaults() +{ +} + template <> void AddedColumns::appendFromBlock(const Block & block, size_t row_num,const bool has_defaults) { @@ -1309,13 +1316,13 @@ void AddedColumns::appendFromBlock(const Block & block, size_t row_num, bo } } template<> -void AddedColumns::appendDefaultRow() +void AddedColumns::appendDefaultRow() { ++lazy_defaults_count; } template<> -void AddedColumns::appendDefaultRow() +void AddedColumns::appendDefaultRow() { if (has_columns_to_add) { @@ -1324,6 +1331,8 @@ void AddedColumns::appendDefaultRow() } } + + template struct JoinFeatures { From 757b2f3369dc83192d0c2ef655882fc067832732 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 26 Feb 2024 16:49:15 +0800 Subject: [PATCH 038/197] fix style --- src/Interpreters/HashJoin.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 83162e16151..bdc7162bbb3 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1170,6 +1170,7 @@ private: template<> void AddedColumns::buildOutput() { } + template<> void AddedColumns::buildOutput() { @@ -1331,8 +1332,6 @@ void AddedColumns::appendDefaultRow() } } - - template struct JoinFeatures { From 4b858f167bf838b71078174f33dfc909133188f0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 27 Feb 2024 05:52:07 +0000 Subject: [PATCH 039/197] consistent mask ids based on node hash, respect format_display_secrets_in_show_and_select setting --- src/Analyzer/ConstantNode.cpp | 9 +++++++-- src/Analyzer/ConstantNode.h | 6 ++++++ src/Analyzer/IQueryTreeNode.cpp | 2 -- src/Analyzer/IQueryTreeNode.h | 6 ------ src/Analyzer/Passes/QueryAnalysisPass.cpp | 21 ++++++++++++++++----- 5 files changed, 29 insertions(+), 15 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 837c05ff5cb..f80e18f6327 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -45,10 +45,15 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state if (hasAlias()) buffer << ", alias: " << getAlias(); - buffer << ", constant_value: " << (is_masked ? "[HIDDEN]" : constant_value->getValue().dump()); + buffer << ", constant_value: "; + if (mask_id) + buffer << "[HIDDEN id: " << mask_id << "]"; + else + buffer << constant_value->getValue().dump(); + buffer << ", constant_value_type: " << constant_value->getType()->getName(); - if (!is_masked && getSourceExpression()) + if (!mask_id && getSourceExpression()) { buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION" << '\n'; getSourceExpression()->dumpTreeImpl(buffer, format_state, indent + 4); diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 51c98a4a3b3..dd2ccee23cd 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -75,6 +75,11 @@ public: return constant_value->getType(); } + void setMaskId(size_t id) + { + mask_id = id; + } + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; protected: @@ -90,6 +95,7 @@ private: ConstantValuePtr constant_value; String value_string; QueryTreeNodePtr source_expression; + size_t mask_id = 0; static constexpr size_t children_size = 0; }; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 3ef323c9648..d61cb0ffab1 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -45,14 +45,12 @@ const char * toString(QueryTreeNodeType type) } IQueryTreeNode::IQueryTreeNode(size_t children_size, size_t weak_pointers_size) - : is_masked(false) { children.resize(children_size); weak_pointers.resize(weak_pointers_size); } IQueryTreeNode::IQueryTreeNode(size_t children_size) - : is_masked(false) { children.resize(children_size); } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index c08a8860749..b07aa2d31b0 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -251,11 +251,6 @@ public: return children; } - void setMasked(bool masked = true) - { - is_masked = masked; - } - protected: /** Construct query tree node. * Resize children to children size. @@ -286,7 +281,6 @@ protected: QueryTreeNodes children; QueryTreeWeakNodes weak_pointers; - bool is_masked; private: String alias; diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 31f07b4ec77..c906fc96976 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -704,7 +704,10 @@ struct IdentifierResolveScope { subquery_depth = parent_scope->subquery_depth; context = parent_scope->context; + projection_mask_map = parent_scope->projection_mask_map; } + else + projection_mask_map = std::make_shared>(); if (auto * union_node = scope_node->as()) { @@ -782,6 +785,9 @@ struct IdentifierResolveScope */ QueryTreeNodePtr expression_join_tree_node; + /// Node hash to mask id map + std::shared_ptr> projection_mask_map; + [[maybe_unused]] const IdentifierResolveScope * getNearestQueryScope() const { const IdentifierResolveScope * scope_to_check = this; @@ -5120,9 +5126,6 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi allow_table_expressions /*allow_table_expression*/, {secret_arguments.start, secret_arguments.count}); - for (size_t n = secret_arguments.start; n < secret_arguments.start + secret_arguments.count; ++n) - arguments_projection_names[n] = "[HIDDEN]"; - auto & function_node = *function_node_ptr; /// Replace right IN function argument if it is table or table function with subquery that read ordinary columns @@ -6122,8 +6125,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node { auto node_to_resolve = node; auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression); - if (n >= secrets.first && n < secrets.first + secrets.second) - node_to_resolve->setMasked(); size_t expected_projection_names_size = 1; if (auto * expression_list = node_to_resolve->as()) @@ -6134,6 +6135,16 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node } else { + if (n >= secrets.first && n < secrets.first + secrets.second && !scope.context->getSettingsRef().format_display_secrets_in_show_and_select) + { + if (auto * constant = node_to_resolve->as()) + { + auto [mask, _] = scope.projection_mask_map->insert( {node->getTreeHash(), scope.projection_mask_map->size() + 1} ); + + constant->setMaskId(mask->second); + expression_node_projection_names[0] = "[HIDDEN id: " + std::to_string(mask->second) + "]"; + } + } result_nodes.push_back(std::move(node_to_resolve)); } From c1df83a27fad5503a8f6024c353162a470dfaa48 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 27 Feb 2024 10:31:14 +0800 Subject: [PATCH 040/197] refactor code --- src/Interpreters/HashJoin.cpp | 82 ++++++++++++++--------------------- 1 file changed, 33 insertions(+), 49 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index bdc7162bbb3..2b6a5f27a1f 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1142,20 +1142,48 @@ public: column->reserve(reserve_size); } -protected: +private: + + void checkBlock(const Block & block) + { + for (size_t j = 0; j < right_indexes.size(); ++j) + { + const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); + const auto * dest_column = columns[j].get(); + if (auto * nullable_col = nullable_column_ptrs[j]) + { + if (!is_join_get) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Columns {} and {} can have different nullability only in joinGetOrNull", + dest_column->getName(), column_from_block->getName()); + dest_column = nullable_col->getNestedColumnPtr().get(); + } + /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, + * because dictionaries can be different, while calling insertFrom on them is safe, for example: + * ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1))) + * and + * ColumnLowCardinality(size = 0, UInt16(size = 0), ColumnUnique(size = 1, String(size = 1))) + */ + if (typeid(*dest_column) != typeid(*column_from_block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", + dest_column->getName(), column_from_block->getName(), + demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); + } + } + MutableColumns columns; bool is_join_get; std::vector right_indexes; std::vector type_name; std::vector nullable_column_ptrs; + size_t lazy_defaults_count = 0; + /// for lazy // The default row is represented by an empty RowRef, so that fixed-size blocks can be generated sequentially, // default_count cannot represent the position of the row LazyOutput lazy_output; bool has_columns_to_add; -private: - size_t lazy_defaults_count = 0; /// for ASOF const IColumn * left_asof_key = nullptr; @@ -1235,29 +1263,7 @@ void AddedColumns::appendFromBlock(const Block & block, size_t row_num,co applyLazyDefaults(); #ifndef NDEBUG - for (size_t j = 0; j < right_indexes.size(); ++j) - { - const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); - const auto * dest_column = columns[j].get(); - if (auto * nullable_col = nullable_column_ptrs[j]) - { - if (!is_join_get) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Columns {} and {} can have different nullability only in joinGetOrNull", - dest_column->getName(), column_from_block->getName()); - dest_column = nullable_col->getNestedColumnPtr().get(); - } - /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, - * because dictionaries can be different, while calling insertFrom on them is safe, for example: - * ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1))) - * and - * ColumnLowCardinality(size = 0, UInt16(size = 0), ColumnUnique(size = 1, String(size = 1))) - */ - if (typeid(*dest_column) != typeid(*column_from_block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", - dest_column->getName(), column_from_block->getName(), - demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); - } + checkBlock(block); #endif if (is_join_get) { @@ -1286,29 +1292,7 @@ template <> void AddedColumns::appendFromBlock(const Block & block, size_t row_num, bool) { #ifndef NDEBUG - for (size_t j = 0; j < right_indexes.size(); ++j) - { - const auto * column_from_block = block.getByPosition(right_indexes[j]).column.get(); - const auto * dest_column = columns[j].get(); - if (auto * nullable_col = nullable_column_ptrs[j]) - { - if (!is_join_get) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Columns {} and {} can have different nullability only in joinGetOrNull", - dest_column->getName(), column_from_block->getName()); - dest_column = nullable_col->getNestedColumnPtr().get(); - } - /** Using dest_column->structureEquals(*column_from_block) will not work for low cardinality columns, - * because dictionaries can be different, while calling insertFrom on them is safe, for example: - * ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1))) - * and - * ColumnLowCardinality(size = 0, UInt16(size = 0), ColumnUnique(size = 1, String(size = 1))) - */ - if (typeid(*dest_column) != typeid(*column_from_block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} have different types {} and {}", - dest_column->getName(), column_from_block->getName(), - demangle(typeid(*dest_column).name()), demangle(typeid(*column_from_block).name())); - } + checkBlock(block); #endif if (has_columns_to_add) { From 0c7d12c392a8b52328368d048eb5bb61376ba3ac Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 27 Feb 2024 14:56:18 +0800 Subject: [PATCH 041/197] supress errors --- src/Processors/Transforms/FilterTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 4591177850b..0f2509c7510 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -340,6 +340,7 @@ void FilterTransform::doTransform(Chunk & chunk) break; } } + (void)min_size_in_memory; /// Suppress error of clang-analyzer-deadcode.DeadStores size_t num_filtered_rows = 0; if (first_non_constant_column != num_columns) From 4af339555d2d0ac02f275fc886a8c0ad85d9c753 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 27 Feb 2024 16:19:34 +0800 Subject: [PATCH 042/197] revert test case changes --- tests/performance/any_join.xml | 23 ------------------- .../performance/storage_join_direct_join.xml | 14 ----------- 2 files changed, 37 deletions(-) delete mode 100644 tests/performance/any_join.xml diff --git a/tests/performance/any_join.xml b/tests/performance/any_join.xml deleted file mode 100644 index ed473c36d55..00000000000 --- a/tests/performance/any_join.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - 1 - - - CREATE TABLE keys (key UInt64) ENGINE = MergeTree ORDER BY key; - CREATE TABLE dict (key UInt64, value1 UInt64, value2 Float64, value3 String, - value4 String, value5 String, value6 String, value7 String, value8 String, value9 String, - value10 String) ENGINE = MergeTree ORDER BY key; - - INSERT INTO keys SELECT rand() %500000 FROM numbers(10000000); - INSERT INTO dict SELECT rand() %300000, rand()%1000, rand()*0.0001, toString(number), - toString(number), toString(number), toString(number), toString(number), toString(number), - toString(number), toString(number) FROM numbers(1000000); - - - - SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; - SELECT keys.key, value1 FROM keys ALL INNER JOIN dict AS d ON (keys.key = d.key) FORMAT Null; - - DROP TABLE IF EXISTS keys - DROP TABLE IF EXISTS dict - \ No newline at end of file diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index d3bc25cd96b..2fc63c2c926 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -8,26 +8,12 @@ value4 String, value5 String, value6 String, value7 String, value8 String, value9 String, value10 String) ENGINE = Join(ANY, LEFT, key); - CREATE TABLE dict2 (key UInt64, value1 UInt64, value2 Float64, value3 String, - value4 String, value5 String, value6 String, value7 String, value8 String, value9 String, - value10 String) ENGINE = MergeTree ORDER BY key; - INSERT INTO keys SELECT rand() FROM numbers(10000000); INSERT INTO dict SELECT rand(), rand()%1000, rand()*0.0001, toString(number), toString(number), toString(number), toString(number), toString(number), toString(number), toString(number), toString(number) FROM numbers(1000000); - INSERT INTO dict2 SELECT rand(), rand()%1000, rand()*0.0001, toString(number), - toString(number), toString(number), toString(number), toString(number), toString(number), - toString(number), toString(number) FROM numbers(1000000); - SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; - SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict2 AS d ON (keys.key = d.key) FORMAT Null; - SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS allow_experimental_analyzer=1 - - DROP TABLE IF EXISTS keys - DROP TABLE IF EXISTS dict - DROP TABLE IF EXISTS dict2 \ No newline at end of file From 5ed09dc76b01e08cdd758285ebd984e327d8b7c2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 27 Feb 2024 14:58:17 +0000 Subject: [PATCH 043/197] refactoring, add test --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 40 ++++++----- ...8_analyzer_secret_args_tree_node.reference | 71 +++++++++++++++++++ .../02998_analyzer_secret_args_tree_node.sql | 9 +++ 3 files changed, 102 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference create mode 100644 tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c906fc96976..a479b4a1162 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1380,7 +1380,7 @@ private: ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const std::pair & secrets = std::pair()); + ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope); @@ -5118,13 +5118,30 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } /// Resolve function arguments - FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinder{function_node_ptr->toAST()->as()}.getResult(); bool allow_table_expressions = is_special_function_in; auto arguments_projection_names = resolveExpressionNodeList(function_node_ptr->getArgumentsNode(), scope, true /*allow_lambda_expression*/, - allow_table_expressions /*allow_table_expression*/, - {secret_arguments.start, secret_arguments.count}); + allow_table_expressions /*allow_table_expression*/); + + /// Mask arguments if needed + if (!scope.context->getSettingsRef().format_display_secrets_in_show_and_select) + { + if (FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinder{function_node_ptr->toAST()->as()}.getResult(); secret_arguments.count) + { + auto & argument_nodes = function_node_ptr->getArgumentsNode()->as().getNodes(); + + for (size_t n = secret_arguments.start; n < secret_arguments.start + secret_arguments.count; ++n) + { + if (auto * constant = argument_nodes[n]->as()) + { + auto [mask, _] = scope.projection_mask_map->insert( {constant->getTreeHash(), scope.projection_mask_map->size() + 1} ); + constant->setMaskId(mask->second); + arguments_projection_names[n] = "[HIDDEN id: " + std::to_string(mask->second) + "]"; + } + } + } + } auto & function_node = *function_node_ptr; @@ -6110,7 +6127,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id * Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table; * Example: SELECT *** FROM system.one; */ -ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const std::pair & secrets) +ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { auto & node_list_typed = node_list->as(); size_t node_list_size = node_list_typed.getNodes().size(); @@ -6120,7 +6137,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node ProjectionNames result_projection_names; - size_t n = 0; for (auto & node : node_list_typed.getNodes()) { auto node_to_resolve = node; @@ -6135,16 +6151,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node } else { - if (n >= secrets.first && n < secrets.first + secrets.second && !scope.context->getSettingsRef().format_display_secrets_in_show_and_select) - { - if (auto * constant = node_to_resolve->as()) - { - auto [mask, _] = scope.projection_mask_map->insert( {node->getTreeHash(), scope.projection_mask_map->size() + 1} ); - - constant->setMaskId(mask->second); - expression_node_projection_names[0] = "[HIDDEN id: " + std::to_string(mask->second) + "]"; - } - } result_nodes.push_back(std::move(node_to_resolve)); } @@ -6156,8 +6162,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node result_projection_names.insert(result_projection_names.end(), expression_node_projection_names.begin(), expression_node_projection_names.end()); expression_node_projection_names.clear(); - - ++n; } node_list_typed.getNodes() = std::move(result_nodes); diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference new file mode 100644 index 00000000000..ffd2f68990b --- /dev/null +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference @@ -0,0 +1,71 @@ +-- { echoOn } +SET allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT encrypt('aes-256-ofb', (SELECT 'qwerty'), '12345678901234567890123456789012'), encrypt('aes-256-ofb', (SELECT 'asdf'), '12345678901234567890123456789012'); +QUERY id: 0 + PROJECTION COLUMNS + encrypt(\'aes-256-ofb\', [HIDDEN id: 1], [HIDDEN id: 2]) Nullable(String) + encrypt(\'aes-256-ofb\', [HIDDEN id: 3], [HIDDEN id: 2]) Nullable(String) + PROJECTION + LIST id: 1, nodes: 2 + CONSTANT id: 2, constant_value: \'\\nãì&\', constant_value_type: Nullable(String) + EXPRESSION + FUNCTION id: 3, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) + ARGUMENTS + LIST id: 4, nodes: 3 + CONSTANT id: 5, constant_value: \'aes-256-ofb\', constant_value_type: String + CONSTANT id: 6, constant_value: [HIDDEN id: 1], constant_value_type: Nullable(String) + CONSTANT id: 7, constant_value: [HIDDEN id: 2], constant_value_type: String + CONSTANT id: 8, constant_value: \'çø\', constant_value_type: Nullable(String) + EXPRESSION + FUNCTION id: 9, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) + ARGUMENTS + LIST id: 10, nodes: 3 + CONSTANT id: 11, constant_value: \'aes-256-ofb\', constant_value_type: String + CONSTANT id: 12, constant_value: [HIDDEN id: 3], constant_value_type: Nullable(String) + CONSTANT id: 13, constant_value: [HIDDEN id: 2], constant_value_type: String + JOIN TREE + TABLE id: 14, alias: __table1, table_name: system.one +SET format_display_secrets_in_show_and_select = 1; +EXPLAIN QUERY TREE SELECT encrypt('aes-256-ofb', (SELECT 'qwerty'), '12345678901234567890123456789012'), encrypt('aes-256-ofb', (SELECT 'asdf'), '12345678901234567890123456789012'); +QUERY id: 0 + PROJECTION COLUMNS + encrypt(\'aes-256-ofb\', _subquery_1, \'12345678901234567890123456789012\') Nullable(String) + encrypt(\'aes-256-ofb\', _subquery_2, \'12345678901234567890123456789012\') Nullable(String) + PROJECTION + LIST id: 1, nodes: 2 + CONSTANT id: 2, constant_value: \'\\nãì&\', constant_value_type: Nullable(String) + EXPRESSION + FUNCTION id: 3, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) + ARGUMENTS + LIST id: 4, nodes: 3 + CONSTANT id: 5, constant_value: \'aes-256-ofb\', constant_value_type: String + CONSTANT id: 6, constant_value: \'qwerty\', constant_value_type: Nullable(String) + EXPRESSION + QUERY id: 7, is_subquery: 1 + PROJECTION COLUMNS + \'qwerty\' String + PROJECTION + LIST id: 8, nodes: 1 + CONSTANT id: 9, constant_value: \'qwerty\', constant_value_type: String + JOIN TREE + TABLE id: 10, table_name: system.one + CONSTANT id: 11, constant_value: \'12345678901234567890123456789012\', constant_value_type: String + CONSTANT id: 12, constant_value: \'çø\', constant_value_type: Nullable(String) + EXPRESSION + FUNCTION id: 13, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) + ARGUMENTS + LIST id: 14, nodes: 3 + CONSTANT id: 15, constant_value: \'aes-256-ofb\', constant_value_type: String + CONSTANT id: 16, constant_value: \'asdf\', constant_value_type: Nullable(String) + EXPRESSION + QUERY id: 17, is_subquery: 1 + PROJECTION COLUMNS + \'asdf\' String + PROJECTION + LIST id: 18, nodes: 1 + CONSTANT id: 19, constant_value: \'asdf\', constant_value_type: String + JOIN TREE + TABLE id: 20, table_name: system.one + CONSTANT id: 21, constant_value: \'12345678901234567890123456789012\', constant_value_type: String + JOIN TREE + TABLE id: 22, alias: __table1, table_name: system.one diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql new file mode 100644 index 00000000000..7ac718de98e --- /dev/null +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql @@ -0,0 +1,9 @@ +-- { echoOn } +SET allow_experimental_analyzer = 1; + +EXPLAIN QUERY TREE SELECT encrypt('aes-256-ofb', (SELECT 'qwerty'), '12345678901234567890123456789012'), encrypt('aes-256-ofb', (SELECT 'asdf'), '12345678901234567890123456789012'); + +SET format_display_secrets_in_show_and_select = 1; + +EXPLAIN QUERY TREE SELECT encrypt('aes-256-ofb', (SELECT 'qwerty'), '12345678901234567890123456789012'), encrypt('aes-256-ofb', (SELECT 'asdf'), '12345678901234567890123456789012'); +-- { echoOff } From 802db19e3b6a7d6d2828c64f8d9b4bfbc68db3f8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 27 Feb 2024 20:42:02 +0000 Subject: [PATCH 044/197] dedicated FunctionSecretArgumentsFinder for TreeNode, refactoring of FunctionSecretArgumentsFinder --- .../FunctionSecretArgumentsFinderTreeNode.h | 371 +++++++++++++ src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Parsers/ASTFunction.cpp | 4 +- src/Parsers/FunctionSecretArgumentsFinder.cpp | 19 + src/Parsers/FunctionSecretArgumentsFinder.h | 488 +---------------- .../FunctionSecretArgumentsFinderAST.h | 499 ++++++++++++++++++ 6 files changed, 895 insertions(+), 488 deletions(-) create mode 100644 src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h create mode 100644 src/Parsers/FunctionSecretArgumentsFinder.cpp create mode 100644 src/Parsers/FunctionSecretArgumentsFinderAST.h diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h new file mode 100644 index 00000000000..7e9a31868b2 --- /dev/null +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -0,0 +1,371 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + + +/// Finds arguments of a specified function which should not be displayed for most users for security reasons. +/// That involves passwords and secret keys. +class FunctionSecretArgumentsFinderTreeNode +{ +public: + explicit FunctionSecretArgumentsFinderTreeNode(const FunctionNode & function_) : function(function_), arguments(function.getArguments()) + { + if (arguments.getNodes().empty()) + return; + + findFunctionSecretArguments(); + } + + struct Result + { + /// Result constructed by default means no arguments will be hidden. + size_t start = static_cast(-1); + size_t count = 0; /// Mostly it's either 0 or 1. There are only a few cases where `count` can be greater than 1 (e.g. see `encrypt`). + /// In all known cases secret arguments are consecutive + bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. + /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` + std::vector nested_maps; + + bool hasSecrets() const + { + return count != 0 || !nested_maps.empty(); + } + }; + + FunctionSecretArgumentsFinder::Result getResult() const { return result; } + +private: + const FunctionNode & function; + const ListNode & arguments; + FunctionSecretArgumentsFinder::Result result; + + void markSecretArgument(size_t index, bool argument_is_named = false) + { + if (index >= arguments.getNodes().size()) + return; + if (!result.count) + { + result.start = index; + result.are_named = argument_is_named; + } + chassert(index >= result.start); /// We always check arguments consecutively + result.count = index + 1 - result.start; + if (!argument_is_named) + result.are_named = false; + } + + void findFunctionSecretArguments() + { + const auto & name = function.getFunctionName(); + + if ((name == "mysql") || (name == "postgresql") || (name == "mongodb")) + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) + /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((name == "s3") || (name == "cosn") || (name == "oss") || + (name == "deltaLake") || (name == "hudi") || (name == "iceberg")) + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (name == "s3Cluster") + { + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ true); + } + else if ((name == "remote") || (name == "remoteSecure")) + { + /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) + findRemoteFunctionSecretArguments(); + } + else if ((name == "encrypt") || (name == "decrypt") || + (name == "aes_encrypt_mysql") || (name == "aes_decrypt_mysql") || + (name == "tryDecrypt")) + { + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) + findEncryptionFunctionSecretArguments(); + } + else if (name == "url") + { + findURLSecretArguments(); + } + } + + void findMySQLFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// mysql(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + markSecretArgument(4); + } + } + + /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should + /// always be at the end). Marks "headers" as secret, if found. + size_t excludeS3OrURLNestedMaps() + { + const auto & nodes = arguments.getNodes(); + size_t count = nodes.size(); + while (count > 0) + { + const FunctionNode * f = nodes.at(count - 1)->as(); + if (!f) + break; + if (f->getFunctionName() == "headers") + result.nested_maps.push_back(f->getFunctionName()); + else if (f->getFunctionName() != "extra_credentials") + break; + count -= 1; + } + return count; + } + + void findS3FunctionSecretArguments(bool is_cluster_function) + { + /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (url_arg_idx + 2 < count) + markSecretArgument(url_arg_idx + 2); + } + + void findURLSecretArguments() + { + if (!isNamedCollectionName(0)) + excludeS3OrURLNestedMaps(); + } + + bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const + { + if (arg_idx >= arguments.getNodes().size()) + return false; + + return tryGetStringFromArgument(arguments.getNodes()[arg_idx], res, allow_identifier); + } + + static bool tryGetStringFromArgument(const QueryTreeNodePtr argument, String * res, bool allow_identifier = true) + { + if (const auto * literal = argument->as()) + { + if (literal->getValue().getType() != Field::Types::String) + return false; + if (res) + *res = literal->getValue().safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->getIdentifier().getFullName(); + return true; + } + } + + return false; + } + + void findRemoteFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// remote(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + return; + } + + /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: + /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// remote('addresses_expr', db.table) + if (arguments.getNodes().size() < 3) + return; + + size_t arg_num = 1; + + /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. + const auto * table_function = arguments.getNodes()[arg_num]->as(); + if (table_function && KnownTableFunctionNames::instance().exists(table_function->getFunctionName())) + { + ++arg_num; + } + else + { + std::optional database; + std::optional qualified_table_name; + if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `user`. + markSecretArgument(arg_num + 2); + } + if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `sharding_key`. + markSecretArgument(arg_num + 3); + } + return; + } + + /// Skip the current argument (which is either a database name or a qualified table name). + ++arg_num; + if (database) + { + /// Skip the 'table' argument if the previous argument was a database name. + ++arg_num; + } + } + + /// Skip username. + ++arg_num; + + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); + if (can_be_password) + markSecretArgument(arg_num); + } + + /// Tries to get either a database name or a qualified table name from an argument. + /// Empty string is also allowed (it means the default database). + /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. + bool tryGetDatabaseNameOrQualifiedTableName( + size_t arg_idx, + std::optional & res_database, + std::optional & res_qualified_table_name) const + { + res_database.reset(); + res_qualified_table_name.reset(); + + String str; + if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) + return false; + + if (str.empty()) + { + res_database = ""; + return true; + } + + auto qualified_table_name = QualifiedTableName::tryParseFromString(str); + if (!qualified_table_name) + return false; + + if (qualified_table_name->database.empty()) + res_database = std::move(qualified_table_name->table); + else + res_qualified_table_name = std::move(qualified_table_name); + return true; + } + + void findEncryptionFunctionSecretArguments() + { + if (arguments.getNodes().empty()) + return; + + /// We replace all arguments after 'mode' with '[HIDDEN]': + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') + result.start = 1; + result.count = arguments.getNodes().size() - 1; + } + + + /// Whether a specified argument can be the name of a named collection? + bool isNamedCollectionName(size_t arg_idx) const + { + if (arguments.getNodes().size() <= arg_idx) + return false; + + const auto * identifier = arguments.getNodes()[arg_idx]->as(); + return identifier != nullptr; + } + + /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + { + for (size_t i = start; i < arguments.getNodes().size(); ++i) + { + const auto & argument = arguments.getNodes()[i]; + const auto * equals_func = argument->as(); + if (!equals_func || (equals_func->getFunctionName() != "equals")) + continue; + + const auto * expr_list = equals_func->getArguments().as(); + if (!expr_list) + continue; + + const auto & equal_args = expr_list->getNodes(); + if (equal_args.size() != 2) + continue; + + String found_key; + if (!tryGetStringFromArgument(equal_args[0], &found_key)) + continue; + + if (found_key == key) + markSecretArgument(i, /* argument_is_named= */ true); + } + } +}; + +} diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index a479b4a1162..f21df6b5ccc 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5127,7 +5127,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi /// Mask arguments if needed if (!scope.context->getSettingsRef().format_display_secrets_in_show_and_select) { - if (FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinder{function_node_ptr->toAST()->as()}.getResult(); secret_arguments.count) + if (FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinder::find(*function_node_ptr); secret_arguments.count) { auto & argument_nodes = function_node_ptr->getArgumentsNode()->as().getNodes(); diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 7468c41910a..4dac9c090f6 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -693,7 +693,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format FunctionSecretArgumentsFinder::Result secret_arguments; if (!settings.show_secrets) - secret_arguments = FunctionSecretArgumentsFinder{*this}.getResult(); + secret_arguments = FunctionSecretArgumentsFinder::find(*this); for (size_t i = 0, size = arguments->children.size(); i < size; ++i) { @@ -757,7 +757,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format bool ASTFunction::hasSecretParts() const { - return (FunctionSecretArgumentsFinder{*this}.getResult().hasSecrets()) || childrenHaveSecretParts(); + return (FunctionSecretArgumentsFinder::find(*this).hasSecrets()) || childrenHaveSecretParts(); } String getFunctionName(const IAST * ast) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.cpp b/src/Parsers/FunctionSecretArgumentsFinder.cpp new file mode 100644 index 00000000000..bdeb29a37ba --- /dev/null +++ b/src/Parsers/FunctionSecretArgumentsFinder.cpp @@ -0,0 +1,19 @@ +#include +#include +#include + + +namespace DB +{ + +FunctionSecretArgumentsFinder::Result FunctionSecretArgumentsFinder::find(const ASTFunction & function) +{ + return FunctionSecretArgumentsFinderAST(function).getResult(); +} + +FunctionSecretArgumentsFinder::Result FunctionSecretArgumentsFinder::find(const FunctionNode & function) +{ + return FunctionSecretArgumentsFinderTreeNode(function).getResult(); +} + +} diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 355dd99a21a..950d913fe20 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -1,44 +1,14 @@ #pragma once -#include +#include #include -#include -#include -#include - -#include - namespace DB { - -/// Finds arguments of a specified function which should not be displayed for most users for security reasons. -/// That involves passwords and secret keys. class FunctionSecretArgumentsFinder { public: - explicit FunctionSecretArgumentsFinder(const ASTFunction & function_) : function(function_) - { - if (!function.arguments) - return; - - const auto * expr_list = function.arguments->as(); - if (!expr_list) - return; - - arguments = &expr_list->children; - switch (function.kind) - { - case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; - case ASTFunction::Kind::WINDOW_FUNCTION: break; - case ASTFunction::Kind::LAMBDA_FUNCTION: break; - case ASTFunction::Kind::TABLE_ENGINE: findTableEngineSecretArguments(); break; - case ASTFunction::Kind::DATABASE_ENGINE: findDatabaseEngineSecretArguments(); break; - case ASTFunction::Kind::BACKUP_NAME: findBackupNameSecretArguments(); break; - } - } - struct Result { /// Result constructed by default means no arguments will be hidden. @@ -55,460 +25,8 @@ public: } }; - Result getResult() const { return result; } - -private: - const ASTFunction & function; - const ASTs * arguments = nullptr; - Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments->size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findOrdinaryFunctionSecretArguments() - { - if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || - (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((function.name == "remote") || (function.name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((function.name == "encrypt") || (function.name == "decrypt") || - (function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") || - (function.name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (function.name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - size_t count = arguments->size(); - while (count > 0) - { - const ASTFunction * f = arguments->at(count - 1)->as(); - if (!f) - break; - if (f->name == "headers") - result.nested_maps.push_back(f->name); - else if (f->name != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments->size()) - return false; - - return tryGetStringFromArgument(*(*arguments)[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const IAST & argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument.as()) - { - if (literal->value.getType() != Field::Types::String) - return false; - if (res) - *res = literal->value.safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument.as()) - { - if (res) - *res = id->name(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments->size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = (*arguments)[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments->empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments->size() - 1; - } - - void findTableEngineSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "ExternalDistributed") - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - findExternalDistributedTableEngineSecretArguments(); - } - else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) - { - /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) - /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || - (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) - { - /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) - findS3TableEngineSecretArguments(); - } - else if (engine_name == "URL") - { - findURLSecretArguments(); - } - } - - void findExternalDistributedTableEngineSecretArguments() - { - if (isNamedCollectionName(1)) - { - /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 2); - } - else - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - markSecretArgument(5); - } - } - - void findS3TableEngineSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'secret_access_key') - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((3 <= count) && (count <= 4)) - { - String second_arg; - if (tryGetStringFromArgument(1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (count == 3) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: S3('url', 'format', ...) - } - } - } - - /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (2 < count) - markSecretArgument(2); - } - - void findDatabaseEngineSecretArguments() - { - const String & engine_name = function.name; - if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || - (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL")) - { - /// MySQL('host:port', 'database', 'user', 'password') - /// PostgreSQL('host:port', 'database', 'user', 'password') - findMySQLDatabaseSecretArguments(); - } - else if (engine_name == "S3") - { - /// S3('url', 'access_key_id', 'secret_access_key') - findS3DatabaseSecretArguments(); - } - } - - void findMySQLDatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// MySQL(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// MySQL('host:port', 'database', 'user', 'password') - markSecretArgument(3); - } - } - - void findS3DatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'password', ...) - findSecretNamedArgument("secret_access_key", 1); - } - else - { - /// S3('url', 'access_key_id', 'secret_access_key') - markSecretArgument(2); - } - } - - void findBackupNameSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "S3") - { - /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) - markSecretArgument(2); - } - } - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments->size() <= arg_idx) - return false; - - const auto * identifier = (*arguments)[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments->size(); ++i) - { - const auto & argument = (*arguments)[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->name != "equals")) - continue; - - const auto * expr_list = equals_func->arguments->as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->children; - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(*equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } + static Result find(const ASTFunction & function); + static Result find(const FunctionNode & function); }; } diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h new file mode 100644 index 00000000000..348b2ca9e3a --- /dev/null +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -0,0 +1,499 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + + +/// Finds arguments of a specified function which should not be displayed for most users for security reasons. +/// That involves passwords and secret keys. +class FunctionSecretArgumentsFinderAST +{ +public: + explicit FunctionSecretArgumentsFinderAST(const ASTFunction & function_) : function(function_) + { + if (!function.arguments) + return; + + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return; + + arguments = &expr_list->children; + switch (function.kind) + { + case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; + case ASTFunction::Kind::WINDOW_FUNCTION: break; + case ASTFunction::Kind::LAMBDA_FUNCTION: break; + case ASTFunction::Kind::TABLE_ENGINE: findTableEngineSecretArguments(); break; + case ASTFunction::Kind::DATABASE_ENGINE: findDatabaseEngineSecretArguments(); break; + case ASTFunction::Kind::BACKUP_NAME: findBackupNameSecretArguments(); break; + } + } + + FunctionSecretArgumentsFinder::Result getResult() const { return result; } + +private: + const ASTFunction & function; + const ASTs * arguments = nullptr; + FunctionSecretArgumentsFinder::Result result; + + void markSecretArgument(size_t index, bool argument_is_named = false) + { + if (index >= arguments->size()) + return; + if (!result.count) + { + result.start = index; + result.are_named = argument_is_named; + } + chassert(index >= result.start); /// We always check arguments consecutively + result.count = index + 1 - result.start; + if (!argument_is_named) + result.are_named = false; + } + + void findOrdinaryFunctionSecretArguments() + { + if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb")) + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) + /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || + (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function.name == "s3Cluster") + { + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ true); + } + else if ((function.name == "remote") || (function.name == "remoteSecure")) + { + /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) + findRemoteFunctionSecretArguments(); + } + else if ((function.name == "encrypt") || (function.name == "decrypt") || + (function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") || + (function.name == "tryDecrypt")) + { + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) + findEncryptionFunctionSecretArguments(); + } + else if (function.name == "url") + { + findURLSecretArguments(); + } + } + + void findMySQLFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// mysql(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + markSecretArgument(4); + } + } + + /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should + /// always be at the end). Marks "headers" as secret, if found. + size_t excludeS3OrURLNestedMaps() + { + size_t count = arguments->size(); + while (count > 0) + { + const ASTFunction * f = arguments->at(count - 1)->as(); + if (!f) + break; + if (f->name == "headers") + result.nested_maps.push_back(f->name); + else if (f->name != "extra_credentials") + break; + count -= 1; + } + return count; + } + + void findS3FunctionSecretArguments(bool is_cluster_function) + { + /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (url_arg_idx + 2 < count) + markSecretArgument(url_arg_idx + 2); + } + + void findURLSecretArguments() + { + if (!isNamedCollectionName(0)) + excludeS3OrURLNestedMaps(); + } + + bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const + { + if (arg_idx >= arguments->size()) + return false; + + return tryGetStringFromArgument(*(*arguments)[arg_idx], res, allow_identifier); + } + + static bool tryGetStringFromArgument(const IAST & argument, String * res, bool allow_identifier = true) + { + if (const auto * literal = argument.as()) + { + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument.as()) + { + if (res) + *res = id->name(); + return true; + } + } + + return false; + } + + void findRemoteFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// remote(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + return; + } + + /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: + /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// remote('addresses_expr', db.table) + if (arguments->size() < 3) + return; + + size_t arg_num = 1; + + /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. + const auto * table_function = (*arguments)[arg_num]->as(); + if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) + { + ++arg_num; + } + else + { + std::optional database; + std::optional qualified_table_name; + if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `user`. + markSecretArgument(arg_num + 2); + } + if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `sharding_key`. + markSecretArgument(arg_num + 3); + } + return; + } + + /// Skip the current argument (which is either a database name or a qualified table name). + ++arg_num; + if (database) + { + /// Skip the 'table' argument if the previous argument was a database name. + ++arg_num; + } + } + + /// Skip username. + ++arg_num; + + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); + if (can_be_password) + markSecretArgument(arg_num); + } + + /// Tries to get either a database name or a qualified table name from an argument. + /// Empty string is also allowed (it means the default database). + /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. + bool tryGetDatabaseNameOrQualifiedTableName( + size_t arg_idx, + std::optional & res_database, + std::optional & res_qualified_table_name) const + { + res_database.reset(); + res_qualified_table_name.reset(); + + String str; + if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) + return false; + + if (str.empty()) + { + res_database = ""; + return true; + } + + auto qualified_table_name = QualifiedTableName::tryParseFromString(str); + if (!qualified_table_name) + return false; + + if (qualified_table_name->database.empty()) + res_database = std::move(qualified_table_name->table); + else + res_qualified_table_name = std::move(qualified_table_name); + return true; + } + + void findEncryptionFunctionSecretArguments() + { + if (arguments->empty()) + return; + + /// We replace all arguments after 'mode' with '[HIDDEN]': + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') + result.start = 1; + result.count = arguments->size() - 1; + } + + void findTableEngineSecretArguments() + { + const String & engine_name = function.name; + if (engine_name == "ExternalDistributed") + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + findExternalDistributedTableEngineSecretArguments(); + } + else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) + { + /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) + /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || + (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) + { + /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) + findS3TableEngineSecretArguments(); + } + else if (engine_name == "URL") + { + findURLSecretArguments(); + } + } + + void findExternalDistributedTableEngineSecretArguments() + { + if (isNamedCollectionName(1)) + { + /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 2); + } + else + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + markSecretArgument(5); + } + } + + void findS3TableEngineSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'secret_access_key') + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((3 <= count) && (count <= 4)) + { + String second_arg; + if (tryGetStringFromArgument(1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (count == 3) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: S3('url', 'format', ...) + } + } + } + + /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (2 < count) + markSecretArgument(2); + } + + void findDatabaseEngineSecretArguments() + { + const String & engine_name = function.name; + if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || + (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL")) + { + /// MySQL('host:port', 'database', 'user', 'password') + /// PostgreSQL('host:port', 'database', 'user', 'password') + findMySQLDatabaseSecretArguments(); + } + else if (engine_name == "S3") + { + /// S3('url', 'access_key_id', 'secret_access_key') + findS3DatabaseSecretArguments(); + } + } + + void findMySQLDatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// MySQL(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// MySQL('host:port', 'database', 'user', 'password') + markSecretArgument(3); + } + } + + void findS3DatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'password', ...) + findSecretNamedArgument("secret_access_key", 1); + } + else + { + /// S3('url', 'access_key_id', 'secret_access_key') + markSecretArgument(2); + } + } + + void findBackupNameSecretArguments() + { + const String & engine_name = function.name; + if (engine_name == "S3") + { + /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) + markSecretArgument(2); + } + } + + /// Whether a specified argument can be the name of a named collection? + bool isNamedCollectionName(size_t arg_idx) const + { + if (arguments->size() <= arg_idx) + return false; + + const auto * identifier = (*arguments)[arg_idx]->as(); + return identifier != nullptr; + } + + /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + { + for (size_t i = start; i < arguments->size(); ++i) + { + const auto & argument = (*arguments)[i]; + const auto * equals_func = argument->as(); + if (!equals_func || (equals_func->name != "equals")) + continue; + + const auto * expr_list = equals_func->arguments->as(); + if (!expr_list) + continue; + + const auto & equal_args = expr_list->children; + if (equal_args.size() != 2) + continue; + + String found_key; + if (!tryGetStringFromArgument(*equal_args[0], &found_key)) + continue; + + if (found_key == key) + markSecretArgument(i, /* argument_is_named= */ true); + } + } +}; + +} From d4b71748fc4aa54bc233d37f14ad54225eaf3267 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 27 Feb 2024 22:38:41 +0000 Subject: [PATCH 045/197] fix style --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index f21df6b5ccc..5046a879fa5 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5135,9 +5135,9 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi { if (auto * constant = argument_nodes[n]->as()) { - auto [mask, _] = scope.projection_mask_map->insert( {constant->getTreeHash(), scope.projection_mask_map->size() + 1} ); - constant->setMaskId(mask->second); - arguments_projection_names[n] = "[HIDDEN id: " + std::to_string(mask->second) + "]"; + auto mask = scope.projection_mask_map->insert( {constant->getTreeHash(), scope.projection_mask_map->size() + 1} ).first->second; + constant->setMaskId(mask); + arguments_projection_names[n] = "[HIDDEN id: " + std::to_string(mask) + "]"; } } } From ad7715b2fa2f63183b6989bfedd2a741a4df30cd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 27 Feb 2024 23:12:29 +0000 Subject: [PATCH 046/197] fix style --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5046a879fa5..819a3758e26 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5135,7 +5135,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi { if (auto * constant = argument_nodes[n]->as()) { - auto mask = scope.projection_mask_map->insert( {constant->getTreeHash(), scope.projection_mask_map->size() + 1} ).first->second; + auto mask = scope.projection_mask_map->insert({constant->getTreeHash(), scope.projection_mask_map->size() + 1}).first->second; constant->setMaskId(mask); arguments_projection_names[n] = "[HIDDEN id: " + std::to_string(mask) + "]"; } From cca154b6b9a443d29bc3609a77c8b14d33a00317 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B1=AA=E8=82=A5=E8=82=A5?= Date: Wed, 28 Feb 2024 12:16:34 +0800 Subject: [PATCH 047/197] fix option ambiguous --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 867e7a69a6d..68f0e52ce08 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -841,7 +841,7 @@ void LocalServer::addOptions(OptionsDescription & options_description) /// If structure argument is omitted then initial query is not generated ("structure,S", po::value(), "structure of the initial table (list of column and type names)") - ("file,f", po::value(), "path to file with data of the initial table (stdin if not specified)") + ("file,F", po::value(), "path to file with data of the initial table (stdin if not specified)") ("input-format", po::value(), "input format of the initial table data") ("output-format", po::value(), "default output format") From 4f2ab335af2df8a6e5d4c45810231c31f1bc573c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Feb 2024 17:07:49 +0800 Subject: [PATCH 048/197] fix some bugs --- src/Functions/multiIf.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 0477e99dfcd..68febd54eaf 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -157,6 +157,10 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) const override { + /// Fast path when data is empty + if (input_rows_count == 0) + return result_type->createColumn(); + ColumnsWithTypeAndName arguments = args; executeShortCircuitArguments(arguments); /** We will gather values from columns in branches to result column, @@ -257,7 +261,7 @@ public: } const WhichDataType which(removeNullable(result_type)); - bool execute_multiif_columnar = settings.allow_execute_multiif_columnar && !contains_short + bool execute_multiif_columnar = allow_execute_multiif_columnar && !contains_short && instructions.size() <= std::numeric_limits::max() && (which.isInt() || which.isUInt() || which.isFloat() || which.isDecimal() || which.isDateOrDate32OrDateTimeOrDateTime64() || which.isEnum() || which.isIPv4() || which.isIPv6()); @@ -266,6 +270,7 @@ public: if (!execute_multiif_columnar) { MutableColumnPtr res = return_type->createColumn(); + res->reserve(rows); executeInstructions(instructions, rows, res); return std::move(res); } @@ -367,7 +372,7 @@ private: template static void calculateInserts(std::vector & instructions, size_t rows, PaddedPODArray & inserts) { - for (S i = static_cast(instructions.size() - 1); i >= 0; --i) + for (S i = instructions.size() - 1; i != static_cast(-1); --i) { auto & instruction = instructions[i]; if (instruction.condition_always_true) @@ -450,6 +455,7 @@ private: data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); } } + for (size_t row_i = 0; row_i < rows; ++row_i) { S insert = inserts[row_i]; From 4beb6fdcb037abce135b7d189b990737a824b2fe Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Feb 2024 22:39:49 +0800 Subject: [PATCH 049/197] fix code stule --- src/Functions/multiIf.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 68febd54eaf..b0a344be96e 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; } namespace @@ -370,7 +369,7 @@ private: /// We should read source from which instruction on each row? template - static void calculateInserts(std::vector & instructions, size_t rows, PaddedPODArray & inserts) + static NO_INLINE void calculateInserts(std::vector & instructions, size_t rows, PaddedPODArray & inserts) { for (S i = instructions.size() - 1; i != static_cast(-1); --i) { @@ -411,7 +410,7 @@ private: } template - static void executeInstructionsColumnar( + static NO_INLINE void executeInstructionsColumnar( std::vector & instructions, size_t rows, PaddedPODArray & res_data, From f86213ecb615aa0949b878c3ac58ec63560dd886 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 28 Feb 2024 16:29:24 +0100 Subject: [PATCH 050/197] Cancel PipelineExecutor properly in case of exception in spawnThreads --- src/Processors/Executors/PipelineExecutor.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index a06bacd7d3b..c3fbe6788c6 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -399,7 +399,18 @@ void PipelineExecutor::executeImpl(size_t num_threads, bool concurrency_control) if (num_threads > 1) { - spawnThreads(); // start at least one thread + try + { + spawnThreads(); // start at least one thread + } + catch (...) + { + /// spawnThreads can throw an exception, for example CANNOT_SCHEDULE_TASK. + /// We should cancel execution properly before rethrow. + cancel(); + throw; + } + tasks.processAsyncTasks(); pool->wait(); } From aef5818216fddaf9e2c13674e5e43632a3cc8cb8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 28 Feb 2024 15:45:33 +0000 Subject: [PATCH 051/197] fix test --- .../0_stateless/02998_analyzer_secret_args_tree_node.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql index 7ac718de98e..f1d916eca80 100644 --- a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest: encrypt function doesn't exist in the fastest build + -- { echoOn } SET allow_experimental_analyzer = 1; From 0fa22abb95c4848b785edab868439143f024b1eb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 28 Feb 2024 16:53:55 +0000 Subject: [PATCH 052/197] refactoring of virtual columns --- .../Transforms/buildPushingToViewsChain.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 35 +++++----- src/Storages/FileLog/StorageFileLog.h | 5 +- src/Storages/Hive/StorageHive.cpp | 11 ++-- src/Storages/Hive/StorageHive.h | 2 - src/Storages/IStorage.cpp | 7 ++ src/Storages/IStorage.h | 4 +- src/Storages/Kafka/KafkaSource.cpp | 2 +- src/Storages/Kafka/StorageKafka.cpp | 66 ++++++++----------- src/Storages/Kafka/StorageKafka.h | 5 +- src/Storages/LiveView/StorageLiveView.cpp | 12 ++-- src/Storages/LiveView/StorageLiveView.h | 2 - .../MergeTree/StorageFromMergeTreeDataPart.h | 6 +- src/Storages/NATS/StorageNATS.cpp | 32 ++++----- src/Storages/NATS/StorageNATS.h | 2 +- .../StorageMaterializedPostgreSQL.cpp | 19 +++--- .../StorageMaterializedPostgreSQL.h | 5 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 45 +++++++------ src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 +- src/Storages/StorageKeeperMap.cpp | 11 ++-- src/Storages/StorageKeeperMap.h | 2 - src/Storages/StorageMaterializedMySQL.cpp | 13 +--- src/Storages/StorageMaterializedMySQL.h | 1 - src/Storages/StorageMaterializedView.cpp | 7 +- src/Storages/StorageMaterializedView.h | 3 - src/Storages/StorageMerge.h | 3 + src/Storages/StorageProxy.h | 1 - src/Storages/StorageValues.cpp | 5 +- src/Storages/StorageValues.h | 16 ++--- .../System/StorageSystemDictionaries.cpp | 17 +++-- .../System/StorageSystemDictionaries.h | 5 +- .../System/StorageSystemPartsBase.cpp | 11 ++-- src/Storages/System/StorageSystemPartsBase.h | 2 - 33 files changed, 170 insertions(+), 193 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index c7a03c0bc6d..c4b707a0ce0 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -574,7 +574,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat views_data.source_storage_id, views_data.source_metadata_snapshot->getColumns(), std::move(block), - views_data.source_storage->getVirtuals())); + *views_data.source_storage->getVirtualsDescription())); QueryPipelineBuilder pipeline; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index b0bac081d38..23e869bfa89 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -27,6 +27,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include @@ -148,6 +149,9 @@ StorageFileLog::StorageFileLog( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + auto virtuals = createVirtuals(filelog_settings->handle_error_mode); + setVirtuals(virtuals); + if (!fileOrSymlinkPathStartsWith(path, getContext()->getUserFilesPath())) { if (LoadingStrictnessLevel::ATTACH <= mode) @@ -203,6 +207,22 @@ StorageFileLog::StorageFileLog( } } +VirtualColumnsDescription StorageFileLog::createVirtuals(StreamingHandleErrorMode handle_error_mode) +{ + VirtualColumnsDescription desc; + + desc.addEphemeral("_filename", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_offset", std::make_shared(), ""); + + if (handle_error_mode == StreamingHandleErrorMode::STREAM) + { + desc.addEphemeral("_raw_record", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_error", std::make_shared(std::make_shared()), ""); + } + + return desc; +} + void StorageFileLog::loadMetaFiles(bool attach) { /// Attach table @@ -1009,19 +1029,4 @@ bool StorageFileLog::updateFileInfos() return events.empty() || file_infos.file_names.empty(); } -NamesAndTypesList StorageFileLog::getVirtuals() const -{ - auto virtuals = NamesAndTypesList{ - {"_filename", std::make_shared(std::make_shared())}, - {"_offset", std::make_shared()}}; - - if (filelog_settings->handle_error_mode == StreamingHandleErrorMode::STREAM) - { - virtuals.push_back({"_raw_record", std::make_shared(std::make_shared())}); - virtuals.push_back({"_error", std::make_shared(std::make_shared())}); - } - - return virtuals; -} - } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index bdcf85ad6e5..ede36600f92 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -9,6 +9,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #include @@ -102,8 +103,6 @@ public: String getFullMetaPath(const String & file_name) const { return std::filesystem::path(metadata_base_path) / file_name; } String getFullDataPath(const String & file_name) const { return std::filesystem::path(root_data_path) / file_name; } - NamesAndTypesList getVirtuals() const override; - static UInt64 getInode(const String & file_name); void openFilesAndSetPos(); @@ -212,6 +211,8 @@ private: UInt64 inode = 0; }; ReadMetadataResult readMetadata(const String & filename) const; + + static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); }; } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 183a4532281..f8c10ea4249 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -45,6 +45,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -444,6 +445,9 @@ StorageHive::StorageHive( storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext()); setInMemoryMetadata(storage_metadata); + + auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); + setVirtuals(virtuals); } void StorageHive::lazyInitialize() @@ -1020,13 +1024,6 @@ SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetad throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not implemented for StorageHive"); } -NamesAndTypesList StorageHive::getVirtuals() const -{ - return NamesAndTypesList{ - {"_path", std::make_shared(std::make_shared())}, - {"_file", std::make_shared(std::make_shared())}}; -} - std::optional StorageHive::totalRows(const Settings & settings) const { /// query_info is not used when prune_level == PruneLevel::None diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 07440097f7a..67ef153af0e 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -54,8 +54,6 @@ public: SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool async_insert) override; - NamesAndTypesList getVirtuals() const override; - bool supportsSubsetOfColumns() const; std::optional totalRows(const Settings & settings) const override; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 9e2c7bd58ee..27593bfe7a8 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -28,6 +28,13 @@ namespace ErrorCodes extern const int CANNOT_RESTORE_TABLE; } +IStorage::IStorage(StorageID storage_id_) + : storage_id(std::move(storage_id_)) + , metadata(std::make_unique()) + , virtuals(std::make_unique()) +{ +} + bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const { /// Virtual column maybe overridden by real column diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 1ce70d3c84c..bd64447a00e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -99,9 +99,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo public: IStorage() = delete; /// Storage metadata can be set separately in setInMemoryMetadata method - explicit IStorage(StorageID storage_id_) - : storage_id(std::move(storage_id_)) - , metadata(std::make_unique()) {} + explicit IStorage(StorageID storage_id_); IStorage(const IStorage &) = delete; IStorage & operator=(const IStorage &) = delete; diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index dc62c13f633..9c68107872e 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -45,7 +45,7 @@ KafkaSource::KafkaSource( , max_block_size(max_block_size_) , commit_in_suffix(commit_in_suffix_) , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) - , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames())) + , virtual_header(storage.getVirtualsHeader()) , handle_error_mode(storage.getStreamingHandleErrorMode()) { } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index aa347fc719d..3656b599788 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -52,6 +52,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #if USE_KRB5 @@ -344,6 +345,10 @@ StorageKafka::StorageKafka( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + + auto virtuals = createVirtuals(kafka_settings->kafka_handle_error_mode); + setVirtuals(virtuals); + auto task_count = thread_per_consumer ? num_consumers : 1; for (size_t i = 0; i < task_count; ++i) { @@ -365,6 +370,28 @@ StorageKafka::StorageKafka( }); } +VirtualColumnsDescription StorageKafka::createVirtuals(StreamingHandleErrorMode handle_error_mode) +{ + VirtualColumnsDescription desc; + + desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_key", std::make_shared(), ""); + desc.addEphemeral("_offset", std::make_shared(), ""); + desc.addEphemeral("_partition", std::make_shared(), ""); + desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); + desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); + + if (handle_error_mode == StreamingHandleErrorMode::STREAM) + { + desc.addEphemeral("_raw_message", std::make_shared(), ""); + desc.addEphemeral("_error", std::make_shared(), ""); + } + + return desc; +} + SettingsChanges StorageKafka::createSettingsAdjustments() { SettingsChanges result; @@ -1187,43 +1214,4 @@ void registerStorageKafka(StorageFactory & factory) factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); } -NamesAndTypesList StorageKafka::getVirtuals() const -{ - auto result = NamesAndTypesList{ - {"_topic", std::make_shared(std::make_shared())}, - {"_key", std::make_shared()}, - {"_offset", std::make_shared()}, - {"_partition", std::make_shared()}, - {"_timestamp", std::make_shared(std::make_shared())}, - {"_timestamp_ms", std::make_shared(std::make_shared(3))}, - {"_headers.name", std::make_shared(std::make_shared())}, - {"_headers.value", std::make_shared(std::make_shared())}}; - if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) - { - result.push_back({"_raw_message", std::make_shared()}); - result.push_back({"_error", std::make_shared()}); - } - return result; -} - -Names StorageKafka::getVirtualColumnNames() const -{ - auto result = Names { - "_topic", - "_key", - "_offset", - "_partition", - "_timestamp", - "_timestamp_ms", - "_headers.name", - "_headers.value", - }; - if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) - { - result.push_back({"_raw_message"}); - result.push_back({"_error"}); - } - return result; -} - } diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index f9a1e3ff6f3..6a14bd64cc6 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -7,6 +7,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include @@ -74,8 +75,6 @@ public: const auto & getFormatName() const { return format_name; } - NamesAndTypesList getVirtuals() const override; - Names getVirtualColumnNames() const; StreamingHandleErrorMode getStreamingHandleErrorMode() const { return kafka_settings->kafka_handle_error_mode; } struct SafeConsumers @@ -159,6 +158,8 @@ private: bool checkDependencies(const StorageID & table_id); void cleanConsumers(); + + static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); }; } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 2f011567b90..631c88a4dad 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -27,6 +27,7 @@ limitations under the License. */ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #include @@ -218,6 +219,10 @@ StorageLiveView::StorageLiveView( setInMemoryMetadata(storage_metadata); + VirtualColumnsDescription virtuals; + virtuals.addEphemeral("_version", std::make_shared(), ""); + setVirtuals(virtuals); + if (!query.select) throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName()); @@ -256,13 +261,6 @@ StorageLiveView::~StorageLiveView() shutdown(false); } -NamesAndTypesList StorageLiveView::getVirtuals() const -{ - return NamesAndTypesList{ - NameAndTypePair("_version", std::make_shared()) - }; -} - void StorageLiveView::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const { auto table_id = getStorageID(); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 6b8780cb81b..08f6ed4e38f 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -73,8 +73,6 @@ public: bool supportsFinal() const override { return true; } - NamesAndTypesList getVirtuals() const override; - void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override; void drop() override; diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index bbb38346f38..da4405dfd40 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -34,6 +34,7 @@ public: , partition_id(part_->info.partition_id) { setInMemoryMetadata(storage.getInMemoryMetadata()); + setVirtuals(*storage.getVirtualsDescription()); } /// Used in queries with projection. @@ -90,11 +91,6 @@ public: bool supportsSubcolumns() const override { return true; } - NamesAndTypesList getVirtuals() const override - { - return storage.getVirtuals(); - } - String getPartitionId() const { return partition_id; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index c7a5d0b8d0a..a06221e1b26 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -89,6 +89,9 @@ StorageNATS::StorageNATS( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + auto virtuals = createVirtuals(nats_settings->nats_handle_error_mode); + setVirtuals(virtuals); + nats_context = addSettings(getContext()); nats_context->makeQueryContext(); @@ -131,6 +134,19 @@ StorageNATS::StorageNATS( connection_task->deactivate(); } +VirtualColumnsDescription StorageNATS::createVirtuals(StreamingHandleErrorMode handle_error_mode) +{ + VirtualColumnsDescription desc; + desc.addEphemeral("_subject", std::make_shared(), ""); + + if (handle_error_mode == StreamingHandleErrorMode::STREAM) + { + desc.addEphemeral("_raw_message", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_error", std::make_shared(std::make_shared()), ""); + } + + return desc; +} Names StorageNATS::parseList(const String & list, char delim) { @@ -746,20 +762,4 @@ void registerStorageNATS(StorageFactory & factory) factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); } - -NamesAndTypesList StorageNATS::getVirtuals() const -{ - auto virtuals = NamesAndTypesList{ - {"_subject", std::make_shared()} - }; - - if (nats_settings->nats_handle_error_mode == StreamingHandleErrorMode::STREAM) - { - virtuals.push_back({"_raw_message", std::make_shared(std::make_shared())}); - virtuals.push_back({"_error", std::make_shared(std::make_shared())}); - } - - return virtuals; -} - } diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 94f955ccdae..41d77acfde6 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -61,7 +61,6 @@ public: NATSConsumerPtr popConsumer(std::chrono::milliseconds timeout); const String & getFormatName() const { return format_name; } - NamesAndTypesList getVirtuals() const override; void incrementReader(); void decrementReader(); @@ -137,6 +136,7 @@ private: static Names parseList(const String & list, char delim); static String getTableBasedName(String name, const StorageID & table_id); + static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); ContextMutablePtr addSettings(ContextPtr context) const; size_t getMaxBlockSize() const; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 61d83750c31..2914c17b117 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -1,4 +1,5 @@ #include "StorageMaterializedPostgreSQL.h" +#include "Storages/VirtualColumnsDescription.h" #if USE_LIBPQXX #include @@ -72,6 +73,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializedPostgreSQL is allowed only for Atomic database"); setInMemoryMetadata(storage_metadata); + setVirtuals(createVirtuals()); replication_settings->materialized_postgresql_tables_list = remote_table_name_; @@ -127,8 +129,16 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( , nested_table_id(nested_storage_->getStorageID()) { setInMemoryMetadata(nested_storage_->getInMemoryMetadata()); + setVirtuals(*nested_storage_->getVirtualsDescription()); } +VirtualColumnsDescription StorageMaterializedPostgreSQL::createVirtuals() +{ + VirtualColumnsDescription desc; + desc.addEphemeral("_sign", std::make_shared(), ""); + desc.addEphemeral("_version", std::make_shared(), ""); + return desc; +} /// A temporary clone table might be created for current table in order to update its schema and reload /// all data in the background while current table will still handle read requests. @@ -254,15 +264,6 @@ void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool sync, ContextPtr lo } -NamesAndTypesList StorageMaterializedPostgreSQL::getVirtuals() const -{ - return NamesAndTypesList{ - {"_sign", std::make_shared()}, - {"_version", std::make_shared()} - }; -} - - bool StorageMaterializedPostgreSQL::needRewriteQueryWithFinal(const Names & column_names) const { return needRewriteQueryWithFinalForStorage(column_names, getNested()); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index 2d5c1efae7c..41f72ea79e1 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -1,5 +1,6 @@ #pragma once +#include "Storages/VirtualColumnsDescription.h" #include "config.h" #if USE_LIBPQXX @@ -89,8 +90,6 @@ public: /// Used only for single MaterializedPostgreSQL storage. void dropInnerTableIfAny(bool sync, ContextPtr local_context) override; - NamesAndTypesList getVirtuals() const override; - bool needRewriteQueryWithFinal(const Names & column_names) const override; void read( @@ -138,6 +137,8 @@ private: static std::shared_ptr getMaterializedColumnsDeclaration( String name, String type, UInt64 default_value); + static VirtualColumnsDescription createVirtuals(); + ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; String getNestedTableName() const; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index e2ef23193c5..fb7b9c10d7f 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -30,6 +30,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" namespace DB { @@ -137,6 +138,9 @@ StorageRabbitMQ::StorageRabbitMQ( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + auto virtuals = createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode); + setVirtuals(virtuals); + rabbitmq_context = addSettings(getContext()); rabbitmq_context->makeQueryContext(); @@ -191,6 +195,26 @@ StorageRabbitMQ::StorageRabbitMQ( init_task->deactivate(); } +VirtualColumnsDescription StorageRabbitMQ::createVirtuals(StreamingHandleErrorMode handle_error_mode) +{ + VirtualColumnsDescription desc; + + desc.addEphemeral("_exchange_name", std::make_shared(), ""); + desc.addEphemeral("_channel_id", std::make_shared(), ""); + desc.addEphemeral("_delivery_tag", std::make_shared(), ""); + desc.addEphemeral("_redelivered", std::make_shared(), ""); + desc.addEphemeral("_message_id", std::make_shared(), ""); + desc.addEphemeral("_timestamp", std::make_shared(), ""); + + + if (handle_error_mode == StreamingHandleErrorMode::STREAM) + { + desc.addEphemeral("_raw_message", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_error", std::make_shared(std::make_shared()), ""); + } + + return desc; +} Names StorageRabbitMQ::parseSettings(String settings_list) { @@ -1213,25 +1237,4 @@ void registerStorageRabbitMQ(StorageFactory & factory) factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); } - -NamesAndTypesList StorageRabbitMQ::getVirtuals() const -{ - auto virtuals = NamesAndTypesList{ - {"_exchange_name", std::make_shared()}, - {"_channel_id", std::make_shared()}, - {"_delivery_tag", std::make_shared()}, - {"_redelivered", std::make_shared()}, - {"_message_id", std::make_shared()}, - {"_timestamp", std::make_shared()} - }; - - if (rabbitmq_settings->rabbitmq_handle_error_mode == StreamingHandleErrorMode::STREAM) - { - virtuals.push_back({"_raw_message", std::make_shared(std::make_shared())}); - virtuals.push_back({"_error", std::make_shared(std::make_shared())}); - } - - return virtuals; -} - } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index b3a0b53cde5..603a1f1d68f 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -9,6 +9,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #include #include @@ -68,7 +69,6 @@ public: RabbitMQConsumerPtr popConsumer(std::chrono::milliseconds timeout); const String & getFormatName() const { return format_name; } - NamesAndTypesList getVirtuals() const override; String getExchange() const { return exchange_name; } void unbindExchange(); @@ -191,6 +191,8 @@ private: bool tryStreamToViews(); bool hasDependencies(const StorageID & table_id); + static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); + static String getRandomName() { std::uniform_int_distribution distribution('a', 'z'); diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 5113dccda5b..6c7a8e06cc3 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -41,6 +41,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #include @@ -331,6 +332,10 @@ StorageKeeperMap::StorageKeeperMap( setInMemoryMetadata(metadata); + VirtualColumnsDescription virtuals; + virtuals.addEphemeral(String(version_column_name), std::make_shared(), ""); + setVirtuals(virtuals); + WriteBufferFromOwnString out; out << "KeeperMap metadata format version: 1\n" << "columns: " << metadata.columns.toString() @@ -634,12 +639,6 @@ void StorageKeeperMap::drop() dropTable(client, metadata_drop_lock); } -NamesAndTypesList StorageKeeperMap::getVirtuals() const -{ - return NamesAndTypesList{ - {std::string{version_column_name}, std::make_shared()}}; -} - namespace { diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 9dca96a24a3..d1ed282bac5 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -50,8 +50,6 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; void drop() override; - NamesAndTypesList getVirtuals() const override; - std::string getName() const override { return "KeeperMap"; } Names getPrimaryKey() const override { return {primary_key}; } diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index 0dc0b1bff0b..1651b0499ad 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -22,9 +22,8 @@ namespace DB StorageMaterializedMySQL::StorageMaterializedMySQL(const StoragePtr & nested_storage_, const IDatabase * database_) : StorageProxy(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) { - StorageInMemoryMetadata in_memory_metadata; - in_memory_metadata = nested_storage->getInMemoryMetadata(); - setInMemoryMetadata(in_memory_metadata); + setInMemoryMetadata(nested_storage->getInMemoryMetadata()); + setVirtuals(*nested_storage->getVirtualsDescription()); } bool StorageMaterializedMySQL::needRewriteQueryWithFinal(const Names & column_names) const @@ -49,14 +48,6 @@ void StorageMaterializedMySQL::read( query_info, context, processed_stage, max_block_size, num_streams); } -NamesAndTypesList StorageMaterializedMySQL::getVirtuals() const -{ - if (const auto * db = typeid_cast(database)) - db->rethrowExceptionIfNeeded(); - - return nested_storage->getVirtuals(); -} - IStorage::ColumnSizeByName StorageMaterializedMySQL::getColumnSizes() const { auto sizes = nested_storage->getColumnSizes(); diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index 2cd589bfd75..9f5d157ce3b 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -34,7 +34,6 @@ public: SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool) override { throwNotAllowed(); } - NamesAndTypesList getVirtuals() const override; ColumnSizeByName getColumnSizes() const override; StoragePtr getNested() const override { return nested_storage; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 34edc5482f4..9e623001520 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -153,6 +153,8 @@ StorageMaterializedView::StorageMaterializedView( *query.refresh_strategy); refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; } + + setVirtuals(*getTargetTable()->getVirtualsDescription()); } QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( @@ -503,11 +505,6 @@ StoragePtr StorageMaterializedView::tryGetTargetTable() const return DatabaseCatalog::instance().tryGetTable(getTargetTableId(), getContext()); } -NamesAndTypesList StorageMaterializedView::getVirtuals() const -{ - return getTargetTable()->getVirtuals(); -} - Strings StorageMaterializedView::getDataPaths() const { if (auto table = tryGetTargetTable()) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 4d574a821ec..d716527b5ad 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -75,9 +75,6 @@ public: StoragePtr tryGetTargetTable() const; StorageID getTargetTableId() const; - /// Get the virtual column of the target table; - NamesAndTypesList getVirtuals() const override; - ActionLock getActionLock(StorageActionBlockType type) override; void onActionLockRemove(StorageActionBlockType action_type) override; diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 8b062a392d4..6959545430c 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -5,6 +5,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" namespace DB @@ -281,6 +282,8 @@ private: ContextPtr query_context, bool filter_by_database_virtual_column, bool filter_by_table_virtual_column) const; + + // static VirtualColumnsDescription createVirtuals(StoragePtr first_table); }; } diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 18a1f9086ae..a8e7fd528dd 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -30,7 +30,6 @@ public: bool hasEvenlyDistributedRead() const override { return getNested()->hasEvenlyDistributedRead(); } ColumnSizeByName getColumnSizes() const override { return getNested()->getColumnSizes(); } - NamesAndTypesList getVirtuals() const override { return getNested()->getVirtuals(); } QueryProcessingStage::Enum getQueryProcessingStage( ContextPtr context, diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 300b11b7346..191cdab1a40 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -12,12 +12,13 @@ StorageValues::StorageValues( const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_, - const NamesAndTypesList & virtuals_) - : IStorage(table_id_), res_block(res_block_), virtuals(virtuals_) + const VirtualColumnsDescription & virtuals_) + : IStorage(table_id_), res_block(res_block_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + setVirtuals(virtuals_); } Pipe StorageValues::read( diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index af1c134077b..0db3ecec888 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -11,8 +11,14 @@ namespace DB class StorageValues final : public IStorage { public: + /// Why we may have virtual columns in the storage from a single block? + /// Because it used as tmp storage for pushing blocks into views, and some + /// views may contain virtual columns from original storage. StorageValues( - const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_, const NamesAndTypesList & virtuals_ = {}); + const StorageID & table_id_, + const ColumnsDescription & columns_, + const Block & res_block_, + const VirtualColumnsDescription & virtuals_ = {}); std::string getName() const override { return "Values"; } @@ -25,13 +31,6 @@ public: size_t max_block_size, size_t num_streams) override; - /// Why we may have virtual columns in the storage from a single block? - /// Because it used as tmp storage for pushing blocks into views, and some - /// views may contain virtual columns from original storage. - NamesAndTypesList getVirtuals() const override - { - return virtuals; - } /// FIXME probably it should return false, but StorageValues is used in ExecutingInnerQueryFromViewTransform (whatever it is) bool supportsTransactions() const override { return true; } @@ -40,7 +39,6 @@ public: private: Block res_block; - NamesAndTypesList virtuals; }; } diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index c2ed35c5510..353c61e6347 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -16,6 +16,8 @@ #include #include +#include "Storages/System/IStorageSystemOneBlock.h" +#include "Storages/VirtualColumnsDescription.h" #include namespace DB @@ -52,6 +54,14 @@ catch (const DB::Exception &) } +StorageSystemDictionaries::StorageSystemDictionaries(const StorageID & storage_id_) + : DB::IStorageSystemOneBlock(storage_id_) +{ + VirtualColumnsDescription virtuals; + virtuals.addEphemeral("key", std::make_shared(), ""); + setVirtuals(virtuals); +} + ColumnsDescription StorageSystemDictionaries::getColumnsDescription() { return ColumnsDescription @@ -92,13 +102,6 @@ ColumnsDescription StorageSystemDictionaries::getColumnsDescription() }; } -NamesAndTypesList StorageSystemDictionaries::getVirtuals() const -{ - return { - {"key", std::make_shared()} - }; -} - void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & /*query_info*/) const { const auto access = context->getAccess(); diff --git a/src/Storages/System/StorageSystemDictionaries.h b/src/Storages/System/StorageSystemDictionaries.h index 792b3c0dd30..e4f07e3c4bf 100644 --- a/src/Storages/System/StorageSystemDictionaries.h +++ b/src/Storages/System/StorageSystemDictionaries.h @@ -1,6 +1,7 @@ #pragma once #include +#include "Interpreters/StorageID.h" namespace DB @@ -12,12 +13,12 @@ class Context; class StorageSystemDictionaries final : public IStorageSystemOneBlock { public: + explicit StorageSystemDictionaries(const StorageID & storage_id_); + std::string getName() const override { return "SystemDictionaries"; } static ColumnsDescription getColumnsDescription(); - NamesAndTypesList getVirtuals() const override; - protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 48dab8c4777..b4900d26470 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -2,6 +2,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #include #include @@ -259,12 +260,10 @@ StorageSystemPartsBase::StorageSystemPartsBase(const StorageID & table_id_, Colu StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns); setInMemoryMetadata(storage_metadata); + + VirtualColumnsDescription virtuals; + virtuals.addEphemeral("_state", std::make_shared(), ""); + setVirtuals(virtuals); } -NamesAndTypesList StorageSystemPartsBase::getVirtuals() const -{ - return NamesAndTypesList{ - NameAndTypePair("_state", std::make_shared()) - }; -} } diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 0a45d0f9dfe..204f093e46e 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -131,8 +131,6 @@ public: size_t max_block_size, size_t num_streams) override; - NamesAndTypesList getVirtuals() const override; - bool isSystemStorage() const override { return true; } private: From 0c902f8d648789f8cf1117c975f5db2bba1898b4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 28 Feb 2024 18:28:54 +0100 Subject: [PATCH 053/197] Fix style --- src/Processors/Executors/PipelineExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index c3fbe6788c6..8477e011763 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -410,7 +410,7 @@ void PipelineExecutor::executeImpl(size_t num_threads, bool concurrency_control) cancel(); throw; } - + tasks.processAsyncTasks(); pool->wait(); } From 53714e6f7a2cb22ceb703f4cb7a14345b86a7fe9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 28 Feb 2024 19:10:00 +0000 Subject: [PATCH 054/197] Refactor StorageSystemOneBlock --- src/Interpreters/InterpreterSystemQuery.cpp | 1 + src/Storages/RocksDB/StorageSystemRocksDB.cpp | 4 +- src/Storages/RocksDB/StorageSystemRocksDB.h | 4 +- src/Storages/System/IStorageSystemOneBlock.h | 44 ++++----------- ...rageSystemAggregateFunctionCombinators.cpp | 2 +- ...torageSystemAggregateFunctionCombinators.h | 4 +- .../System/StorageSystemAsyncLoader.cpp | 2 +- .../System/StorageSystemAsyncLoader.h | 4 +- .../StorageSystemAsynchronousInserts.cpp | 2 +- .../System/StorageSystemAsynchronousInserts.h | 4 +- .../StorageSystemAsynchronousMetrics.cpp | 4 +- .../System/StorageSystemAsynchronousMetrics.h | 4 +- src/Storages/System/StorageSystemBackups.cpp | 2 +- src/Storages/System/StorageSystemBackups.h | 4 +- .../System/StorageSystemBuildOptions.cpp | 2 +- .../System/StorageSystemBuildOptions.h | 4 +- .../System/StorageSystemCertificates.cpp | 2 +- .../System/StorageSystemCertificates.h | 4 +- src/Storages/System/StorageSystemClusters.cpp | 2 +- src/Storages/System/StorageSystemClusters.h | 4 +- .../System/StorageSystemCollations.cpp | 4 +- src/Storages/System/StorageSystemCollations.h | 4 +- .../System/StorageSystemContributors.cpp | 2 +- .../System/StorageSystemContributors.h | 4 +- .../System/StorageSystemCurrentRoles.cpp | 2 +- .../System/StorageSystemCurrentRoles.h | 4 +- .../System/StorageSystemDDLWorkerQueue.cpp | 2 +- .../System/StorageSystemDDLWorkerQueue.h | 4 +- .../System/StorageSystemDashboards.cpp | 2 +- src/Storages/System/StorageSystemDashboards.h | 4 +- .../System/StorageSystemDataTypeFamilies.cpp | 2 +- .../System/StorageSystemDataTypeFamilies.h | 4 +- .../System/StorageSystemDatabaseEngines.cpp | 2 +- .../System/StorageSystemDatabaseEngines.h | 4 +- .../System/StorageSystemDatabases.cpp | 9 ++- src/Storages/System/StorageSystemDatabases.h | 4 +- .../System/StorageSystemDictionaries.cpp | 2 +- .../System/StorageSystemDictionaries.h | 4 +- .../System/StorageSystemDistributionQueue.cpp | 4 +- .../System/StorageSystemDistributionQueue.h | 4 +- .../System/StorageSystemDroppedTables.cpp | 2 +- .../System/StorageSystemDroppedTables.h | 4 +- .../System/StorageSystemEnabledRoles.cpp | 2 +- .../System/StorageSystemEnabledRoles.h | 4 +- src/Storages/System/StorageSystemErrors.cpp | 2 +- src/Storages/System/StorageSystemErrors.h | 4 +- src/Storages/System/StorageSystemEvents.cpp | 2 +- src/Storages/System/StorageSystemEvents.h | 4 +- .../System/StorageSystemFilesystemCache.cpp | 4 +- .../System/StorageSystemFilesystemCache.h | 4 +- src/Storages/System/StorageSystemFormats.cpp | 2 +- src/Storages/System/StorageSystemFormats.h | 4 +- .../System/StorageSystemFunctions.cpp | 2 +- src/Storages/System/StorageSystemFunctions.h | 4 +- src/Storages/System/StorageSystemGrants.cpp | 2 +- src/Storages/System/StorageSystemGrants.h | 4 +- src/Storages/System/StorageSystemGraphite.cpp | 2 +- src/Storages/System/StorageSystemGraphite.h | 4 +- .../System/StorageSystemKafkaConsumers.cpp | 2 +- .../System/StorageSystemKafkaConsumers.h | 4 +- src/Storages/System/StorageSystemLicenses.cpp | 2 +- src/Storages/System/StorageSystemLicenses.h | 4 +- src/Storages/System/StorageSystemMacros.cpp | 2 +- src/Storages/System/StorageSystemMacros.h | 4 +- .../System/StorageSystemMergeTreeSettings.cpp | 2 +- .../System/StorageSystemMergeTreeSettings.h | 6 +- src/Storages/System/StorageSystemMerges.cpp | 2 +- src/Storages/System/StorageSystemMerges.h | 4 +- src/Storages/System/StorageSystemMetrics.cpp | 2 +- src/Storages/System/StorageSystemMetrics.h | 4 +- src/Storages/System/StorageSystemModels.cpp | 2 +- src/Storages/System/StorageSystemModels.h | 4 +- src/Storages/System/StorageSystemMoves.cpp | 2 +- src/Storages/System/StorageSystemMoves.h | 4 +- .../System/StorageSystemMutations.cpp | 4 +- src/Storages/System/StorageSystemMutations.h | 4 +- .../System/StorageSystemNamedCollections.cpp | 4 +- .../System/StorageSystemNamedCollections.h | 4 +- .../StorageSystemPartMovesBetweenShards.cpp | 4 +- .../StorageSystemPartMovesBetweenShards.h | 4 +- .../System/StorageSystemPrivileges.cpp | 2 +- src/Storages/System/StorageSystemPrivileges.h | 4 +- .../System/StorageSystemProcesses.cpp | 2 +- src/Storages/System/StorageSystemProcesses.h | 4 +- .../System/StorageSystemQueryCache.cpp | 4 +- src/Storages/System/StorageSystemQueryCache.h | 4 +- .../System/StorageSystemQuotaLimits.cpp | 2 +- .../System/StorageSystemQuotaLimits.h | 4 +- .../System/StorageSystemQuotaUsage.cpp | 2 +- src/Storages/System/StorageSystemQuotaUsage.h | 4 +- src/Storages/System/StorageSystemQuotas.cpp | 2 +- src/Storages/System/StorageSystemQuotas.h | 4 +- .../System/StorageSystemQuotasUsage.cpp | 2 +- .../System/StorageSystemQuotasUsage.h | 4 +- .../System/StorageSystemRemoteDataPaths.cpp | 1 + .../System/StorageSystemReplicatedFetches.cpp | 2 +- .../System/StorageSystemReplicatedFetches.h | 4 +- .../System/StorageSystemReplicationQueue.cpp | 4 +- .../System/StorageSystemReplicationQueue.h | 4 +- .../System/StorageSystemRoleGrants.cpp | 2 +- src/Storages/System/StorageSystemRoleGrants.h | 4 +- src/Storages/System/StorageSystemRoles.cpp | 2 +- src/Storages/System/StorageSystemRoles.h | 4 +- .../System/StorageSystemRowPolicies.cpp | 2 +- .../System/StorageSystemRowPolicies.h | 4 +- src/Storages/System/StorageSystemS3Queue.cpp | 4 +- src/Storages/System/StorageSystemS3Queue.h | 4 +- .../System/StorageSystemScheduler.cpp | 2 +- src/Storages/System/StorageSystemScheduler.h | 4 +- .../StorageSystemSchemaInferenceCache.cpp | 2 +- .../StorageSystemSchemaInferenceCache.h | 4 +- .../System/StorageSystemServerSettings.cpp | 2 +- .../System/StorageSystemServerSettings.h | 4 +- src/Storages/System/StorageSystemSettings.cpp | 2 +- src/Storages/System/StorageSystemSettings.h | 4 +- .../System/StorageSystemSettingsChanges.cpp | 2 +- .../System/StorageSystemSettingsChanges.h | 4 +- .../StorageSystemSettingsProfileElements.cpp | 2 +- .../StorageSystemSettingsProfileElements.h | 4 +- .../System/StorageSystemSettingsProfiles.cpp | 2 +- .../System/StorageSystemSettingsProfiles.h | 4 +- .../System/StorageSystemTableEngines.cpp | 2 +- .../System/StorageSystemTableEngines.h | 4 +- .../System/StorageSystemTableFunctions.cpp | 2 +- .../System/StorageSystemTableFunctions.h | 4 +- .../System/StorageSystemTimeZones.cpp | 2 +- src/Storages/System/StorageSystemTimeZones.h | 4 +- .../System/StorageSystemTransactions.cpp | 2 +- .../System/StorageSystemTransactions.h | 4 +- .../System/StorageSystemUserDirectories.cpp | 2 +- .../System/StorageSystemUserDirectories.h | 4 +- .../System/StorageSystemUserProcesses.cpp | 2 +- .../System/StorageSystemUserProcesses.h | 4 +- src/Storages/System/StorageSystemUsers.cpp | 2 +- src/Storages/System/StorageSystemUsers.h | 4 +- .../System/StorageSystemViewRefreshes.cpp | 2 +- .../System/StorageSystemViewRefreshes.h | 4 +- src/Storages/System/StorageSystemWarnings.cpp | 4 +- src/Storages/System/StorageSystemWarnings.h | 4 +- .../StorageSystemZooKeeperConnection.cpp | 2 +- .../System/StorageSystemZooKeeperConnection.h | 4 +- src/Storages/System/attachSystemTables.cpp | 56 +++++++++---------- src/Storages/System/attachSystemTablesImpl.h | 27 +++++++-- 143 files changed, 288 insertions(+), 288 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 266b2ac9fc9..d2c5eea9914 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -60,6 +60,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp index d0533b5ba0c..eec2f53381f 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp +++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp @@ -39,7 +39,7 @@ ColumnsDescription StorageSystemRocksDB::getColumnsDescription() } -void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); @@ -87,7 +87,7 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.h b/src/Storages/RocksDB/StorageSystemRocksDB.h index c1f10a7722d..ec351c75446 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.h +++ b/src/Storages/RocksDB/StorageSystemRocksDB.h @@ -11,7 +11,7 @@ class Context; /** Implements the `rocksdb` system table, which expose various rocksdb metrics. */ -class StorageSystemRocksDB final : public IStorageSystemOneBlock +class StorageSystemRocksDB final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemRocksDB"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; }; } diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 3b2807965a4..a20434fd97e 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -1,13 +1,6 @@ #pragma once -#include -#include -#include #include -#include -#include -#include -#include namespace DB { @@ -17,8 +10,8 @@ class Context; /** IStorageSystemOneBlock is base class for system tables whose all columns can be synchronously fetched. * - * Client class need to provide static method static NamesAndTypesList getNamesAndTypes() that will return list of column names and - * their types. IStorageSystemOneBlock during read will create result columns in same order as result of getNamesAndTypes + * Client class need to provide columns_description. + * IStorageSystemOneBlock during read will create result columns in same order as in columns_description * and pass it with fillData method. * * Client also must override fillData and fill result columns. @@ -26,49 +19,32 @@ class Context; * If subclass want to support virtual columns, it should override getVirtuals method of IStorage interface. * IStorageSystemOneBlock will add virtuals columns at the end of result columns of fillData method. */ -template class IStorageSystemOneBlock : public IStorage { protected: - virtual void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const = 0; + virtual void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector columns_mask) const = 0; virtual bool supportsColumnsMask() const { return false; } + friend class ReadFromSystemOneBlock; + public: - explicit IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_) + explicit IStorageSystemOneBlock(const StorageID & table_id_, ColumnsDescription columns_description) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(Self::getColumnsDescription()); + storage_metadata.setColumns(std::move(columns_description)); setInMemoryMetadata(storage_metadata); } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - size_t /*num_streams*/) override - { - storage_snapshot->check(column_names); - Block sample_block = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); - - if (supportsColumnsMask()) - { - auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); - query_info.columns_mask = std::move(columns_mask); - sample_block = std::move(header); - } - - MutableColumns res_columns = sample_block.cloneEmptyColumns(); - fillData(res_columns, context, query_info); - - UInt64 num_rows = res_columns.at(0)->size(); - Chunk chunk(std::move(res_columns), num_rows); - - return Pipe(std::make_shared(sample_block, std::move(chunk))); - } + size_t /*num_streams*/) override; bool isSystemStorage() const override { return true; } diff --git a/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp b/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp index 8e32a137fcb..7f3994528a9 100644 --- a/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp +++ b/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp @@ -13,7 +13,7 @@ ColumnsDescription StorageSystemAggregateFunctionCombinators::getColumnsDescript }; } -void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { const auto & combinators = AggregateFunctionCombinatorFactory::instance().getAllAggregateFunctionCombinators(); for (const auto & pair : combinators) diff --git a/src/Storages/System/StorageSystemAggregateFunctionCombinators.h b/src/Storages/System/StorageSystemAggregateFunctionCombinators.h index 6f3f90b58af..45036043636 100644 --- a/src/Storages/System/StorageSystemAggregateFunctionCombinators.h +++ b/src/Storages/System/StorageSystemAggregateFunctionCombinators.h @@ -6,10 +6,10 @@ namespace DB { -class StorageSystemAggregateFunctionCombinators final : public IStorageSystemOneBlock +class StorageSystemAggregateFunctionCombinators final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; public: diff --git a/src/Storages/System/StorageSystemAsyncLoader.cpp b/src/Storages/System/StorageSystemAsyncLoader.cpp index c56a3c3ce78..a7ffa282429 100644 --- a/src/Storages/System/StorageSystemAsyncLoader.cpp +++ b/src/Storages/System/StorageSystemAsyncLoader.cpp @@ -74,7 +74,7 @@ ColumnsDescription StorageSystemAsyncLoader::getColumnsDescription() }; } -void StorageSystemAsyncLoader::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemAsyncLoader::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { TimePoint now = std::chrono::system_clock::now(); diff --git a/src/Storages/System/StorageSystemAsyncLoader.h b/src/Storages/System/StorageSystemAsyncLoader.h index fa0ce11efe3..685db264a10 100644 --- a/src/Storages/System/StorageSystemAsyncLoader.h +++ b/src/Storages/System/StorageSystemAsyncLoader.h @@ -10,7 +10,7 @@ namespace DB class Context; /// system.asynchronous_loader table. Takes data from context.getAsyncLoader() -class StorageSystemAsyncLoader final : public IStorageSystemOneBlock +class StorageSystemAsyncLoader final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemAsyncLoader"; } @@ -20,7 +20,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemAsynchronousInserts.cpp b/src/Storages/System/StorageSystemAsynchronousInserts.cpp index 20ba4d1cdfb..b5f413f2e20 100644 --- a/src/Storages/System/StorageSystemAsynchronousInserts.cpp +++ b/src/Storages/System/StorageSystemAsynchronousInserts.cpp @@ -30,7 +30,7 @@ ColumnsDescription StorageSystemAsynchronousInserts::getColumnsDescription() }; } -void StorageSystemAsynchronousInserts::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemAsynchronousInserts::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { using namespace std::chrono; diff --git a/src/Storages/System/StorageSystemAsynchronousInserts.h b/src/Storages/System/StorageSystemAsynchronousInserts.h index 891494ffbeb..748937cffb8 100644 --- a/src/Storages/System/StorageSystemAsynchronousInserts.h +++ b/src/Storages/System/StorageSystemAsynchronousInserts.h @@ -8,7 +8,7 @@ namespace DB /** Implements the system table `asynhronous_inserts`, * which contains information about pending asynchronous inserts in queue. */ -class StorageSystemAsynchronousInserts final : public IStorageSystemOneBlock +class StorageSystemAsynchronousInserts final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemAsynchronousInserts"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index 58940a7e52e..a54b577d72b 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -19,11 +19,11 @@ ColumnsDescription StorageSystemAsynchronousMetrics::getColumnsDescription() StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_) - : IStorageSystemOneBlock(table_id_), async_metrics(async_metrics_) + : IStorageSystemOneBlock(table_id_, getColumnsDescription()), async_metrics(async_metrics_) { } -void StorageSystemAsynchronousMetrics::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemAsynchronousMetrics::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { auto async_metrics_values = async_metrics.getValues(); for (const auto & name_value : async_metrics_values) diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.h b/src/Storages/System/StorageSystemAsynchronousMetrics.h index 026377c77a0..3543b7684d7 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -11,7 +11,7 @@ class Context; /** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics. */ -class StorageSystemAsynchronousMetrics final : public IStorageSystemOneBlock +class StorageSystemAsynchronousMetrics final : public IStorageSystemOneBlock { public: StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_); @@ -24,7 +24,7 @@ private: const AsynchronousMetrics & async_metrics; protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemBackups.cpp b/src/Storages/System/StorageSystemBackups.cpp index 0063d9e308f..fec92229556 100644 --- a/src/Storages/System/StorageSystemBackups.cpp +++ b/src/Storages/System/StorageSystemBackups.cpp @@ -39,7 +39,7 @@ ColumnsDescription StorageSystemBackups::getColumnsDescription() } -void StorageSystemBackups::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemBackups::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { size_t column_index = 0; auto & column_id = assert_cast(*res_columns[column_index++]); diff --git a/src/Storages/System/StorageSystemBackups.h b/src/Storages/System/StorageSystemBackups.h index a081bd52b4c..48e03d12e18 100644 --- a/src/Storages/System/StorageSystemBackups.h +++ b/src/Storages/System/StorageSystemBackups.h @@ -7,7 +7,7 @@ namespace DB { /// Implements `grants` system table, which allows you to get information about grants. -class StorageSystemBackups final : public IStorageSystemOneBlock +class StorageSystemBackups final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemBackups"; } @@ -15,7 +15,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp b/src/Storages/System/StorageSystemBuildOptions.cpp index c12935078af..a7144299eca 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp +++ b/src/Storages/System/StorageSystemBuildOptions.cpp @@ -17,7 +17,7 @@ ColumnsDescription StorageSystemBuildOptions::getColumnsDescription() }; } -void StorageSystemBuildOptions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemBuildOptions::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (auto * it = auto_config_build; *it; it += 2) { diff --git a/src/Storages/System/StorageSystemBuildOptions.h b/src/Storages/System/StorageSystemBuildOptions.h index 7c0bbf6b5fd..dbe651d7513 100644 --- a/src/Storages/System/StorageSystemBuildOptions.h +++ b/src/Storages/System/StorageSystemBuildOptions.h @@ -11,10 +11,10 @@ class Context; /** System table "build_options" with many params used for clickhouse building */ -class StorageSystemBuildOptions final : public IStorageSystemOneBlock +class StorageSystemBuildOptions final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemCertificates.cpp b/src/Storages/System/StorageSystemCertificates.cpp index c372e4b32b0..0e4c5648b74 100644 --- a/src/Storages/System/StorageSystemCertificates.cpp +++ b/src/Storages/System/StorageSystemCertificates.cpp @@ -169,7 +169,7 @@ static void enumCertificates(const std::string & dir, bool def, MutableColumns & #endif -void StorageSystemCertificates::fillData([[maybe_unused]] MutableColumns & res_columns, ContextPtr/* context*/, const SelectQueryInfo &) const +void StorageSystemCertificates::fillData([[maybe_unused]] MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { #if USE_SSL const auto & ca_paths = Poco::Net::SSLManager::instance().defaultServerContext()->getCAPaths(); diff --git a/src/Storages/System/StorageSystemCertificates.h b/src/Storages/System/StorageSystemCertificates.h index f8c8477c998..bc3fef83853 100644 --- a/src/Storages/System/StorageSystemCertificates.h +++ b/src/Storages/System/StorageSystemCertificates.h @@ -13,7 +13,7 @@ class Cluster; * that allows to obtain information about available certificates * and their sources. */ -class StorageSystemCertificates final : public IStorageSystemOneBlock +class StorageSystemCertificates final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemCertificates"; } @@ -23,7 +23,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 6172d4329cd..3c01b4717cc 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -39,7 +39,7 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() return description; } -void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { for (const auto & name_and_cluster : context->getClusters()) writeCluster(res_columns, name_and_cluster, {}); diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 7b568641cb2..0f7c792261d 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -15,7 +15,7 @@ class Cluster; * that allows to obtain information about available clusters * (which may be specified in Distributed tables). */ -class StorageSystemClusters final : public IStorageSystemOneBlock +class StorageSystemClusters final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemClusters"; } @@ -26,7 +26,7 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; using NameAndCluster = std::pair>; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); }; diff --git a/src/Storages/System/StorageSystemCollations.cpp b/src/Storages/System/StorageSystemCollations.cpp index 2e5d11943af..5ef075e53be 100644 --- a/src/Storages/System/StorageSystemCollations.cpp +++ b/src/Storages/System/StorageSystemCollations.cpp @@ -1,6 +1,8 @@ #include #include +#include #include +#include namespace DB { @@ -14,7 +16,7 @@ ColumnsDescription StorageSystemCollations::getColumnsDescription() }; } -void StorageSystemCollations::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemCollations::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (const auto & [locale, lang]: AvailableCollationLocales::instance().getAvailableCollations()) { diff --git a/src/Storages/System/StorageSystemCollations.h b/src/Storages/System/StorageSystemCollations.h index 1fc0ff0e024..449e0d8ac06 100644 --- a/src/Storages/System/StorageSystemCollations.h +++ b/src/Storages/System/StorageSystemCollations.h @@ -5,10 +5,10 @@ namespace DB { -class StorageSystemCollations final : public IStorageSystemOneBlock +class StorageSystemCollations final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemContributors.cpp b/src/Storages/System/StorageSystemContributors.cpp index 860a96c4388..f928337e1bd 100644 --- a/src/Storages/System/StorageSystemContributors.cpp +++ b/src/Storages/System/StorageSystemContributors.cpp @@ -17,7 +17,7 @@ ColumnsDescription StorageSystemContributors::getColumnsDescription() }; } -void StorageSystemContributors::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemContributors::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { std::vector contributors; for (auto * it = auto_contributors; *it; ++it) diff --git a/src/Storages/System/StorageSystemContributors.h b/src/Storages/System/StorageSystemContributors.h index ed983c5e61f..f8d082542d2 100644 --- a/src/Storages/System/StorageSystemContributors.h +++ b/src/Storages/System/StorageSystemContributors.h @@ -9,10 +9,10 @@ class Context; /** System table "contributors" with list of clickhouse contributors */ -class StorageSystemContributors final : public IStorageSystemOneBlock +class StorageSystemContributors final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemCurrentRoles.cpp b/src/Storages/System/StorageSystemCurrentRoles.cpp index 88bdf088175..bfa3a7c2e18 100644 --- a/src/Storages/System/StorageSystemCurrentRoles.cpp +++ b/src/Storages/System/StorageSystemCurrentRoles.cpp @@ -22,7 +22,7 @@ ColumnsDescription StorageSystemCurrentRoles::getColumnsDescription() } -void StorageSystemCurrentRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemCurrentRoles::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto roles_info = context->getRolesInfo(); auto user = context->getUser(); diff --git a/src/Storages/System/StorageSystemCurrentRoles.h b/src/Storages/System/StorageSystemCurrentRoles.h index 4cc9b11d3f4..db1245e0ea7 100644 --- a/src/Storages/System/StorageSystemCurrentRoles.h +++ b/src/Storages/System/StorageSystemCurrentRoles.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `current_roles` system table, which allows you to get information about current roles. -class StorageSystemCurrentRoles final : public IStorageSystemOneBlock +class StorageSystemCurrentRoles final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemCurrentRoles"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index c133a1b597c..ac5dd6c05d0 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -204,7 +204,7 @@ static void fillStatusColumns(MutableColumns & res_columns, size_t & col, } -void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto& ddl_worker = context->getDDLWorker(); fs::path ddl_zookeeper_path = ddl_worker.getQueueDir(); diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.h b/src/Storages/System/StorageSystemDDLWorkerQueue.h index 871bb706f94..eaffb488322 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.h +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.h @@ -11,10 +11,10 @@ class Context; /** System table "distributed_ddl_queue" with list of queries that are currently in the DDL worker queue. */ -class StorageSystemDDLWorkerQueue final : public IStorageSystemOneBlock +class StorageSystemDDLWorkerQueue final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemDashboards.cpp b/src/Storages/System/StorageSystemDashboards.cpp index 7c9e8b73519..23d8fcfc481 100644 --- a/src/Storages/System/StorageSystemDashboards.cpp +++ b/src/Storages/System/StorageSystemDashboards.cpp @@ -22,7 +22,7 @@ String trim(const char * text) return String(view); } -void StorageSystemDashboards::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemDashboards::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { static const std::vector> dashboards { diff --git a/src/Storages/System/StorageSystemDashboards.h b/src/Storages/System/StorageSystemDashboards.h index 83a8664ad27..f3e957e06c5 100644 --- a/src/Storages/System/StorageSystemDashboards.h +++ b/src/Storages/System/StorageSystemDashboards.h @@ -12,7 +12,7 @@ namespace DB class Context; -class StorageSystemDashboards final : public IStorageSystemOneBlock +class StorageSystemDashboards final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemDashboards"; } @@ -22,7 +22,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/src/Storages/System/StorageSystemDataTypeFamilies.cpp index 821a43ed530..da607ade410 100644 --- a/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -17,7 +17,7 @@ ColumnsDescription StorageSystemDataTypeFamilies::getColumnsDescription() }; } -void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { const auto & factory = DataTypeFactory::instance(); auto names = factory.getAllRegisteredNames(); diff --git a/src/Storages/System/StorageSystemDataTypeFamilies.h b/src/Storages/System/StorageSystemDataTypeFamilies.h index 2cb834f6931..6322c3bcfdd 100644 --- a/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -5,10 +5,10 @@ namespace DB { -class StorageSystemDataTypeFamilies final : public IStorageSystemOneBlock +class StorageSystemDataTypeFamilies final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemDatabaseEngines.cpp b/src/Storages/System/StorageSystemDatabaseEngines.cpp index 514ca6d0ab3..59ba60fa871 100644 --- a/src/Storages/System/StorageSystemDatabaseEngines.cpp +++ b/src/Storages/System/StorageSystemDatabaseEngines.cpp @@ -14,7 +14,7 @@ ColumnsDescription StorageSystemDatabaseEngines::getColumnsDescription() }; } -void StorageSystemDatabaseEngines::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemDatabaseEngines::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (const auto & [engine, _] : DatabaseFactory::instance().getDatabaseEngines()) { diff --git a/src/Storages/System/StorageSystemDatabaseEngines.h b/src/Storages/System/StorageSystemDatabaseEngines.h index 16b517c91e6..3af13598c17 100644 --- a/src/Storages/System/StorageSystemDatabaseEngines.h +++ b/src/Storages/System/StorageSystemDatabaseEngines.h @@ -6,10 +6,10 @@ namespace DB { -class StorageSystemDatabaseEngines final : public IStorageSystemOneBlock +class StorageSystemDatabaseEngines final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index 51ecb8f17ca..f5537b508ba 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -72,7 +72,7 @@ static String getEngineFull(const ContextPtr & ctx, const DatabasePtr & database return engine_full; } -static ColumnPtr getFilteredDatabases(const Databases & databases, const SelectQueryInfo & query_info, ContextPtr context) +static ColumnPtr getFilteredDatabases(const Databases & databases, const ActionsDAG::Node * predicate, ContextPtr context) { MutableColumnPtr name_column = ColumnString::create(); MutableColumnPtr engine_column = ColumnString::create(); @@ -94,17 +94,17 @@ static ColumnPtr getFilteredDatabases(const Databases & databases, const SelectQ ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine"), ColumnWithTypeAndName(std::move(uuid_column), std::make_shared(), "uuid") }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); return block.getByPosition(0).column; } -void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector columns_mask) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES); const auto databases = DatabaseCatalog::instance().getDatabases(); - ColumnPtr filtered_databases_column = getFilteredDatabases(databases, query_info, context); + ColumnPtr filtered_databases_column = getFilteredDatabases(databases, predicate, context); for (size_t i = 0; i < filtered_databases_column->size(); ++i) { @@ -120,7 +120,6 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c size_t src_index = 0; size_t res_index = 0; - const auto & columns_mask = query_info.columns_mask; if (columns_mask[src_index++]) res_columns[res_index++]->insert(database_name); if (columns_mask[src_index++]) diff --git a/src/Storages/System/StorageSystemDatabases.h b/src/Storages/System/StorageSystemDatabases.h index 2fd9ccdc970..fa55f0aea32 100644 --- a/src/Storages/System/StorageSystemDatabases.h +++ b/src/Storages/System/StorageSystemDatabases.h @@ -11,7 +11,7 @@ class Context; /** Implements `databases` system table, which allows you to get information about all databases. */ -class StorageSystemDatabases final : public IStorageSystemOneBlock +class StorageSystemDatabases final : public IStorageSystemOneBlock { public: std::string getName() const override @@ -26,7 +26,7 @@ protected: bool supportsColumnsMask() const override { return true; } - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector columns_mask) const override; }; } diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index c2ed35c5510..b3a60e16d05 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -99,7 +99,7 @@ NamesAndTypesList StorageSystemDictionaries::getVirtuals() const }; } -void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & /*query_info*/) const +void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_dictionaries = access->isGranted(AccessType::SHOW_DICTIONARIES); diff --git a/src/Storages/System/StorageSystemDictionaries.h b/src/Storages/System/StorageSystemDictionaries.h index 792b3c0dd30..1a071fda872 100644 --- a/src/Storages/System/StorageSystemDictionaries.h +++ b/src/Storages/System/StorageSystemDictionaries.h @@ -9,7 +9,7 @@ namespace DB class Context; -class StorageSystemDictionaries final : public IStorageSystemOneBlock +class StorageSystemDictionaries final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemDictionaries"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index f0247275a8b..50c6436f316 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -107,7 +107,7 @@ ColumnsDescription StorageSystemDistributionQueue::getColumnsDescription() } -void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); @@ -159,7 +159,7 @@ void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, Cont { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemDistributionQueue.h b/src/Storages/System/StorageSystemDistributionQueue.h index 477a9d6e245..159a86bf082 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.h +++ b/src/Storages/System/StorageSystemDistributionQueue.h @@ -11,7 +11,7 @@ class Context; /** Implements the `distribution_queue` system table, which allows you to view the INSERT queues for the Distributed tables. */ -class StorageSystemDistributionQueue final : public IStorageSystemOneBlock +class StorageSystemDistributionQueue final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemDistributionQueue"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemDroppedTables.cpp b/src/Storages/System/StorageSystemDroppedTables.cpp index a95127847f8..528f4d8995d 100644 --- a/src/Storages/System/StorageSystemDroppedTables.cpp +++ b/src/Storages/System/StorageSystemDroppedTables.cpp @@ -29,7 +29,7 @@ ColumnsDescription StorageSystemDroppedTables::getColumnsDescription() } -void StorageSystemDroppedTables::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemDroppedTables::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); diff --git a/src/Storages/System/StorageSystemDroppedTables.h b/src/Storages/System/StorageSystemDroppedTables.h index d7c3569eb62..4fc620ab962 100644 --- a/src/Storages/System/StorageSystemDroppedTables.h +++ b/src/Storages/System/StorageSystemDroppedTables.h @@ -6,7 +6,7 @@ namespace DB { -class StorageSystemDroppedTables final : public IStorageSystemOneBlock +class StorageSystemDroppedTables final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMarkedDroppedTables"; } @@ -14,7 +14,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemEnabledRoles.cpp b/src/Storages/System/StorageSystemEnabledRoles.cpp index 6dbb6f18488..42806a832cf 100644 --- a/src/Storages/System/StorageSystemEnabledRoles.cpp +++ b/src/Storages/System/StorageSystemEnabledRoles.cpp @@ -23,7 +23,7 @@ ColumnsDescription StorageSystemEnabledRoles::getColumnsDescription() } -void StorageSystemEnabledRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemEnabledRoles::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto roles_info = context->getRolesInfo(); auto user = context->getUser(); diff --git a/src/Storages/System/StorageSystemEnabledRoles.h b/src/Storages/System/StorageSystemEnabledRoles.h index 5367b2ccbea..1b11d025367 100644 --- a/src/Storages/System/StorageSystemEnabledRoles.h +++ b/src/Storages/System/StorageSystemEnabledRoles.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `enabled_roles` system table, which allows you to get information about enabled roles. -class StorageSystemEnabledRoles final : public IStorageSystemOneBlock +class StorageSystemEnabledRoles final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemEnabledRoles"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemErrors.cpp b/src/Storages/System/StorageSystemErrors.cpp index 730e4cf05da..2da268305f8 100644 --- a/src/Storages/System/StorageSystemErrors.cpp +++ b/src/Storages/System/StorageSystemErrors.cpp @@ -25,7 +25,7 @@ ColumnsDescription StorageSystemErrors::getColumnsDescription() } -void StorageSystemErrors::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemErrors::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto add_row = [&](std::string_view name, size_t code, const auto & error, bool remote) { diff --git a/src/Storages/System/StorageSystemErrors.h b/src/Storages/System/StorageSystemErrors.h index 9e8ec628bac..bc86c085ff1 100644 --- a/src/Storages/System/StorageSystemErrors.h +++ b/src/Storages/System/StorageSystemErrors.h @@ -13,7 +13,7 @@ class Context; * Implements the `errors` system table, which shows the error code and the number of times it happens * (i.e. Exception with this code had been thrown). */ -class StorageSystemErrors final : public IStorageSystemOneBlock +class StorageSystemErrors final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemErrors"; } @@ -23,7 +23,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemEvents.cpp b/src/Storages/System/StorageSystemEvents.cpp index 5d9bc3a773a..822d5c77788 100644 --- a/src/Storages/System/StorageSystemEvents.cpp +++ b/src/Storages/System/StorageSystemEvents.cpp @@ -23,7 +23,7 @@ ColumnsDescription StorageSystemEvents::getColumnsDescription() return description; } -void StorageSystemEvents::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemEvents::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { diff --git a/src/Storages/System/StorageSystemEvents.h b/src/Storages/System/StorageSystemEvents.h index cbd92f90d7a..9217fdfb53e 100644 --- a/src/Storages/System/StorageSystemEvents.h +++ b/src/Storages/System/StorageSystemEvents.h @@ -10,7 +10,7 @@ class Context; /** Implements `events` system table, which allows you to obtain information for profiling. */ -class StorageSystemEvents final : public IStorageSystemOneBlock +class StorageSystemEvents final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemEvents"; } @@ -20,7 +20,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index d2bd085f934..53cd76e4219 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -36,11 +36,11 @@ ColumnsDescription StorageSystemFilesystemCache::getColumnsDescription() } StorageSystemFilesystemCache::StorageSystemFilesystemCache(const StorageID & table_id_) - : IStorageSystemOneBlock(table_id_) + : IStorageSystemOneBlock(table_id_, getColumnsDescription()) { } -void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { auto caches = FileCacheFactory::instance().getAll(); diff --git a/src/Storages/System/StorageSystemFilesystemCache.h b/src/Storages/System/StorageSystemFilesystemCache.h index 4b13b375f95..ea49fd16ba2 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.h +++ b/src/Storages/System/StorageSystemFilesystemCache.h @@ -29,7 +29,7 @@ namespace DB * FORMAT Vertical */ -class StorageSystemFilesystemCache final : public IStorageSystemOneBlock +class StorageSystemFilesystemCache final : public IStorageSystemOneBlock { public: explicit StorageSystemFilesystemCache(const StorageID & table_id_); @@ -39,7 +39,7 @@ public: static ColumnsDescription getColumnsDescription(); protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemFormats.cpp b/src/Storages/System/StorageSystemFormats.cpp index a360971e1f7..0d8a5f8bd47 100644 --- a/src/Storages/System/StorageSystemFormats.cpp +++ b/src/Storages/System/StorageSystemFormats.cpp @@ -18,7 +18,7 @@ ColumnsDescription StorageSystemFormats::getColumnsDescription() }; } -void StorageSystemFormats::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemFormats::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { const auto & formats = FormatFactory::instance().getAllFormats(); for (const auto & pair : formats) diff --git a/src/Storages/System/StorageSystemFormats.h b/src/Storages/System/StorageSystemFormats.h index 9f9d1df1bde..f93641ee8a8 100644 --- a/src/Storages/System/StorageSystemFormats.h +++ b/src/Storages/System/StorageSystemFormats.h @@ -4,10 +4,10 @@ namespace DB { -class StorageSystemFormats final : public IStorageSystemOneBlock +class StorageSystemFormats final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemFunctions.cpp b/src/Storages/System/StorageSystemFunctions.cpp index 45c00e6de27..1184ef2c6bf 100644 --- a/src/Storages/System/StorageSystemFunctions.cpp +++ b/src/Storages/System/StorageSystemFunctions.cpp @@ -133,7 +133,7 @@ ColumnsDescription StorageSystemFunctions::getColumnsDescription() }; } -void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto & functions_factory = FunctionFactory::instance(); const auto & function_names = functions_factory.getAllRegisteredNames(); diff --git a/src/Storages/System/StorageSystemFunctions.h b/src/Storages/System/StorageSystemFunctions.h index ac1129e8127..c6f85d436fc 100644 --- a/src/Storages/System/StorageSystemFunctions.h +++ b/src/Storages/System/StorageSystemFunctions.h @@ -12,7 +12,7 @@ class Context; /** Implements `functions`system table, which allows you to get a list * all normal and aggregate functions. */ -class StorageSystemFunctions final : public IStorageSystemOneBlock +class StorageSystemFunctions final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemFunctions"; } @@ -25,7 +25,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemGrants.cpp b/src/Storages/System/StorageSystemGrants.cpp index f5f3fa07e53..afa49536983 100644 --- a/src/Storages/System/StorageSystemGrants.cpp +++ b/src/Storages/System/StorageSystemGrants.cpp @@ -38,7 +38,7 @@ ColumnsDescription StorageSystemGrants::getColumnsDescription() } -void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemGrants.h b/src/Storages/System/StorageSystemGrants.h index 2202b52ad5f..6bf3793c3dc 100644 --- a/src/Storages/System/StorageSystemGrants.h +++ b/src/Storages/System/StorageSystemGrants.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `grants` system table, which allows you to get information about grants. -class StorageSystemGrants final : public IStorageSystemOneBlock +class StorageSystemGrants final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemGrants"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemGraphite.cpp b/src/Storages/System/StorageSystemGraphite.cpp index ffeb1b6c890..a638a08fac7 100644 --- a/src/Storages/System/StorageSystemGraphite.cpp +++ b/src/Storages/System/StorageSystemGraphite.cpp @@ -75,7 +75,7 @@ static StorageSystemGraphite::Configs getConfigs(ContextPtr context) return graphite_configs; } -void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { Configs graphite_configs = getConfigs(context); diff --git a/src/Storages/System/StorageSystemGraphite.h b/src/Storages/System/StorageSystemGraphite.h index be101181cf7..78379afac9d 100644 --- a/src/Storages/System/StorageSystemGraphite.h +++ b/src/Storages/System/StorageSystemGraphite.h @@ -10,7 +10,7 @@ namespace DB { /// Provides information about Graphite configuration. -class StorageSystemGraphite final : public IStorageSystemOneBlock +class StorageSystemGraphite final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemGraphite"; } @@ -30,7 +30,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 95962d8de8b..6c9b1681c8b 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -47,7 +47,7 @@ ColumnsDescription StorageSystemKafkaConsumers::getColumnsDescription() }; } -void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); diff --git a/src/Storages/System/StorageSystemKafkaConsumers.h b/src/Storages/System/StorageSystemKafkaConsumers.h index ae2c726849d..8d1fd504810 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.h +++ b/src/Storages/System/StorageSystemKafkaConsumers.h @@ -11,7 +11,7 @@ namespace DB { -class StorageSystemKafkaConsumers final : public IStorageSystemOneBlock +class StorageSystemKafkaConsumers final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemKafkaConsumers"; } @@ -19,7 +19,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemLicenses.cpp b/src/Storages/System/StorageSystemLicenses.cpp index c5c04b4eb94..2faf772aca4 100644 --- a/src/Storages/System/StorageSystemLicenses.cpp +++ b/src/Storages/System/StorageSystemLicenses.cpp @@ -19,7 +19,7 @@ ColumnsDescription StorageSystemLicenses::getColumnsDescription() }; } -void StorageSystemLicenses::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemLicenses::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (const auto * it = library_licenses; *it; it += 4) { diff --git a/src/Storages/System/StorageSystemLicenses.h b/src/Storages/System/StorageSystemLicenses.h index 57a3ff201a2..ab74a590dea 100644 --- a/src/Storages/System/StorageSystemLicenses.h +++ b/src/Storages/System/StorageSystemLicenses.h @@ -10,10 +10,10 @@ class Context; /** System table "licenses" with list of licenses of 3rd party libraries */ -class StorageSystemLicenses final : public IStorageSystemOneBlock +class StorageSystemLicenses final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemMacros.cpp b/src/Storages/System/StorageSystemMacros.cpp index 6c1a24d152a..82408fd5a7e 100644 --- a/src/Storages/System/StorageSystemMacros.cpp +++ b/src/Storages/System/StorageSystemMacros.cpp @@ -15,7 +15,7 @@ ColumnsDescription StorageSystemMacros::getColumnsDescription() }; } -void StorageSystemMacros::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemMacros::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto macros = context->getMacros(); diff --git a/src/Storages/System/StorageSystemMacros.h b/src/Storages/System/StorageSystemMacros.h index ffbeb70796e..c272985e978 100644 --- a/src/Storages/System/StorageSystemMacros.h +++ b/src/Storages/System/StorageSystemMacros.h @@ -12,7 +12,7 @@ class Context; /** Information about macros for introspection. */ -class StorageSystemMacros final : public IStorageSystemOneBlock +class StorageSystemMacros final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMacros"; } @@ -22,7 +22,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 85caa572edd..7781e3789a4 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -31,7 +31,7 @@ ColumnsDescription SystemMergeTreeSettings::getColumnsDescription() } template -void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto & settings = replicated ? context->getReplicatedMergeTreeSettings() : context->getMergeTreeSettings(); auto constraints_and_current_profiles = context->getSettingsConstraintsAndCurrentProfiles(); diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.h b/src/Storages/System/StorageSystemMergeTreeSettings.h index 48e83f0a880..e2913a7e55b 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.h +++ b/src/Storages/System/StorageSystemMergeTreeSettings.h @@ -14,7 +14,7 @@ class Context; * which allows to get information about the current MergeTree settings. */ template -class SystemMergeTreeSettings final : public IStorageSystemOneBlock> +class SystemMergeTreeSettings final : public IStorageSystemOneBlock { public: std::string getName() const override { return replicated ? "SystemReplicatedMergeTreeSettings" : "SystemMergeTreeSettings"; } @@ -22,9 +22,9 @@ public: static ColumnsDescription getColumnsDescription(); protected: - using IStorageSystemOneBlock>::IStorageSystemOneBlock; + using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 4129e4c235b..fac653e524e 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -39,7 +39,7 @@ ColumnsDescription StorageSystemMerges::getColumnsDescription() } -void StorageSystemMerges::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemMerges::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); diff --git a/src/Storages/System/StorageSystemMerges.h b/src/Storages/System/StorageSystemMerges.h index 961d28daf9a..fd9077c56d5 100644 --- a/src/Storages/System/StorageSystemMerges.h +++ b/src/Storages/System/StorageSystemMerges.h @@ -12,7 +12,7 @@ namespace DB class Context; -class StorageSystemMerges final : public IStorageSystemOneBlock +class StorageSystemMerges final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMerges"; } @@ -22,7 +22,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemMetrics.cpp b/src/Storages/System/StorageSystemMetrics.cpp index 30544d66070..ae34a04cd87 100644 --- a/src/Storages/System/StorageSystemMetrics.cpp +++ b/src/Storages/System/StorageSystemMetrics.cpp @@ -25,7 +25,7 @@ ColumnsDescription StorageSystemMetrics::getColumnsDescription() return description; } -void StorageSystemMetrics::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemMetrics::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) { diff --git a/src/Storages/System/StorageSystemMetrics.h b/src/Storages/System/StorageSystemMetrics.h index ec0c67cf6b7..829fc231a79 100644 --- a/src/Storages/System/StorageSystemMetrics.h +++ b/src/Storages/System/StorageSystemMetrics.h @@ -11,7 +11,7 @@ class Context; /** Implements `metrics` system table, which provides information about the operation of the server. */ -class StorageSystemMetrics final : public IStorageSystemOneBlock +class StorageSystemMetrics final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMetrics"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemModels.cpp b/src/Storages/System/StorageSystemModels.cpp index 03b659d79bd..e715238ddd4 100644 --- a/src/Storages/System/StorageSystemModels.cpp +++ b/src/Storages/System/StorageSystemModels.cpp @@ -23,7 +23,7 @@ ColumnsDescription StorageSystemModels::getColumnsDescription() }; } -void StorageSystemModels::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemModels::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto bridge_helper = std::make_unique(context); ExternalModelInfos infos = bridge_helper->listModels(); diff --git a/src/Storages/System/StorageSystemModels.h b/src/Storages/System/StorageSystemModels.h index 91fa3761743..419b623dac0 100644 --- a/src/Storages/System/StorageSystemModels.h +++ b/src/Storages/System/StorageSystemModels.h @@ -9,7 +9,7 @@ namespace DB class Context; -class StorageSystemModels final : public IStorageSystemOneBlock +class StorageSystemModels final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemModels"; } @@ -19,7 +19,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemMoves.cpp b/src/Storages/System/StorageSystemMoves.cpp index 9b749218283..2c43043d820 100644 --- a/src/Storages/System/StorageSystemMoves.cpp +++ b/src/Storages/System/StorageSystemMoves.cpp @@ -23,7 +23,7 @@ ColumnsDescription StorageSystemMoves::getColumnsDescription() } -void StorageSystemMoves::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemMoves::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); diff --git a/src/Storages/System/StorageSystemMoves.h b/src/Storages/System/StorageSystemMoves.h index acdd9642f8f..45b1e9c6121 100644 --- a/src/Storages/System/StorageSystemMoves.h +++ b/src/Storages/System/StorageSystemMoves.h @@ -12,7 +12,7 @@ namespace DB class Context; -class StorageSystemMoves final : public IStorageSystemOneBlock +class StorageSystemMoves final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMoves"; } @@ -22,7 +22,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index 50545a55c7f..7d263d9468d 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -46,7 +46,7 @@ ColumnsDescription StorageSystemMutations::getColumnsDescription() } -void StorageSystemMutations::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemMutations::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); @@ -100,7 +100,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, ContextPtr c { col_table, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemMutations.h b/src/Storages/System/StorageSystemMutations.h index 2db6e0c17f1..c60157cd853 100644 --- a/src/Storages/System/StorageSystemMutations.h +++ b/src/Storages/System/StorageSystemMutations.h @@ -11,7 +11,7 @@ class Context; /// Implements the `mutations` system table, which provides information about the status of mutations /// in the MergeTree tables. -class StorageSystemMutations final : public IStorageSystemOneBlock +class StorageSystemMutations final : public IStorageSystemOneBlock { public: String getName() const override { return "SystemMutations"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 25401bb751b..156fa5e5a9b 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -25,11 +25,11 @@ ColumnsDescription StorageSystemNamedCollections::getColumnsDescription() } StorageSystemNamedCollections::StorageSystemNamedCollections(const StorageID & table_id_) - : IStorageSystemOneBlock(table_id_) + : IStorageSystemOneBlock(table_id_, getColumnsDescription()) { } -void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto & access = context->getAccess(); diff --git a/src/Storages/System/StorageSystemNamedCollections.h b/src/Storages/System/StorageSystemNamedCollections.h index 596df99be83..ab302b400fc 100644 --- a/src/Storages/System/StorageSystemNamedCollections.h +++ b/src/Storages/System/StorageSystemNamedCollections.h @@ -5,7 +5,7 @@ namespace DB { -class StorageSystemNamedCollections final : public IStorageSystemOneBlock +class StorageSystemNamedCollections final : public IStorageSystemOneBlock { public: explicit StorageSystemNamedCollections(const StorageID & table_id_); @@ -15,7 +15,7 @@ public: static ColumnsDescription getColumnsDescription(); protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp index 1a2646d3295..9dd2ba0b156 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp @@ -44,7 +44,7 @@ ColumnsDescription StorageSystemPartMovesBetweenShards::getColumnsDescription() } -void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); @@ -95,7 +95,7 @@ void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.h b/src/Storages/System/StorageSystemPartMovesBetweenShards.h index 93a26bcd1b7..6a859d4de80 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.h +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.h @@ -9,7 +9,7 @@ namespace DB class Context; -class StorageSystemPartMovesBetweenShards final : public IStorageSystemOneBlock +class StorageSystemPartMovesBetweenShards final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemShardMoves"; } @@ -19,7 +19,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index f45f3c6ed01..0aca9921257 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -77,7 +77,7 @@ ColumnsDescription StorageSystemPrivileges::getColumnsDescription() } -void StorageSystemPrivileges::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemPrivileges::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { size_t column_index = 0; auto & column_access_type = assert_cast(*res_columns[column_index++]).getData(); diff --git a/src/Storages/System/StorageSystemPrivileges.h b/src/Storages/System/StorageSystemPrivileges.h index 4441cf78d5c..eaef7f0db6d 100644 --- a/src/Storages/System/StorageSystemPrivileges.h +++ b/src/Storages/System/StorageSystemPrivileges.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `privileges` system table, which allows you to get information about access types. -class StorageSystemPrivileges final : public IStorageSystemOneBlock +class StorageSystemPrivileges final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemPrivileges"; } @@ -17,7 +17,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemProcesses.cpp b/src/Storages/System/StorageSystemProcesses.cpp index 6702e68b81e..b6fd1aabd45 100644 --- a/src/Storages/System/StorageSystemProcesses.cpp +++ b/src/Storages/System/StorageSystemProcesses.cpp @@ -81,7 +81,7 @@ ColumnsDescription StorageSystemProcesses::getColumnsDescription() return description; } -void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { ProcessList::Info info = context->getProcessList().getInfo(true, true, true); diff --git a/src/Storages/System/StorageSystemProcesses.h b/src/Storages/System/StorageSystemProcesses.h index 3017f9fd367..eb241d4b1ae 100644 --- a/src/Storages/System/StorageSystemProcesses.h +++ b/src/Storages/System/StorageSystemProcesses.h @@ -11,7 +11,7 @@ class Context; /** Implements `processes` system table, which allows you to get information about the queries that are currently executing. */ -class StorageSystemProcesses final : public IStorageSystemOneBlock +class StorageSystemProcesses final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemProcesses"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 03111755904..e454012da3a 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -24,11 +24,11 @@ ColumnsDescription StorageSystemQueryCache::getColumnsDescription() } StorageSystemQueryCache::StorageSystemQueryCache(const StorageID & table_id_) - : IStorageSystemOneBlock(table_id_) + : IStorageSystemOneBlock(table_id_, getColumnsDescription()) { } -void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { QueryCachePtr query_cache = context->getQueryCache(); diff --git a/src/Storages/System/StorageSystemQueryCache.h b/src/Storages/System/StorageSystemQueryCache.h index 08ad30afb81..22856c2b1bb 100644 --- a/src/Storages/System/StorageSystemQueryCache.h +++ b/src/Storages/System/StorageSystemQueryCache.h @@ -5,7 +5,7 @@ namespace DB { -class StorageSystemQueryCache final : public IStorageSystemOneBlock +class StorageSystemQueryCache final : public IStorageSystemOneBlock { public: explicit StorageSystemQueryCache(const StorageID & table_id_); @@ -15,7 +15,7 @@ public: static ColumnsDescription getColumnsDescription(); protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemQuotaLimits.cpp b/src/Storages/System/StorageSystemQuotaLimits.cpp index f125a990a88..dba449d4f1d 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.cpp +++ b/src/Storages/System/StorageSystemQuotaLimits.cpp @@ -75,7 +75,7 @@ ColumnsDescription StorageSystemQuotaLimits::getColumnsDescription() } -void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemQuotaLimits.h b/src/Storages/System/StorageSystemQuotaLimits.h index acc977d0df7..a8385e878ca 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.h +++ b/src/Storages/System/StorageSystemQuotaLimits.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `quota_limits` system table, which allows you to get information about the limits set for quotas. -class StorageSystemQuotaLimits final : public IStorageSystemOneBlock +class StorageSystemQuotaLimits final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemQuotaLimits"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemQuotaUsage.cpp b/src/Storages/System/StorageSystemQuotaUsage.cpp index a91e8b7b2c1..2df36aee240 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -89,7 +89,7 @@ ColumnsDescription StorageSystemQuotaUsage::getColumnsDescriptionImpl(bool add_c } -void StorageSystemQuotaUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemQuotaUsage::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemQuotaUsage.h b/src/Storages/System/StorageSystemQuotaUsage.h index a3109e9ca31..3100098fe87 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.h +++ b/src/Storages/System/StorageSystemQuotaUsage.h @@ -12,7 +12,7 @@ struct QuotaUsage; /** Implements the `quota_usage` system table, which allows you to get information about * how the current user uses the quota. */ -class StorageSystemQuotaUsage final : public IStorageSystemOneBlock +class StorageSystemQuotaUsage final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemQuotaUsage"; } @@ -23,7 +23,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemQuotas.cpp b/src/Storages/System/StorageSystemQuotas.cpp index ee302f2f163..641bbb319d5 100644 --- a/src/Storages/System/StorageSystemQuotas.cpp +++ b/src/Storages/System/StorageSystemQuotas.cpp @@ -66,7 +66,7 @@ ColumnsDescription StorageSystemQuotas::getColumnsDescription() } -void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemQuotas.h b/src/Storages/System/StorageSystemQuotas.h index cafd8b921fa..76e6f1df536 100644 --- a/src/Storages/System/StorageSystemQuotas.h +++ b/src/Storages/System/StorageSystemQuotas.h @@ -9,7 +9,7 @@ class Context; /** Implements the `quotas` system tables, which allows you to get information about quotas. */ -class StorageSystemQuotas final : public IStorageSystemOneBlock +class StorageSystemQuotas final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemQuotas"; } @@ -20,7 +20,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemQuotasUsage.cpp b/src/Storages/System/StorageSystemQuotasUsage.cpp index ed22f73dd50..1587048e7e7 100644 --- a/src/Storages/System/StorageSystemQuotasUsage.cpp +++ b/src/Storages/System/StorageSystemQuotasUsage.cpp @@ -13,7 +13,7 @@ ColumnsDescription StorageSystemQuotasUsage::getColumnsDescription() return StorageSystemQuotaUsage::getColumnsDescriptionImpl(/* add_column_is_current = */ true); } -void StorageSystemQuotasUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemQuotasUsage::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemQuotasUsage.h b/src/Storages/System/StorageSystemQuotasUsage.h index ecdc62865d1..516e722f7df 100644 --- a/src/Storages/System/StorageSystemQuotasUsage.h +++ b/src/Storages/System/StorageSystemQuotasUsage.h @@ -10,7 +10,7 @@ class Context; /** Implements the `quotas_usage` system table, which allows you to get information about * how all users use the quotas. */ -class StorageSystemQuotasUsage final : public IStorageSystemOneBlock +class StorageSystemQuotasUsage final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemQuotasUsage"; } @@ -18,7 +18,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 87b7a84e8ba..55002d6d00f 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/System/StorageSystemReplicatedFetches.cpp b/src/Storages/System/StorageSystemReplicatedFetches.cpp index e643cb9f86a..6913665a8d2 100644 --- a/src/Storages/System/StorageSystemReplicatedFetches.cpp +++ b/src/Storages/System/StorageSystemReplicatedFetches.cpp @@ -34,7 +34,7 @@ ColumnsDescription StorageSystemReplicatedFetches::getColumnsDescription() }; } -void StorageSystemReplicatedFetches::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemReplicatedFetches::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); diff --git a/src/Storages/System/StorageSystemReplicatedFetches.h b/src/Storages/System/StorageSystemReplicatedFetches.h index a176912cac0..dba9124b39d 100644 --- a/src/Storages/System/StorageSystemReplicatedFetches.h +++ b/src/Storages/System/StorageSystemReplicatedFetches.h @@ -10,7 +10,7 @@ namespace DB class Context; /// system.replicated_fetches table. Takes data from context.getReplicatedFetchList() -class StorageSystemReplicatedFetches final : public IStorageSystemOneBlock +class StorageSystemReplicatedFetches final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemReplicatedFetches"; } @@ -20,7 +20,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index 194a2ae6fb8..14b641f46c7 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -62,7 +62,7 @@ ColumnsDescription StorageSystemReplicationQueue::getColumnsDescription() } -void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); @@ -113,7 +113,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, Conte { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemReplicationQueue.h b/src/Storages/System/StorageSystemReplicationQueue.h index 003e4eeb927..a9e57851be1 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.h +++ b/src/Storages/System/StorageSystemReplicationQueue.h @@ -11,7 +11,7 @@ class Context; /** Implements the `replication_queue` system table, which allows you to view the replication queues for the replicated tables. */ -class StorageSystemReplicationQueue final : public IStorageSystemOneBlock +class StorageSystemReplicationQueue final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemReplicationQueue"; } @@ -20,7 +20,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemRoleGrants.cpp b/src/Storages/System/StorageSystemRoleGrants.cpp index 241481275c4..e5baeed4873 100644 --- a/src/Storages/System/StorageSystemRoleGrants.cpp +++ b/src/Storages/System/StorageSystemRoleGrants.cpp @@ -40,7 +40,7 @@ ColumnsDescription StorageSystemRoleGrants::getColumnsDescription() } -void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemRoleGrants.h b/src/Storages/System/StorageSystemRoleGrants.h index 969f82f85d5..f82aece3f24 100644 --- a/src/Storages/System/StorageSystemRoleGrants.h +++ b/src/Storages/System/StorageSystemRoleGrants.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `role_grants` system table, which allows you to get information about granted roles. -class StorageSystemRoleGrants final : public IStorageSystemOneBlock +class StorageSystemRoleGrants final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemRoleGrants"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemRoles.cpp b/src/Storages/System/StorageSystemRoles.cpp index ec26a50db77..9bfddc25ebf 100644 --- a/src/Storages/System/StorageSystemRoles.cpp +++ b/src/Storages/System/StorageSystemRoles.cpp @@ -25,7 +25,7 @@ ColumnsDescription StorageSystemRoles::getColumnsDescription() } -void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemRoles.h b/src/Storages/System/StorageSystemRoles.h index 2b4ae93a932..e96bb70d9cd 100644 --- a/src/Storages/System/StorageSystemRoles.h +++ b/src/Storages/System/StorageSystemRoles.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `roles` system table, which allows you to get information about roles. -class StorageSystemRoles final : public IStorageSystemOneBlock +class StorageSystemRoles final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemRoles"; } @@ -19,7 +19,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index 5a959cdf9af..ea819e88993 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -61,7 +61,7 @@ ColumnsDescription StorageSystemRowPolicies::getColumnsDescription() } -void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemRowPolicies.h b/src/Storages/System/StorageSystemRowPolicies.h index f8aa5618126..cdbc4731000 100644 --- a/src/Storages/System/StorageSystemRowPolicies.h +++ b/src/Storages/System/StorageSystemRowPolicies.h @@ -10,7 +10,7 @@ class Context; /// Implements `row_policies` system table, which allows you to get information about row policies. -class StorageSystemRowPolicies final : public IStorageSystemOneBlock +class StorageSystemRowPolicies final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemRowPolicies"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 3637734b225..557f0fd1208 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -37,11 +37,11 @@ ColumnsDescription StorageSystemS3Queue::getColumnsDescription() } StorageSystemS3Queue::StorageSystemS3Queue(const StorageID & table_id_) - : IStorageSystemOneBlock(table_id_) + : IStorageSystemOneBlock(table_id_, getColumnsDescription()) { } -void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (const auto & [zookeeper_path, metadata] : S3QueueMetadataFactory::instance().getAll()) { diff --git a/src/Storages/System/StorageSystemS3Queue.h b/src/Storages/System/StorageSystemS3Queue.h index 1dc5c521941..c89e18f458f 100644 --- a/src/Storages/System/StorageSystemS3Queue.h +++ b/src/Storages/System/StorageSystemS3Queue.h @@ -7,7 +7,7 @@ namespace DB { -class StorageSystemS3Queue final : public IStorageSystemOneBlock +class StorageSystemS3Queue final : public IStorageSystemOneBlock { public: explicit StorageSystemS3Queue(const StorageID & table_id_); @@ -17,7 +17,7 @@ public: static ColumnsDescription getColumnsDescription(); protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index ba07d44dbf9..cae42011fc5 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -80,7 +80,7 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription() } -void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) { diff --git a/src/Storages/System/StorageSystemScheduler.h b/src/Storages/System/StorageSystemScheduler.h index 1de72a85e9b..c6a259e5b51 100644 --- a/src/Storages/System/StorageSystemScheduler.h +++ b/src/Storages/System/StorageSystemScheduler.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `system.scheduler` table, which allows you to get information about scheduling nodes. -class StorageSystemScheduler final : public IStorageSystemOneBlock +class StorageSystemScheduler final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemScheduler"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp index 1426ea83800..634089bd1cd 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp @@ -72,7 +72,7 @@ static void fillDataImpl(MutableColumns & res_columns, SchemaCache & schema_cach } } -void StorageSystemSchemaInferenceCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemSchemaInferenceCache::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { fillDataImpl(res_columns, StorageFile::getSchemaCache(context), "File"); #if USE_AWS_S3 diff --git a/src/Storages/System/StorageSystemSchemaInferenceCache.h b/src/Storages/System/StorageSystemSchemaInferenceCache.h index e6d306f8252..3e12f4b850b 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.h +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.h @@ -6,7 +6,7 @@ namespace DB { -class StorageSystemSchemaInferenceCache final : public IStorageSystemOneBlock +class StorageSystemSchemaInferenceCache final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemSettingsChanges"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index f390985546b..b75f4280877 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -55,7 +55,7 @@ ColumnsDescription StorageSystemServerSettings::getColumnsDescription() }; } -void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// When the server configuration file is periodically re-loaded from disk, the server components (e.g. memory tracking) are updated /// with new the setting values but the settings themselves are not stored between re-loads. As a result, if one wants to know the diff --git a/src/Storages/System/StorageSystemServerSettings.h b/src/Storages/System/StorageSystemServerSettings.h index 276f21d674b..03c363c9920 100644 --- a/src/Storages/System/StorageSystemServerSettings.h +++ b/src/Storages/System/StorageSystemServerSettings.h @@ -11,7 +11,7 @@ class Context; /** implements system table "settings", which allows to get information about the current settings. */ -class StorageSystemServerSettings final : public IStorageSystemOneBlock +class StorageSystemServerSettings final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemServerSettings"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index 43877582af6..b437108b00e 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -34,7 +34,7 @@ ColumnsDescription StorageSystemSettings::getColumnsDescription() }; } -void StorageSystemSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemSettings::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const Settings & settings = context->getSettingsRef(); auto constraints_and_current_profiles = context->getSettingsConstraintsAndCurrentProfiles(); diff --git a/src/Storages/System/StorageSystemSettings.h b/src/Storages/System/StorageSystemSettings.h index 6749f9b20a4..fae0d69ada8 100644 --- a/src/Storages/System/StorageSystemSettings.h +++ b/src/Storages/System/StorageSystemSettings.h @@ -11,7 +11,7 @@ class Context; /** implements system table "settings", which allows to get information about the current settings. */ -class StorageSystemSettings final : public IStorageSystemOneBlock +class StorageSystemSettings final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemSettings"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemSettingsChanges.cpp b/src/Storages/System/StorageSystemSettingsChanges.cpp index b1942ea9ac6..ecac76b73a3 100644 --- a/src/Storages/System/StorageSystemSettingsChanges.cpp +++ b/src/Storages/System/StorageSystemSettingsChanges.cpp @@ -24,7 +24,7 @@ ColumnsDescription StorageSystemSettingsChanges::getColumnsDescription() }; } -void StorageSystemSettingsChanges::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemSettingsChanges::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (auto it = settings_changes_history.rbegin(); it != settings_changes_history.rend(); ++it) { diff --git a/src/Storages/System/StorageSystemSettingsChanges.h b/src/Storages/System/StorageSystemSettingsChanges.h index 3a1a8ce23d1..9d8899797fe 100644 --- a/src/Storages/System/StorageSystemSettingsChanges.h +++ b/src/Storages/System/StorageSystemSettingsChanges.h @@ -12,7 +12,7 @@ class Context; /** Implements system table "settings_changes", which allows to get information * about the settings changes through different ClickHouse versions. */ -class StorageSystemSettingsChanges final : public IStorageSystemOneBlock +class StorageSystemSettingsChanges final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemSettingsChanges"; } @@ -22,7 +22,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp index a530bd2c1b3..6ac5d13a249 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.cpp +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -51,7 +51,7 @@ ColumnsDescription StorageSystemSettingsProfileElements::getColumnsDescription() } -void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.h b/src/Storages/System/StorageSystemSettingsProfileElements.h index 1dedd616c82..8b08c463071 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.h +++ b/src/Storages/System/StorageSystemSettingsProfileElements.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `settings_profile_elements` system table, which allows you to get information about elements of settings profiles. -class StorageSystemSettingsProfileElements final : public IStorageSystemOneBlock +class StorageSystemSettingsProfileElements final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemSettingsProfileElements"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemSettingsProfiles.cpp b/src/Storages/System/StorageSystemSettingsProfiles.cpp index 01041bee445..795152e31f3 100644 --- a/src/Storages/System/StorageSystemSettingsProfiles.cpp +++ b/src/Storages/System/StorageSystemSettingsProfiles.cpp @@ -36,7 +36,7 @@ ColumnsDescription StorageSystemSettingsProfiles::getColumnsDescription() } -void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemSettingsProfiles.h b/src/Storages/System/StorageSystemSettingsProfiles.h index b0c8fc8658c..056666ae4c7 100644 --- a/src/Storages/System/StorageSystemSettingsProfiles.h +++ b/src/Storages/System/StorageSystemSettingsProfiles.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `settings_profiles` system table, which allows you to get information about profiles. -class StorageSystemSettingsProfiles final : public IStorageSystemOneBlock +class StorageSystemSettingsProfiles final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemSettingsProfiles"; } @@ -19,7 +19,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemTableEngines.cpp b/src/Storages/System/StorageSystemTableEngines.cpp index c0cf95423d9..e27546aa2a4 100644 --- a/src/Storages/System/StorageSystemTableEngines.cpp +++ b/src/Storages/System/StorageSystemTableEngines.cpp @@ -26,7 +26,7 @@ ColumnsDescription StorageSystemTableEngines::getColumnsDescription() }; } -void StorageSystemTableEngines::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemTableEngines::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (const auto & pair : StorageFactory::instance().getAllStorages()) { diff --git a/src/Storages/System/StorageSystemTableEngines.h b/src/Storages/System/StorageSystemTableEngines.h index 258b9d210b1..d7af471bb2d 100644 --- a/src/Storages/System/StorageSystemTableEngines.h +++ b/src/Storages/System/StorageSystemTableEngines.h @@ -6,10 +6,10 @@ namespace DB { -class StorageSystemTableEngines final : public IStorageSystemOneBlock +class StorageSystemTableEngines final : public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemTableFunctions.cpp b/src/Storages/System/StorageSystemTableFunctions.cpp index 9fb8e11e4d1..94b7d73a67e 100644 --- a/src/Storages/System/StorageSystemTableFunctions.cpp +++ b/src/Storages/System/StorageSystemTableFunctions.cpp @@ -20,7 +20,7 @@ ColumnsDescription StorageSystemTableFunctions::getColumnsDescription() }; } -void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { const auto & factory = TableFunctionFactory::instance(); const auto & functions_names = factory.getAllRegisteredNames(); diff --git a/src/Storages/System/StorageSystemTableFunctions.h b/src/Storages/System/StorageSystemTableFunctions.h index 804c3b51940..07ef8857135 100644 --- a/src/Storages/System/StorageSystemTableFunctions.h +++ b/src/Storages/System/StorageSystemTableFunctions.h @@ -6,12 +6,12 @@ namespace DB { -class StorageSystemTableFunctions final : public IStorageSystemOneBlock +class StorageSystemTableFunctions final : public IStorageSystemOneBlock { protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; public: std::string getName() const override diff --git a/src/Storages/System/StorageSystemTimeZones.cpp b/src/Storages/System/StorageSystemTimeZones.cpp index 14f4ce0f5de..af997c6423f 100644 --- a/src/Storages/System/StorageSystemTimeZones.cpp +++ b/src/Storages/System/StorageSystemTimeZones.cpp @@ -16,7 +16,7 @@ ColumnsDescription StorageSystemTimeZones::getColumnsDescription() }; } -void StorageSystemTimeZones::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemTimeZones::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { for (auto * it = auto_time_zones; *it; ++it) res_columns[0]->insert(String(*it)); diff --git a/src/Storages/System/StorageSystemTimeZones.h b/src/Storages/System/StorageSystemTimeZones.h index f3743a1ef09..160e8854e3e 100644 --- a/src/Storages/System/StorageSystemTimeZones.h +++ b/src/Storages/System/StorageSystemTimeZones.h @@ -10,10 +10,10 @@ class Context; /** System table "time_zones" with list of timezones pulled from /contrib/cctz/testdata/zoneinfo */ -class StorageSystemTimeZones final : public IStorageSystemOneBlock +class StorageSystemTimeZones final : public IStorageSystemOneBlock { public: - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemTransactions.cpp b/src/Storages/System/StorageSystemTransactions.cpp index 47e44688c14..edc3739e713 100644 --- a/src/Storages/System/StorageSystemTransactions.cpp +++ b/src/Storages/System/StorageSystemTransactions.cpp @@ -34,7 +34,7 @@ ColumnsDescription StorageSystemTransactions::getColumnsDescription() }; } -void StorageSystemTransactions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemTransactions::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { auto list = TransactionLog::instance().getTransactionsList(); for (const auto & elem : list) diff --git a/src/Storages/System/StorageSystemTransactions.h b/src/Storages/System/StorageSystemTransactions.h index b5a538b7b55..1c400619c1a 100644 --- a/src/Storages/System/StorageSystemTransactions.h +++ b/src/Storages/System/StorageSystemTransactions.h @@ -8,7 +8,7 @@ namespace DB class Context; -class StorageSystemTransactions final : public IStorageSystemOneBlock +class StorageSystemTransactions final : public IStorageSystemOneBlock { public: String getName() const override { return "SystemTransactions"; } @@ -18,7 +18,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemUserDirectories.cpp b/src/Storages/System/StorageSystemUserDirectories.cpp index 620c9746a4a..7b6c1144ae1 100644 --- a/src/Storages/System/StorageSystemUserDirectories.cpp +++ b/src/Storages/System/StorageSystemUserDirectories.cpp @@ -22,7 +22,7 @@ ColumnsDescription StorageSystemUserDirectories::getColumnsDescription() } -void StorageSystemUserDirectories::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemUserDirectories::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto & access_control = context->getAccessControl(); auto storages = access_control.getStorages(); diff --git a/src/Storages/System/StorageSystemUserDirectories.h b/src/Storages/System/StorageSystemUserDirectories.h index bca6a9b5aa6..3cdaa877b81 100644 --- a/src/Storages/System/StorageSystemUserDirectories.h +++ b/src/Storages/System/StorageSystemUserDirectories.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `users_directories` system table, which allows you to get information about user directories. -class StorageSystemUserDirectories final : public IStorageSystemOneBlock +class StorageSystemUserDirectories final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemUserDirectories"; } @@ -16,7 +16,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemUserProcesses.cpp b/src/Storages/System/StorageSystemUserProcesses.cpp index 4fbbd7ab54d..d36129aea63 100644 --- a/src/Storages/System/StorageSystemUserProcesses.cpp +++ b/src/Storages/System/StorageSystemUserProcesses.cpp @@ -32,7 +32,7 @@ ColumnsDescription StorageSystemUserProcesses::getColumnsDescription() return description; } -void StorageSystemUserProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemUserProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { const auto user_info = context->getProcessList().getUserInfo(true); diff --git a/src/Storages/System/StorageSystemUserProcesses.h b/src/Storages/System/StorageSystemUserProcesses.h index 6eb12e30559..3141eae9662 100644 --- a/src/Storages/System/StorageSystemUserProcesses.h +++ b/src/Storages/System/StorageSystemUserProcesses.h @@ -11,7 +11,7 @@ class Context; /** Implements `processes` system table, which allows you to get information about the queries that are currently executing. */ -class StorageSystemUserProcesses final : public IStorageSystemOneBlock +class StorageSystemUserProcesses final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemUserProcesses"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index cf114a85645..4734aeaaa82 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -77,7 +77,7 @@ ColumnsDescription StorageSystemUsers::getColumnsDescription() } -void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { /// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery. const auto & access_control = context->getAccessControl(); diff --git a/src/Storages/System/StorageSystemUsers.h b/src/Storages/System/StorageSystemUsers.h index cfa5947d370..a1a3d717b35 100644 --- a/src/Storages/System/StorageSystemUsers.h +++ b/src/Storages/System/StorageSystemUsers.h @@ -8,7 +8,7 @@ namespace DB class Context; /// Implements `users` system table, which allows you to get information about users. -class StorageSystemUsers final : public IStorageSystemOneBlock +class StorageSystemUsers final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemUsers"; } @@ -19,7 +19,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index d2b933e65a8..a0ed1d3c3d3 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -49,7 +49,7 @@ ColumnsDescription StorageSystemViewRefreshes::getColumnsDescription() } void StorageSystemViewRefreshes::fillData( - MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const + MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { auto access = context->getAccess(); auto valid_access = AccessType::SHOW_TABLES; diff --git a/src/Storages/System/StorageSystemViewRefreshes.h b/src/Storages/System/StorageSystemViewRefreshes.h index 02d3a39dfff..5a29f3a3bc8 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.h +++ b/src/Storages/System/StorageSystemViewRefreshes.h @@ -10,7 +10,7 @@ namespace DB { -class StorageSystemViewRefreshes final : public IStorageSystemOneBlock +class StorageSystemViewRefreshes final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemViewRefreshes"; } @@ -20,7 +20,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemWarnings.cpp b/src/Storages/System/StorageSystemWarnings.cpp index e2579372b4d..01e96e980a8 100644 --- a/src/Storages/System/StorageSystemWarnings.cpp +++ b/src/Storages/System/StorageSystemWarnings.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include namespace DB @@ -13,7 +15,7 @@ ColumnsDescription StorageSystemWarnings::getColumnsDescription() }; } -void StorageSystemWarnings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemWarnings::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { for (const auto & warning : context->getWarnings()) res_columns[0]->insert(warning); diff --git a/src/Storages/System/StorageSystemWarnings.h b/src/Storages/System/StorageSystemWarnings.h index 42948a765ea..685fb60b430 100644 --- a/src/Storages/System/StorageSystemWarnings.h +++ b/src/Storages/System/StorageSystemWarnings.h @@ -11,7 +11,7 @@ class Context; /** Implements system.warnings table that contains warnings about server configuration * to be displayed in clickhouse-client. */ -class StorageSystemWarnings final : public IStorageSystemOneBlock +class StorageSystemWarnings final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemWarnings"; } @@ -21,6 +21,6 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index c165bfa217d..8041370ee92 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -39,7 +39,7 @@ ColumnsDescription StorageSystemZooKeeperConnection::getColumnsDescription() } void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, ContextPtr context, - const SelectQueryInfo &) const + const ActionsDAG::Node *, std::vector) const { const auto add_enabled_feature_flags = [&](const auto & zookeeper) { diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.h b/src/Storages/System/StorageSystemZooKeeperConnection.h index 2b6d3b2e516..f8263e1f1bc 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.h +++ b/src/Storages/System/StorageSystemZooKeeperConnection.h @@ -11,7 +11,7 @@ class Context; /** Implements `zookeeper_connection` system table, which allows you to get information about the connected zookeeper info. */ -class StorageSystemZooKeeperConnection final : public IStorageSystemOneBlock +class StorageSystemZooKeeperConnection final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemZooKeeperConnection"; } @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index bf898f57833..f995c21326d 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -117,14 +117,14 @@ namespace DB void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { - attach(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); - attach(context, system_database, "numbers", "Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.", false); - attach(context, system_database, "numbers_mt", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", true); - attach(context, system_database, "zeros", "Produces unlimited number of non-materialized zeros.", false); - attach(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true); + attachNoDescription(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); + attachNoDescription(context, system_database, "numbers", "Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.", false); + attachNoDescription(context, system_database, "numbers_mt", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", true); + attachNoDescription(context, system_database, "zeros", "Produces unlimited number of non-materialized zeros.", false); + attachNoDescription(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true); attach(context, system_database, "databases", "Lists all databases of the current server."); - attach(context, system_database, "tables", "Lists all tables of the current server."); - attach(context, system_database, "columns", "Lists all columns from all tables of the current server."); + attachNoDescription(context, system_database, "tables", "Lists all tables of the current server."); + attachNoDescription(context, system_database, "columns", "Lists all columns from all tables of the current server."); attach(context, system_database, "functions", "Contains a list of all available ordinary and aggregate functions with their descriptions."); attach(context, system_database, "events", "Contains profiling events and their current value."); attach(context, system_database, "settings", "Contains a list of all user-level settings (which can be modified in a scope of query or session), their current and default values along with descriptions."); @@ -158,43 +158,43 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "privileges", "Contains a list of all available privileges that could be granted to a user or role."); attach(context, system_database, "errors", "Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace."); attach(context, system_database, "warnings", "Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server."); - attach(context, system_database, "data_skipping_indices", "Contains all the information about all the data skipping indices in tables, similar to system.columns."); + attachNoDescription(context, system_database, "data_skipping_indices", "Contains all the information about all the data skipping indices in tables, similar to system.columns."); attach(context, system_database, "licenses", "Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources."); attach(context, system_database, "time_zones", "Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse."); attach(context, system_database, "backups", "Contains a list of all BACKUP or RESTORE operations with their current states and other propertis. Note, that table is not persistent and it shows only operations executed after the last server restart."); attach(context, system_database, "schema_inference_cache", "Contains information about all cached file schemas."); attach(context, system_database, "dropped_tables", "Contains a list of tables which were dropped from Atomic databases but not completely removed yet."); - attach(context, system_database, "dropped_tables_parts", "Contains parts of system.dropped_tables tables "); + attachNoDescription(context, system_database, "dropped_tables_parts", "Contains parts of system.dropped_tables tables "); attach(context, system_database, "scheduler", "Contains information and status for scheduling nodes residing on the local server."); #if defined(__ELF__) && !defined(OS_FREEBSD) - attach(context, system_database, "symbols", "Contains information for introspection of ClickHouse binary. This table is only useful for C++ experts and ClickHouse engineers."); + attachNoDescription(context, system_database, "symbols", "Contains information for introspection of ClickHouse binary. This table is only useful for C++ experts and ClickHouse engineers."); #endif #if USE_RDKAFKA attach(context, system_database, "kafka_consumers", "Contains information about Kafka consumers. Applicable for Kafka table engine (native ClickHouse integration)."); #endif #ifdef OS_LINUX - attach(context, system_database, "stack_trace", "Allows to obtain an unsymbolized stacktrace from all the threads of the server process."); + attachNoDescription(context, system_database, "stack_trace", "Allows to obtain an unsymbolized stacktrace from all the threads of the server process."); #endif #if USE_ROCKSDB attach(context, system_database, "rocksdb", "Contains a list of metrics exposed from embedded RocksDB."); #endif #if USE_MYSQL - attach(context, system_database, "mysql_binlogs", "Shows a list of active binlogs for MaterializedMySQL."); + attachNoDescription(context, system_database, "mysql_binlogs", "Shows a list of active binlogs for MaterializedMySQL."); #endif - attach(context, system_database, "parts", "Contains a list of currently existing (both active and inactive) parts of all *-MergeTree tables. Each part is represented by a single row."); - attach(context, system_database, "projection_parts", "Contains a list of currently existing projection parts (a copy of some part containing aggregated data or just sorted in different order) created for all the projections for all tables within a cluster."); - attach(context, system_database, "detached_parts", "Contains a list of all parts which are being found in /detached directory along with a reason why it was detached. ClickHouse server doesn't use such parts anyhow."); - attach(context, system_database, "parts_columns", "Contains a list of columns of all currently existing parts of all MergeTree tables. Each column is represented by a single row."); - attach(context, system_database, "projection_parts_columns", "Contains a list of columns of all currently existing projection parts of all MergeTree tables. Each column is represented by a single row."); - attach(context, system_database, "disks", "Contains information about disks defined in the server configuration."); - attach(context, system_database, "storage_policies", "Contains information about storage policies and volumes defined in the server configuration."); + attachNoDescription(context, system_database, "parts", "Contains a list of currently existing (both active and inactive) parts of all *-MergeTree tables. Each part is represented by a single row."); + attachNoDescription(context, system_database, "projection_parts", "Contains a list of currently existing projection parts (a copy of some part containing aggregated data or just sorted in different order) created for all the projections for all tables within a cluster."); + attachNoDescription(context, system_database, "detached_parts", "Contains a list of all parts which are being found in /detached directory along with a reason why it was detached. ClickHouse server doesn't use such parts anyhow."); + attachNoDescription(context, system_database, "parts_columns", "Contains a list of columns of all currently existing parts of all MergeTree tables. Each column is represented by a single row."); + attachNoDescription(context, system_database, "projection_parts_columns", "Contains a list of columns of all currently existing projection parts of all MergeTree tables. Each column is represented by a single row."); + attachNoDescription(context, system_database, "disks", "Contains information about disks defined in the server configuration."); + attachNoDescription(context, system_database, "storage_policies", "Contains information about storage policies and volumes defined in the server configuration."); attach(context, system_database, "processes", "Contains a list of currently executing processes (queries) with their progress."); attach(context, system_database, "metrics", "Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date."); attach(context, system_database, "merges", "Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row."); attach(context, system_database, "moves", "Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row."); attach(context, system_database, "mutations", "Contains a list of mutations and their progress. Each mutation command is represented by a single row."); - attach(context, system_database, "replicas", "Contains information and status of all table replicas on current server. Each replica is represented by a single row."); + attachNoDescription(context, system_database, "replicas", "Contains information and status of all table replicas on current server. Each replica is represented by a single row."); attach(context, system_database, "replication_queue", "Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica."); attach(context, system_database, "distributed_ddl_queue", "Contains information about distributed DDL queries (ON CLUSTER clause) that were executed on a cluster."); attach(context, system_database, "distribution_queue", "Contains information about local files that are in the queue to be sent to the shards. These local files contain new parts that are created by inserting new data into the Distributed table in asynchronous mode."); @@ -206,21 +206,21 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "replicated_fetches", "Contains information about currently running background fetches."); attach(context, system_database, "part_moves_between_shards", "Contains information about parts which are currently in a process of moving between shards and their progress."); attach(context, system_database, "asynchronous_inserts", "Contains information about pending asynchronous inserts in queue in server's memory."); - attach(context, system_database, "filesystem_cache", "Contains information about all entries inside filesystem cache for remote objects."); - attach(context, system_database, "query_cache", "Contains information about all entries inside query cache in server's memory."); - attach(context, system_database, "remote_data_paths", "Contains a mapping from a filename on local filesystem to a blob name inside object storage."); + attachNoDescription(context, system_database, "filesystem_cache", "Contains information about all entries inside filesystem cache for remote objects."); + attachNoDescription(context, system_database, "query_cache", "Contains information about all entries inside query cache in server's memory."); + attachNoDescription(context, system_database, "remote_data_paths", "Contains a mapping from a filename on local filesystem to a blob name inside object storage."); attach(context, system_database, "certificates", "Contains information about available certificates and their sources."); - attach(context, system_database, "named_collections", "Contains a list of all named collections which were created via SQL query or parsed from configuration file."); + attachNoDescription(context, system_database, "named_collections", "Contains a list of all named collections which were created via SQL query or parsed from configuration file."); attach(context, system_database, "asynchronous_loader", "Contains information and status for recent asynchronous jobs (e.g. for tables loading). The table contains a row for every job."); attach(context, system_database, "user_processes", "This system table can be used to get overview of memory usage and ProfileEvents of users."); - attach(context, system_database, "jemalloc_bins", "Contains information about memory allocations done via jemalloc allocator in different size classes (bins) aggregated from all arenas. These statistics might not be absolutely accurate because of thread local caching in jemalloc."); - attach(context, system_database, "s3queue", "Contains in-memory state of S3Queue metadata and currently processed rows per file."); + attachNoDescription(context, system_database, "jemalloc_bins", "Contains information about memory allocations done via jemalloc allocator in different size classes (bins) aggregated from all arenas. These statistics might not be absolutely accurate because of thread local caching in jemalloc."); + attachNoDescription(context, system_database, "s3queue", "Contains in-memory state of S3Queue metadata and currently processed rows per file."); attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); if (has_zookeeper) { - attach(context, system_database, "zookeeper", "Exposes data from the [Zoo]Keeper cluster defined in the config. Allow to get the list of children for a particular node or read the value written inside it."); + attachNoDescription(context, system_database, "zookeeper", "Exposes data from the [Zoo]Keeper cluster defined in the config. Allow to get the list of children for a particular node or read the value written inside it."); attach(context, system_database, "zookeeper_connection", "Shows the information about current connections to [Zoo]Keeper (including auxiliary [ZooKeepers)"); } @@ -230,7 +230,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b void attachSystemTablesAsync(ContextPtr context, IDatabase & system_database, AsynchronousMetrics & async_metrics) { - attach(context, system_database, "asynchronous_metrics", "Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use.", async_metrics); + attachNoDescription(context, system_database, "asynchronous_metrics", "Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use.", async_metrics); } } diff --git a/src/Storages/System/attachSystemTablesImpl.h b/src/Storages/System/attachSystemTablesImpl.h index 9f2c4e8016d..d9ab164f2b3 100644 --- a/src/Storages/System/attachSystemTablesImpl.h +++ b/src/Storages/System/attachSystemTablesImpl.h @@ -10,8 +10,8 @@ namespace DB template using StringLiteral = const char(&)[Length]; -template -void attach(ContextPtr context, IDatabase & system_database, const String & table_name, StringLiteral comment, StorageArgs && ... args) +template +void attachImpl(ContextPtr context, IDatabase & system_database, const String & table_name, StringLiteral comment, StorageArgs && ... args) { static_assert(CommentSize > 15, "The comment for a system table is too short or empty"); assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE); @@ -21,7 +21,10 @@ void attach(ContextPtr context, IDatabase & system_database, const String & tabl { /// Attach to Ordinary database. table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); - system_database.attachTable(context, table_name, std::make_shared(table_id, std::forward(args)...)); + if constexpr (with_description) + system_database.attachTable(context, table_name, std::make_shared(table_id, StorageT::getColumnsDescription(), std::forward(args)...)); + else + system_database.attachTable(context, table_name, std::make_shared(table_id, std::forward(args)...)); } else { @@ -31,7 +34,10 @@ void attach(ContextPtr context, IDatabase & system_database, const String & tabl table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); DatabaseCatalog::instance().addUUIDMapping(table_id.uuid); String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); - system_database.attachTable(context, table_name, std::make_shared(table_id, std::forward(args)...), path); + if constexpr (with_description) + system_database.attachTable(context, table_name, std::make_shared(table_id, StorageT::getColumnsDescription(), std::forward(args)...), path); + else + system_database.attachTable(context, table_name, std::make_shared(table_id, std::forward(args)...), path); } /// Set the comment @@ -42,4 +48,17 @@ void attach(ContextPtr context, IDatabase & system_database, const String & tabl table->setInMemoryMetadata(metadata); } + +template +void attach(ContextPtr context, IDatabase & system_database, const String & table_name, StringLiteral comment, StorageArgs && ... args) +{ + attachImpl(context, system_database, table_name, comment, std::forward(args)...); +} + +template +void attachNoDescription(ContextPtr context, IDatabase & system_database, const String & table_name, StringLiteral comment, StorageArgs && ... args) +{ + attachImpl(context, system_database, table_name, comment, std::forward(args)...); +} + } From 0ceeb13b7ff58752b199482d3683a381d4af4b00 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 28 Feb 2024 19:58:55 +0000 Subject: [PATCH 055/197] refactoring of virtual columns --- src/Interpreters/InterpreterCreateQuery.cpp | 3 +- src/Interpreters/MutationsInterpreter.cpp | 194 ++++-------------- src/Interpreters/TreeRewriter.cpp | 49 ++--- src/Storages/AlterCommands.cpp | 46 ++--- src/Storages/IStorage.cpp | 2 +- src/Storages/LightweightDeleteDescription.cpp | 9 - src/Storages/LightweightDeleteDescription.h | 13 -- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 + src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +- src/Storages/MergeTree/MergeTask.cpp | 17 +- src/Storages/MergeTree/MergeTreeData.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.h | 4 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 6 - .../MergeTree/MergeTreePrefetchedReadPool.cpp | 3 +- src/Storages/MergeTree/MergeTreeReadTask.cpp | 3 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 2 +- .../MergeTree/MergeTreeSequentialSource.cpp | 10 +- src/Storages/MergeTree/MutateTask.cpp | 13 +- src/Storages/StorageDistributed.cpp | 28 ++- src/Storages/StorageDistributed.h | 5 +- src/Storages/StorageMerge.cpp | 53 +++-- src/Storages/StorageMerge.h | 5 +- src/Storages/StorageSnapshot.cpp | 11 +- src/Storages/StorageSnapshot.h | 5 + .../01848_partition_value_column.sql | 4 +- 25 files changed, 195 insertions(+), 303 deletions(-) delete mode 100644 src/Storages/LightweightDeleteDescription.cpp delete mode 100644 src/Storages/LightweightDeleteDescription.h diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 579bca216e9..a1b63960d40 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -907,8 +907,7 @@ void validateVirtualColumns(const IStorage & storage) auto virtual_columns = storage.getVirtualsDescription(); for (const auto & storage_column : storage.getInMemoryMetadataPtr()->getColumns()) { - auto virtual_desc = virtual_columns->tryGetDescription(storage_column.name); - if (virtual_desc && virtual_desc->kind == VirtualsKind::Persistent) + if (virtual_columns->tryGet(storage_column.name, VirtualsKind::Persistent)) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column '{}' for {} engines because it is reserved for persistent virtual column", diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e88dc121a82..8ac25d13bf0 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -31,7 +31,6 @@ #include #include #include -#include #include #include #include @@ -265,7 +264,7 @@ MutationCommand createCommandToApplyDeletedMask(const MutationCommand & command) alter_command->partition = alter_command->children.emplace_back(command.partition).get(); auto row_exists_predicate = makeASTFunction("equals", - std::make_shared(LightweightDeleteDescription::FILTER_COLUMN.name), + std::make_shared(RowExistsColumn::name), std::make_shared(Field(0))); if (command.predicate) @@ -435,60 +434,54 @@ static NameSet getKeyColumns(const MutationsInterpreter::Source & source, const static void validateUpdateColumns( const MutationsInterpreter::Source & source, - const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns, - const std::unordered_map & column_to_affected_materialized) + const StorageMetadataPtr & metadata_snapshot, + const NameSet & updated_columns, + const std::unordered_map & column_to_affected_materialized, + const ContextPtr & context) { + auto storage_snapshot = source.getStorageSnapshot(metadata_snapshot, context); NameSet key_columns = getKeyColumns(source, metadata_snapshot); - for (const String & column_name : updated_columns) + const auto & storage_columns = storage_snapshot->metadata->getColumns(); + const auto & virtual_columns = *storage_snapshot->virtual_columns; + + for (const auto & column_name : updated_columns) { - auto found = false; - for (const auto & col : metadata_snapshot->getColumns().getOrdinary()) - { - if (col.name == column_name) - { - found = true; - break; - } - } - - /// Allow to override value of lightweight delete filter virtual column - if (!found && column_name == LightweightDeleteDescription::FILTER_COLUMN.name) - { - if (!source.supportsLightweightDelete()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); - found = true; - } - - /// Dont allow to override value of block number virtual column - if (!found && column_name == BlockNumberColumn::name) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Update is not supported for virtual column {} ", backQuote(column_name)); - } - - if (!found) - { - for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) - { - if (col.name == column_name) - throw Exception(ErrorCodes::CANNOT_UPDATE_COLUMN, "Cannot UPDATE materialized column {}", backQuote(column_name)); - } - - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", backQuote(column_name)); - } - if (key_columns.contains(column_name)) throw Exception(ErrorCodes::CANNOT_UPDATE_COLUMN, "Cannot UPDATE key column {}", backQuote(column_name)); + if (storage_columns.tryGetColumn(GetColumnsOptions::Materialized, column_name)) + throw Exception(ErrorCodes::CANNOT_UPDATE_COLUMN, "Cannot UPDATE materialized column {}", backQuote(column_name)); + auto materialized_it = column_to_affected_materialized.find(column_name); if (materialized_it != column_to_affected_materialized.end()) { - for (const String & materialized : materialized_it->second) + for (const auto & materialized : materialized_it->second) { if (key_columns.contains(materialized)) + { throw Exception(ErrorCodes::CANNOT_UPDATE_COLUMN, "Updated column {} affects MATERIALIZED column {}, which is a key column. " "Cannot UPDATE it.", backQuote(column_name), backQuote(materialized)); + } + } + } + + if (!storage_columns.tryGetColumn(GetColumnsOptions::Ordinary, column_name)) + { + /// Allow to override value of lightweight delete filter virtual column + if (column_name == RowExistsColumn::name) + { + if (!source.supportsLightweightDelete()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); + } + else if (virtual_columns.tryGet(column_name)) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Update is not supported for virtual column {} ", backQuote(column_name)); + } + else + { + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", backQuote(column_name)); } } } @@ -546,8 +539,8 @@ void MutationsInterpreter::prepare(bool dry_run) /// Add _row_exists column if it is physically present in the part if (source.hasLightweightDeleteMask()) { - all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); - available_columns_set.insert(LightweightDeleteDescription::FILTER_COLUMN.name); + all_columns.emplace_back(RowExistsColumn::name, RowExistsColumn::type); + available_columns_set.insert(RowExistsColumn::name); } NameSet updated_columns; @@ -563,9 +556,7 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & [name, _] : command.column_to_update_expression) { - if (!available_columns_set.contains(name) - && name != LightweightDeleteDescription::FILTER_COLUMN.name - && name != BlockNumberColumn::name) + if (!available_columns_set.contains(name) && name != RowExistsColumn::name) throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is updated but not requested to read", name); @@ -590,7 +581,7 @@ void MutationsInterpreter::prepare(bool dry_run) } } - validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); + validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized, context); } StorageInMemoryMetadata::HasDependencyCallback has_dependency = @@ -666,15 +657,11 @@ void MutationsInterpreter::prepare(bool dry_run) { type = physical_column->type; } - else if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name) + else if (column_name == RowExistsColumn::name) { - type = LightweightDeleteDescription::FILTER_COLUMN.type; + type = RowExistsColumn::type; deleted_mask_updated = true; } - else if (column_name == BlockNumberColumn::name) - { - type = BlockNumberColumn::type; - } else { throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column {}", column_name); @@ -1028,7 +1015,7 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s /// Add _row_exists column if it is present in the part if (source.hasLightweightDeleteMask() || deleted_mask_updated) - all_columns.push_back(LightweightDeleteDescription::FILTER_COLUMN); + all_columns.emplace_back(RowExistsColumn::name, RowExistsColumn::type); bool has_filters = false; /// Next, for each stage calculate columns changed by this and previous stages. @@ -1038,7 +1025,7 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s { for (const auto & column : all_columns) { - if (column.name == LightweightDeleteDescription::FILTER_COLUMN.name && !deleted_mask_updated) + if (column.name == RowExistsColumn::name && !deleted_mask_updated) continue; prepared_stages[i].output_columns.insert(column.name); @@ -1057,7 +1044,7 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s /// and so it is not in the list of AllPhysical columns. for (const auto & [column_name, _] : prepared_stages[i].column_to_updated) { - if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name && has_filters && !deleted_mask_updated) + if (column_name == RowExistsColumn::name && has_filters && !deleted_mask_updated) continue; prepared_stages[i].output_columns.insert(column_name); @@ -1148,93 +1135,6 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s } } -/// This structure re-implements adding virtual columns while reading from MergeTree part. -/// It would be good to unify it with IMergeTreeSelectAlgorithm. -struct VirtualColumns -{ - struct ColumnAndPosition - { - ColumnWithTypeAndName column; - size_t position; - }; - - using Columns = std::vector; - - Columns virtuals; - Names columns_to_read; - - VirtualColumns(Names required_columns, const MergeTreeData::DataPartPtr & part) : columns_to_read(std::move(required_columns)) - { - for (size_t i = 0; i < columns_to_read.size(); ++i) - { - if (columns_to_read[i] == LightweightDeleteDescription::FILTER_COLUMN.name) - { - if (!part->getColumns().contains(LightweightDeleteDescription::FILTER_COLUMN.name)) - { - ColumnWithTypeAndName mask_column; - mask_column.type = LightweightDeleteDescription::FILTER_COLUMN.type; - mask_column.column = mask_column.type->createColumnConst(0, 1); - mask_column.name = std::move(columns_to_read[i]); - - virtuals.emplace_back(ColumnAndPosition{.column = std::move(mask_column), .position = i}); - } - } - else if (columns_to_read[i] == "_partition_id") - { - ColumnWithTypeAndName column; - column.type = std::make_shared(); - column.column = column.type->createColumnConst(0, part->info.partition_id); - column.name = std::move(columns_to_read[i]); - - virtuals.emplace_back(ColumnAndPosition{.column = std::move(column), .position = i}); - } - else if (columns_to_read[i] == BlockNumberColumn::name) - { - if (!part->getColumns().contains(BlockNumberColumn::name)) - { - ColumnWithTypeAndName block_number_column; - block_number_column.type = BlockNumberColumn::type; - block_number_column.column = block_number_column.type->createColumnConst(0, part->info.min_block); - block_number_column.name = std::move(columns_to_read[i]); - - virtuals.emplace_back(ColumnAndPosition{.column = std::move(block_number_column), .position = i}); - } - } - } - - if (!virtuals.empty()) - { - Names columns_no_virtuals; - columns_no_virtuals.reserve(columns_to_read.size()); - size_t next_virtual = 0; - for (size_t i = 0; i < columns_to_read.size(); ++i) - { - if (next_virtual < virtuals.size() && i == virtuals[next_virtual].position) - ++next_virtual; - else - columns_no_virtuals.emplace_back(std::move(columns_to_read[i])); - } - - columns_to_read.swap(columns_no_virtuals); - } - } - - void addVirtuals(QueryPlan & plan) - { - auto dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - - for (auto & column : virtuals) - { - const auto & adding_const = dag->addColumn(std::move(column.column)); - auto & outputs = dag->getOutputs(); - outputs.insert(outputs.begin() + column.position, &adding_const); - } - - auto step = std::make_unique(plan.getCurrentDataStream(), std::move(dag)); - plan.addStep(std::move(step)); - } -}; - void MutationsInterpreter::Source::read( Stage & first_stage, QueryPlan & plan, @@ -1277,16 +1177,12 @@ void MutationsInterpreter::Source::read( filter = ActionsDAG::buildFilterActionsDAG(nodes); } - VirtualColumns virtual_columns(std::move(required_columns), part); - createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, - plan, *data, storage_snapshot, part, - std::move(virtual_columns.columns_to_read), + plan, *data, storage_snapshot, + part, required_columns, apply_deleted_mask_, filter, context_, getLogger("MutationsInterpreter")); - - virtual_columns.addVirtuals(plan); } else { diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 960fad5dec6..1a32b885f4d 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -56,6 +56,7 @@ #include #include #include +#include #include #include @@ -990,8 +991,8 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) { auto options = GetColumnsOptions(add_special ? GetColumnsOptions::All : GetColumnsOptions::AllPhysical); options.withExtendedObjects(); - if (storage->supportsSubcolumns()) - options.withSubcolumns(); + options.withSubcolumns(storage->supportsSubcolumns()); + options.withVirtuals(); auto columns_from_storage = storage_snapshot->getColumns(options); @@ -1109,16 +1110,16 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select const auto & partition_desc = storage_snapshot->metadata->getPartitionKey(); if (partition_desc.expression) { - auto partition_source_columns = partition_desc.expression->getRequiredColumns(); - partition_source_columns.push_back("_part"); - partition_source_columns.push_back("_partition_id"); - partition_source_columns.push_back("_part_uuid"); - partition_source_columns.push_back("_partition_value"); + auto partition_columns = partition_desc.expression->getRequiredColumns(); + NameSet partition_columns_set(partition_columns.begin(), partition_columns.end()); + + const auto & parititon_virtuals = MergeTreeData::virtuals_useful_for_filter; + partition_columns_set.insert(parititon_virtuals.begin(), parititon_virtuals.end()); + optimize_trivial_count = true; for (const auto & required_column : required) { - if (std::find(partition_source_columns.begin(), partition_source_columns.end(), required_column) - == partition_source_columns.end()) + if (partition_columns_set.contains(required_column)) { optimize_trivial_count = false; break; @@ -1129,7 +1130,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select NameSet unknown_required_source_columns = required; - for (NamesAndTypesList::iterator it = source_columns.begin(); it != source_columns.end();) + for (auto it = source_columns.begin(); it != source_columns.end();) { const String & column_name = it->name; unknown_required_source_columns.erase(column_name); @@ -1141,32 +1142,14 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select } has_virtual_shard_num = false; - /// If there are virtual columns among the unknown columns. Remove them from the list of unknown and add - /// in columns list, so that when further processing they are also considered. - if (storage) + if (is_remote_storage) { - const auto storage_virtuals = storage->getVirtuals(); - for (auto it = unknown_required_source_columns.begin(); it != unknown_required_source_columns.end();) + for (const auto & column : *storage_snapshot->virtual_columns) { - auto column = storage_virtuals.tryGetByName(*it); - if (column) + if (column.name == "_shard_num" && storage->isVirtualColumn("_shard_num", storage_snapshot->getMetadataForQuery())) { - source_columns.push_back(*column); - it = unknown_required_source_columns.erase(it); - } - else - ++it; - } - - if (is_remote_storage) - { - for (const auto & name_type : storage_virtuals) - { - if (name_type.name == "_shard_num" && storage->isVirtualColumn("_shard_num", storage_snapshot->getMetadataForQuery())) - { - has_virtual_shard_num = true; - break; - } + has_virtual_shard_num = true; + break; } } } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 792f942fcf1..acd95a2b8d7 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -31,7 +31,6 @@ #include #include #include -#include #include #include #include @@ -955,8 +954,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada /// Drop alias is metadata alter, in other case mutation is required. if (type == DROP_COLUMN) - return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name) || - column_name == LightweightDeleteDescription::FILTER_COLUMN.name || column_name == BlockNumberColumn::name; + return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name); if (type != MODIFY_COLUMN || data_type == nullptr) return false; @@ -1246,7 +1244,9 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata) void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { - const StorageInMemoryMetadata & metadata = table->getInMemoryMetadata(); + const auto & metadata = table->getInMemoryMetadata(); + const auto & virtuals = *table->getVirtualsDescription(); + auto all_columns = metadata.columns; /// Default expression for all added/modified columns ASTPtr default_expr_list = std::make_shared(); @@ -1282,16 +1282,20 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const if (command.data_type->hasDynamicSubcolumns()) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Adding a new column of a type which has dynamic subcolumns to an existing table is not allowed. It has known bugs"); - if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast(table)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: " - "this column name is reserved for lightweight delete feature", backQuote(column_name)); - - if (column_name == BlockNumberColumn::name && std::dynamic_pointer_cast(table)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: " - "this column name is reserved for _block_number persisting feature", backQuote(column_name)); + if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot add column {}: this column name is reserved for persistent virtual column", backQuote(column_name)); if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec, context->getSettingsRef().enable_zstd_qat_codec); + { + const auto & settings = context->getSettingsRef(); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( + command.codec, command.data_type, + !settings.allow_suspicious_codecs, + settings.allow_experimental_codecs, + settings.enable_deflate_qpl_codec, + settings.enable_zstd_qat_codec); + } all_columns.add(ColumnDescription(column_name, command.data_type)); } @@ -1405,9 +1409,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const } else if (command.type == AlterCommand::DROP_COLUMN) { - if (all_columns.has(command.column_name) || - all_columns.hasNested(command.column_name) || - (command.clear && column_name == LightweightDeleteDescription::FILTER_COLUMN.name)) + if (all_columns.has(command.column_name) || all_columns.hasNested(command.column_name)) { if (!command.clear) /// CLEAR column is Ok even if there are dependencies. { @@ -1491,16 +1493,12 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const } if (all_columns.has(command.rename_to)) - throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Cannot rename to {}: " - "column with this name already exists", backQuote(command.rename_to)); + throw Exception(ErrorCodes::DUPLICATE_COLUMN, + "Cannot rename to {}: column with this name already exists", backQuote(command.rename_to)); - if (command.rename_to == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast(table)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: " - "this column name is reserved for lightweight delete feature", backQuote(command.rename_to)); - - if (command.rename_to == BlockNumberColumn::name && std::dynamic_pointer_cast(table)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: " - "this column name is reserved for _block_number persisting feature", backQuote(command.rename_to)); + if (virtuals.tryGet(command.rename_to, VirtualsKind::Persistent)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot rename to {}: this column name is reserved for persistent virtual column", backQuote(command.rename_to)); if (modified_columns.contains(column_name)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rename and modify the same column {} " diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 27593bfe7a8..18a934af767 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -38,7 +38,7 @@ IStorage::IStorage(StorageID storage_id_) bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const { /// Virtual column maybe overridden by real column - return !metadata_snapshot->getColumns().has(column_name) && getVirtuals().contains(column_name); + return !metadata_snapshot->getColumns().has(column_name) && virtuals.get()->has(column_name); } RWLockImpl::LockHolder IStorage::tryLockTimed( diff --git a/src/Storages/LightweightDeleteDescription.cpp b/src/Storages/LightweightDeleteDescription.cpp deleted file mode 100644 index ae5e68da9c2..00000000000 --- a/src/Storages/LightweightDeleteDescription.cpp +++ /dev/null @@ -1,9 +0,0 @@ -#include -#include - -namespace DB -{ - -const NameAndTypePair LightweightDeleteDescription::FILTER_COLUMN {"_row_exists", std::make_shared()}; - -} diff --git a/src/Storages/LightweightDeleteDescription.h b/src/Storages/LightweightDeleteDescription.h deleted file mode 100644 index 45bde59ea71..00000000000 --- a/src/Storages/LightweightDeleteDescription.h +++ /dev/null @@ -1,13 +0,0 @@ -#pragma once -#include -#include "Storages/TTLDescription.h" - -namespace DB -{ - -struct LightweightDeleteDescription -{ - static const NameAndTypePair FILTER_COLUMN; -}; - -} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index e06ea5e560c..fe4ba5fc052 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1451,6 +1451,11 @@ bool IMergeTreeDataPart::supportLightweightDeleteMutate() const parent_part == nullptr && projection_parts.empty(); } +bool IMergeTreeDataPart::hasLightweightDelete() const +{ + return columns.contains(RowExistsColumn::name); +} + void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const { TransactionID expected_tid = txn ? txn->tid : Tx::PrehistoricTID; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index d0eafc42ddc..fba1e6ddbb1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -498,7 +497,7 @@ public: bool supportLightweightDeleteMutate() const; /// True if here is lightweight deleted mask file in part. - bool hasLightweightDelete() const { return columns.contains(LightweightDeleteDescription::FILTER_COLUMN.name); } + bool hasLightweightDelete() const; void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index df64ae33713..4621314cb98 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include @@ -1075,14 +1074,18 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() if (global_ctx->deduplicate) { - /// We don't want to deduplicate by block number column - /// so if deduplicate_by_columns is empty, add all columns except _block_number - if (supportsBlockNumberColumn(global_ctx) && global_ctx->deduplicate_by_columns.empty()) + const auto & virtuals = *global_ctx->data->getVirtualsDescription(); + + /// We don't want to deduplicate by virtual persistent column. + /// If deduplicate_by_columns is empty, add all columns except virtuals. + if (global_ctx->deduplicate_by_columns.empty()) { - for (const auto & col : global_ctx->merging_column_names) + for (const auto & column_name : global_ctx->merging_column_names) { - if (col != BlockNumberColumn::name) - global_ctx->deduplicate_by_columns.emplace_back(col); + if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) + continue; + + global_ctx->deduplicate_by_columns.emplace_back(column_name); } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 539532bfaca..cb319348b60 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -450,7 +450,7 @@ VirtualColumnsDescription MergeTreeData::createVirtuals(const StorageInMemoryMet desc.addEphemeral("_partition_value", std::make_shared(std::move(partition_types)), ""); } - desc.addPersistent(LightweightDeleteDescription::FILTER_COLUMN.name, LightweightDeleteDescription::FILTER_COLUMN.type, nullptr, ""); + desc.addPersistent(RowExistsColumn::name, RowExistsColumn::type, nullptr, ""); desc.addPersistent(BlockNumberColumn::name, BlockNumberColumn::type, BlockNumberColumn::codec, ""); return desc; @@ -3651,6 +3651,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts { auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & columns = metadata_snapshot->getColumns(); + const auto & virtuals = *getVirtualsDescription(); if (!hasDynamicSubcolumns(columns)) return; @@ -3658,7 +3659,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts const auto & part_columns = part->getColumns(); for (const auto & part_column : part_columns) { - if (part_column.name == LightweightDeleteDescription::FILTER_COLUMN.name || part_column.name == BlockNumberColumn::name) + if (virtuals.has(part_column.name)) continue; auto storage_column = columns.getPhysical(part_column.name); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 80ab1f337ee..f7bde252fb9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1085,6 +1085,8 @@ public: bool initializeDiskOnConfigChange(const std::set & /*new_added_disks*/) override; + static VirtualColumnsDescription createVirtuals(const StorageInMemoryMetadata & metadata); + protected: friend class IMergeTreeDataPart; friend class MergeTreeDataMergerMutator; @@ -1675,8 +1677,6 @@ private: void checkColumnFilenamesForCollision(const StorageInMemoryMetadata & metadata, bool throw_on_error) const; void checkColumnFilenamesForCollision(const ColumnsDescription & columns, const MergeTreeSettings & settings, bool throw_on_error) const; - - static VirtualColumnsDescription createVirtuals(const StorageInMemoryMetadata & metadata); }; /// RAII struct to record big parts that are submerging or emerging. diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 8e3f2e07684..ee34a02b0b3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -76,12 +76,6 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( ordered_columns_list.sort([this](const auto & lhs, const auto & rhs) { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); - /// _block_number column is not added by user, but is persisted in a part after merge - /// If _block_number is not present in the parts to be merged, then it won't have a position - /// So check if its not present and add it at the end - if (columns_list.contains(BlockNumberColumn::name) && !ordered_columns_list.contains(BlockNumberColumn::name)) - ordered_columns_list.emplace_back(NameAndTypePair{BlockNumberColumn::name, BlockNumberColumn::type}); - return std::make_unique( shared_from_this(), ordered_columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, getMarksFileExtension(), diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 7cb3d6012d5..059caebcfc8 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -375,7 +376,7 @@ void MergeTreePrefetchedReadPool::fillPerPartStatistics() update_stat_for_column(column.name); if (reader_settings.apply_deleted_mask && read_info.data_part->hasLightweightDelete()) - update_stat_for_column(LightweightDeleteDescription::FILTER_COLUMN.name); + update_stat_for_column(RowExistsColumn::name); for (const auto & pre_columns : read_info.task_columns.pre_columns) for (const auto & column : pre_columns) diff --git a/src/Storages/MergeTree/MergeTreeReadTask.cpp b/src/Storages/MergeTree/MergeTreeReadTask.cpp index 7bbabf6a18d..64fd37d14b1 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.cpp +++ b/src/Storages/MergeTree/MergeTreeReadTask.cpp @@ -1,5 +1,6 @@ #include #include +#include #include namespace DB @@ -57,7 +58,7 @@ MergeTreeReadTask::Readers MergeTreeReadTask::createReaders( /// Add lightweight delete filtering step if (extras.reader_settings.apply_deleted_mask && read_info->data_part->hasLightweightDelete()) - new_readers.prewhere.push_back(create_reader({LightweightDeleteDescription::FILTER_COLUMN})); + new_readers.prewhere.push_back(create_reader({{RowExistsColumn::name, RowExistsColumn::type}})); for (const auto & pre_columns_per_step : read_info->task_columns.pre_columns) new_readers.prewhere.push_back(create_reader(pre_columns_per_step)); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 6b0c45c2431..e3900ccdd73 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -48,7 +48,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( { .type = PrewhereExprStep::Filter, .actions = nullptr, - .filter_column_name = LightweightDeleteDescription::FILTER_COLUMN.name, + .filter_column_name = RowExistsColumn::name, .remove_filter_column = true, .need_filter = true, .perform_alter_conversions = true, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 35f5782b95a..fb0bc617aa4 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -136,7 +136,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( { auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) .withExtendedObjects() - .withVirtuals(VirtualsKind::Persistent) + .withVirtuals() .withSubcolumns(storage.supportsSubcolumns()); columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read); @@ -242,6 +242,7 @@ try if (rows_read) { fillBlockNumberColumns(columns, sample, data_part->info.min_block, current_row, rows_read); + reader->fillVirtualColumns(columns, rows_read); current_row += rows_read; current_mark += (rows_to_read == rows_read); @@ -315,14 +316,13 @@ Pipe createMergeTreeSequentialSource( bool quiet, std::shared_ptr> filtered_rows_count) { - const auto & filter_column = LightweightDeleteDescription::FILTER_COLUMN; /// The part might have some rows masked by lightweight deletes const bool need_to_filter_deleted_rows = apply_deleted_mask && data_part->hasLightweightDelete(); - const bool has_filter_column = std::ranges::find(columns_to_read, filter_column.name) != columns_to_read.end(); + const bool has_filter_column = std::ranges::find(columns_to_read, RowExistsColumn::name) != columns_to_read.end(); if (need_to_filter_deleted_rows && !has_filter_column) - columns_to_read.emplace_back(filter_column.name); + columns_to_read.emplace_back(RowExistsColumn::name); auto column_part_source = std::make_shared(type, storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges), @@ -336,7 +336,7 @@ Pipe createMergeTreeSequentialSource( pipe.addSimpleTransform([filtered_rows_count, has_filter_column](const Block & header) { return std::make_shared( - header, nullptr, filter_column.name, !has_filter_column, false, filtered_rows_count); + header, nullptr, RowExistsColumn::name, !has_filter_column, false, filtered_rows_count); }); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 065a2d86296..b3c36f7180b 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -283,7 +283,6 @@ getColumnsForNewDataPart( ColumnsDescription part_columns(source_part->getColumns()); NamesAndTypesList system_columns; - const auto & deleted_mask_column = LightweightDeleteDescription::FILTER_COLUMN; bool supports_lightweight_deletes = source_part->supportLightweightDeleteMutate(); bool deleted_mask_updated = false; @@ -299,9 +298,9 @@ getColumnsForNewDataPart( { for (const auto & [column_name, _] : command.column_to_update_expression) { - if (column_name == deleted_mask_column.name + if (column_name == RowExistsColumn::name && supports_lightweight_deletes - && !storage_columns_set.contains(deleted_mask_column.name)) + && !storage_columns_set.contains(RowExistsColumn::name)) deleted_mask_updated = true; } } @@ -323,12 +322,12 @@ getColumnsForNewDataPart( } } - if (!storage_columns_set.contains(deleted_mask_column.name)) + if (!storage_columns_set.contains(RowExistsColumn::name)) { - if (deleted_mask_updated || (part_columns.has(deleted_mask_column.name) && !has_delete_command)) + if (deleted_mask_updated || (part_columns.has(RowExistsColumn::name) && !has_delete_command)) { - storage_columns.push_back(deleted_mask_column); - storage_columns_set.insert(deleted_mask_column.name); + storage_columns.emplace_back(RowExistsColumn::name, RowExistsColumn::type); + storage_columns_set.insert(RowExistsColumn::name); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1a34db86d4f..83eb3e55853 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include @@ -30,6 +31,7 @@ #include #include #include +#include "Storages/StorageInMemoryMetadata.h" #include #include @@ -108,7 +110,6 @@ #include #include -#include #include @@ -290,22 +291,18 @@ size_t getClusterQueriedNodes(const Settings & settings, const ClusterPtr & clus StorageDistributed::~StorageDistributed() = default; -NamesAndTypesList StorageDistributed::getVirtuals() const +VirtualColumnsDescription StorageDistributed::createVirtuals() { - /// NOTE This is weird. Most of these virtual columns are part of MergeTree + /// NOTE: This is weird. + /// Most of these virtual columns are part of MergeTree /// tables info. But Distributed is general-purpose engine. - return NamesAndTypesList{ - NameAndTypePair("_table", std::make_shared(std::make_shared())), - NameAndTypePair("_part", std::make_shared(std::make_shared())), - NameAndTypePair("_part_index", std::make_shared()), - NameAndTypePair("_part_uuid", std::make_shared()), - NameAndTypePair("_partition_id", std::make_shared(std::make_shared())), - NameAndTypePair("_sample_factor", std::make_shared()), - NameAndTypePair("_part_offset", std::make_shared()), - NameAndTypePair("_row_exists", std::make_shared()), - NameAndTypePair(BlockNumberColumn::name, BlockNumberColumn::type), - NameAndTypePair("_shard_num", std::make_shared()), /// deprecated - }; + StorageInMemoryMetadata metadata; + auto desc = MergeTreeData::createVirtuals(metadata); + + desc.addEphemeral("_table", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_shard_num", std::make_shared(), "deprecated"); + + return desc; } StorageDistributed::StorageDistributed( @@ -354,6 +351,7 @@ StorageDistributed::StorageDistributed( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + setVirtuals(createVirtuals()); if (sharding_key_) { diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index c00dd8cea04..51e4ccd4da3 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -9,6 +9,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" #include #include @@ -146,8 +147,6 @@ public: ActionLock getActionLock(StorageActionBlockType type) override; - NamesAndTypesList getVirtuals() const override; - /// Used by InterpreterInsertQuery std::string getRemoteDatabaseName() const { return remote_database; } std::string getRemoteTableName() const { return remote_table; } @@ -234,6 +233,8 @@ private: std::optional distributedWriteFromClusterStorage(const IStorageCluster & src_storage_cluster, const ASTInsertQuery & query, ContextPtr context) const; std::optional distributedWriteBetweenDistributedTables(const StorageDistributed & src_distributed, const ASTInsertQuery & query, ContextPtr context) const; + static VirtualColumnsDescription createVirtuals(); + String remote_database; String remote_table; ASTPtr remote_table_function_ptr; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index b827670bd4c..0ddfc2a6bb4 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -60,6 +61,7 @@ #include #include #include +#include "Storages/StorageSnapshot.h" #include #include @@ -132,6 +134,7 @@ StorageMerge::StorageMerge( storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + setVirtuals(createVirtuals()); } StorageMerge::StorageMerge( @@ -154,6 +157,7 @@ StorageMerge::StorageMerge( storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + setVirtuals(createVirtuals()); } StorageMerge::DatabaseTablesIterators StorageMerge::getDatabaseIterators(ContextPtr context_) const @@ -306,6 +310,37 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( return selected_table_size == 1 ? stage_in_source_tables : std::min(stage_in_source_tables, QueryProcessingStage::WithMergeableState); } +VirtualColumnsDescription StorageMerge::createVirtuals() +{ + VirtualColumnsDescription desc; + + desc.addEphemeral("_database", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_table", std::make_shared(std::make_shared()), ""); + + return desc; +} + +StorageSnapshotPtr StorageMerge::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr) const +{ + static const auto common_virtuals = createVirtuals(); + + auto virtuals = common_virtuals; + if (auto first_table = getFirstTable([](auto && table) { return table; })) + { + auto table_virtuals = first_table->getVirtualsDescription(); + for (const auto & column : *table_virtuals) + { + if (virtuals.has(column.name)) + continue; + + virtuals.add(column); + } + } + + auto virtuals_ptr = std::make_shared(std::move(virtuals)); + return std::make_shared(*this, metadata_snapshot, std::move(virtuals_ptr)); +} + void StorageMerge::read( QueryPlan & query_plan, const Names & column_names, @@ -897,7 +932,6 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextPtr & modified_ column_node = std::make_shared(NameAndTypePair{column, storage_columns.getColumn(get_column_options, column).type }, modified_query_info.table_expression); } - PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/); actions_visitor.visit(filter_actions_dag, column_node); } @@ -1375,6 +1409,7 @@ void StorageMerge::alter( params.apply(storage_metadata, local_context); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, storage_metadata); setInMemoryMetadata(storage_metadata); + setVirtuals(createVirtuals()); } void ReadFromMerge::convertAndFilterSourceStream( @@ -1634,20 +1669,4 @@ void registerStorageMerge(StorageFactory & factory) }); } -NamesAndTypesList StorageMerge::getVirtuals() const -{ - NamesAndTypesList virtuals{ - {"_database", std::make_shared(std::make_shared())}, - {"_table", std::make_shared(std::make_shared())}}; - - auto first_table = getFirstTable([](auto && table) { return table; }); - if (first_table) - { - auto table_virtuals = first_table->getVirtuals(); - virtuals.insert(virtuals.end(), table_virtuals.begin(), table_virtuals.end()); - } - - return virtuals; -} - } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 6959545430c..661750fb6dd 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -56,6 +56,8 @@ public: QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; + StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr) const override; + void read( QueryPlan & query_plan, const Names & column_names, @@ -117,11 +119,12 @@ private: template void forEachTable(F && func) const; - NamesAndTypesList getVirtuals() const override; ColumnSizeByName getColumnSizes() const override; ColumnsDescription getColumnsDescriptionFromSourceTables() const; + static VirtualColumnsDescription createVirtuals(); + bool tableSupportsPrewhere() const; template diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 73546093ff6..71d2809e18a 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -28,6 +27,16 @@ StorageSnapshot::StorageSnapshot( { } +StorageSnapshot::StorageSnapshot( + const IStorage & storage_, + StorageMetadataPtr metadata_, + VirtualsDescriptionPtr virtual_columns_) + : storage(storage_) + , metadata(std::move(metadata_)) + , virtual_columns(std::move(virtual_columns_)) +{ +} + StorageSnapshot::StorageSnapshot( const IStorage & storage_, StorageMetadataPtr metadata_, diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index e0af0525b26..390a5037780 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -37,6 +37,11 @@ struct StorageSnapshot const IStorage & storage_, StorageMetadataPtr metadata_); + StorageSnapshot( + const IStorage & storage_, + StorageMetadataPtr metadata_, + VirtualsDescriptionPtr virtual_columns_); + StorageSnapshot( const IStorage & storage_, StorageMetadataPtr metadata_, diff --git a/tests/queries/0_stateless/01848_partition_value_column.sql b/tests/queries/0_stateless/01848_partition_value_column.sql index 28d842af3e9..de5e766c92c 100644 --- a/tests/queries/0_stateless/01848_partition_value_column.sql +++ b/tests/queries/0_stateless/01848_partition_value_column.sql @@ -14,8 +14,8 @@ select count() from tbl where _partition_value.3 = 4 settings max_rows_to_read = create table tbl2(i int) engine MergeTree order by i; insert into tbl2 values (1); -select _partition_value from tbl2; -- { serverError 16 } -select _partition_value from tbl2 group by 1; -- { serverError 16 } +select _partition_value from tbl2; -- { serverError UNKNOWN_IDENTIFIER } +select _partition_value from tbl2 group by 1; -- { serverError UNKNOWN_IDENTIFIER } drop table tbl; drop table tbl2; From 2ccaf954f61576efbae4b2920aea792152fe134d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Feb 2024 20:46:37 +0000 Subject: [PATCH 056/197] Fix wacky primary key sorting in SHOW INDEX --- .../InterpreterShowIndexesQuery.cpp | 4 ++-- .../0_stateless/02724_show_indexes.reference | 24 +++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index e8005ead91e..5be72dc8ce6 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -37,7 +37,7 @@ FROM ( name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - row_number() over (order by column_name) AS seq_in_index, + row_number() over (order by null) AS seq_in_index, arrayJoin(splitByString(', ', primary_key)) AS column_name, 'A' AS collation, 0 AS cardinality, @@ -75,7 +75,7 @@ FROM ( database = '{0}' AND table = '{1}')) {2} -ORDER BY index_type, expression, column_name, seq_in_index;)", database, table, where_expression); +ORDER BY index_type, expression, seq_in_index;)", database, table, where_expression); /// Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to /// sort the output of SHOW INDEXES otherwise (SELECT * FROM (SHOW INDEXES ...) ORDER BY ...) is rejected) and 3. some diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index e41f2521f5c..ac0461fc506 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -2,33 +2,33 @@ tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- EXTENDED tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- WHERE --- Check with weird table names @@ -40,8 +40,8 @@ NULL 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- Equally named table in other database tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b From dc135294f0ecee44c3e56cd5127baf742674f04e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 25 Feb 2024 23:49:25 +0100 Subject: [PATCH 057/197] add seccomp=unconfined --- tests/ci/clickbench.py | 1 + tests/ci/fast_test_check.py | 3 ++- tests/ci/functional_test_check.py | 1 + tests/ci/libfuzzer_test_check.py | 1 + tests/ci/sqllogic_test.py | 1 + tests/ci/unit_tests_check.py | 1 + 6 files changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 50c7bb85d28..a1988abb1f5 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -42,6 +42,7 @@ def get_run_command( f"{ci_logs_args}" f"--volume={result_path}:/test_output " f"--volume={server_log_path}:/var/log/clickhouse-server " + "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls f"--cap-add=SYS_PTRACE {env_str} {image}" ) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 5d528bb4c48..a624ee06e85 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -37,9 +37,10 @@ def get_fasttest_cmd( ) -> str: return ( f"docker run --cap-add=SYS_PTRACE --user={os.geteuid()}:{os.getegid()} " + "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls "--network=host " # required to get access to IAM credentials f"-e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output " - f"-e FASTTEST_SOURCE=/ClickHouse --cap-add=SYS_PTRACE " + f"-e FASTTEST_SOURCE=/ClickHouse " f"-e FASTTEST_CMAKE_FLAGS='-DCOMPILER_CACHE=sccache' " f"-e PULL_REQUEST_NUMBER={pr_number} -e COMMIT_SHA={commit_sha} " f"-e COPY_CLICKHOUSE_BINARY_TO_OUTPUT=1 " diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index da2dea60fc1..160da202a1a 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -107,6 +107,7 @@ def get_run_command( f"{volume_with_broken_test}" f"--volume={result_path}:/test_output " f"--volume={server_log_path}:/var/log/clickhouse-server " + "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" ) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 5f41afe9fb6..465b9a6b3a7 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -81,6 +81,7 @@ def get_run_command( f"--volume={fuzzers_path}:/fuzzers " f"--volume={repo_path}/tests:/usr/share/clickhouse-test " f"--volume={result_path}:/test_output " + "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" ) diff --git a/tests/ci/sqllogic_test.py b/tests/ci/sqllogic_test.py index e9a109e425e..6ea6fa19d91 100755 --- a/tests/ci/sqllogic_test.py +++ b/tests/ci/sqllogic_test.py @@ -42,6 +42,7 @@ def get_run_command( f"--volume={repo_tests_path}:/clickhouse-tests " f"--volume={result_path}:/test_output " f"--volume={server_log_path}:/var/log/clickhouse-server " + "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls f"--cap-add=SYS_PTRACE {image}" ) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 41c52d53020..f64f114d3de 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -176,6 +176,7 @@ def main(): run_command = ( f"docker run --cap-add=SYS_PTRACE --volume={tests_binary}:/unit_tests_dbms " + "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls f"--volume={test_output}:/test_output {docker_image}" ) From 27923ee61918ad3ffc7e42cc0f12f471dd11c8eb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 28 Feb 2024 19:04:25 +0100 Subject: [PATCH 058/197] revert revert --- src/Storages/StorageFile.cpp | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 595573b566d..7d674fea9ca 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -26,6 +26,8 @@ #include #include #include +#include +#include #include #include @@ -92,6 +94,7 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; extern const int CANNOT_DETECT_FORMAT; extern const int CANNOT_COMPILE_REGEXP; + extern const int UNSUPPORTED_METHOD; } namespace @@ -276,6 +279,22 @@ std::unique_ptr selectReadBuffer( ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); } + else if (read_method == LocalFSReadMethod::io_uring && !use_table_fd) + { +#if USE_LIBURING + auto & reader = context->getIOURingReader(); + if (!reader.isSupported()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "io_uring is not supported by this system"); + + res = std::make_unique( + reader, + Priority{}, + current_path, + context->getSettingsRef().max_read_buffer_size); +#else + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Read method io_uring is only supported in Linux"); +#endif + } else { if (use_table_fd) From 3b18eb4f17e5c85afff8d2a6f07ec77c0ab98129 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Feb 2024 22:47:34 +0000 Subject: [PATCH 059/197] Fix clang-tidy in some headers --- .clang-tidy | 2 +- programs/copier/Internals.h | 2 +- programs/odbc-bridge/ODBCPooledConnectionFactory.h | 3 +-- programs/server/Server.cpp | 2 +- src/Backups/registerBackupEngineS3.cpp | 6 ++---- 5 files changed, 6 insertions(+), 9 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index 3903911a277..0dacf813c7e 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -10,7 +10,7 @@ # TODO Let clang-tidy check headers in further directories # --> HeaderFilterRegex: '^.*/(src|base|programs|utils)/.*(h|hpp)$' -HeaderFilterRegex: '^.*/(base)/.*(h|hpp)$' +HeaderFilterRegex: '^.*/(base|programs|utils)/.*(h|hpp)$' Checks: '*, -abseil-*, diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 48f4b0fab09..27fedd5d9e8 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -102,7 +102,7 @@ struct TaskStateWithOwner return TaskStateWithOwner(state, owner).toString(); } - String toString() + String toString() const { WriteBufferFromOwnString wb; wb << static_cast(state) << "\n" << escape << owner; diff --git a/programs/odbc-bridge/ODBCPooledConnectionFactory.h b/programs/odbc-bridge/ODBCPooledConnectionFactory.h index b70e45f2b9d..c4e3d4c12c6 100644 --- a/programs/odbc-bridge/ODBCPooledConnectionFactory.h +++ b/programs/odbc-bridge/ODBCPooledConnectionFactory.h @@ -40,7 +40,6 @@ public: explicit ConnectionHolder(const String & connection_string_) : pool(nullptr) - , connection() , connection_string(connection_string_) { updateConnection(); @@ -143,7 +142,7 @@ public: { std::lock_guard lock(mutex); - if (!factory.count(connection_string)) + if (!factory.contains(connection_string)) factory.emplace(std::make_pair(connection_string, std::make_shared(pool_size))); auto & pool = factory[connection_string]; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6dc33042a05..eb38c7e2ff5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -184,7 +184,7 @@ static bool jemallocOptionEnabled(const char *name) return value; } #else -static bool jemallocOptionEnabled(const char *) { return 0; } +static bool jemallocOptionEnabled(const char *) { return false; } #endif int mainEntryClickHouseServer(int argc, char ** argv) diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 86941040021..fed5c6b4d22 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -15,8 +15,6 @@ namespace DB { -namespace fs = std::filesystem; - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -65,13 +63,13 @@ void registerBackupEngineS3(BackupFactory & factory) secret_access_key = config.getString(config_prefix + ".secret_access_key", ""); if (config.has(config_prefix + ".filename")) - s3_uri = fs::path(s3_uri) / config.getString(config_prefix + ".filename"); + s3_uri = std::filesystem::path(s3_uri) / config.getString(config_prefix + ".filename"); if (args.size() > 1) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Backup S3 requires 1 or 2 arguments: named_collection, [filename]"); if (args.size() == 1) - s3_uri = fs::path(s3_uri) / args[0].safeGet(); + s3_uri = std::filesystem::path(s3_uri) / args[0].safeGet(); } else { From fc58ccfa9ee0a37539152d11e401c979f63d4713 Mon Sep 17 00:00:00 2001 From: Yong Wang Date: Wed, 28 Feb 2024 20:15:54 -0800 Subject: [PATCH 060/197] Fix_max_query_size_for_kql_compound_operato: Fix the issue of max_query_size for kql compound operator like mv-expand this fix another use case for PR 59626 --- src/Parsers/Kusto/ParserKQLQuery.cpp | 5 +++-- tests/queries/0_stateless/02366_kql_mvexpand.sql | 4 ++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Parsers/Kusto/ParserKQLQuery.cpp b/src/Parsers/Kusto/ParserKQLQuery.cpp index 29b26b4e3fe..a54a2b0eda9 100644 --- a/src/Parsers/Kusto/ParserKQLQuery.cpp +++ b/src/Parsers/Kusto/ParserKQLQuery.cpp @@ -416,8 +416,9 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserToken s_dash(TokenType::Minus); if (s_dash.ignore(pos, expected)) { - String tmp_op(op_pos_begin->begin, pos->end); - kql_operator = tmp_op; + if (!isValidKQLPos(pos)) + return false; + kql_operator = String(op_pos_begin->begin, pos->end); } else --pos; diff --git a/tests/queries/0_stateless/02366_kql_mvexpand.sql b/tests/queries/0_stateless/02366_kql_mvexpand.sql index e7798609646..ac1a6d9a8f4 100644 --- a/tests/queries/0_stateless/02366_kql_mvexpand.sql +++ b/tests/queries/0_stateless/02366_kql_mvexpand.sql @@ -33,3 +33,7 @@ print '-- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bo mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool); print '-- mv_expand_test_table | mv-expand c to typeof(bool) --'; mv_expand_test_table | mv-expand c to typeof(bool); +SET max_query_size = 28; +SET dialect='kusto'; +mv_expand_test_table | mv-expand c, d; -- { serverError 62 } +SET max_query_size=262144; From f89cb57d02603f848e1d6108a35bae51d4dd636c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Feb 2024 12:32:27 +0800 Subject: [PATCH 061/197] fix failed ut --- src/Functions/multiIf.cpp | 85 +++++++++++++++++++++------------------ 1 file changed, 45 insertions(+), 40 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index b0a344be96e..3555c195bf0 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -278,18 +278,17 @@ public: if (which.is##TYPE()) \ { \ MutableColumnPtr res = result_type->createColumn(); \ - res->reserve(rows); \ if (result_type->isNullable()) \ { \ auto & res_nullable = assert_cast(*res); \ auto & res_data = assert_cast &>(res_nullable.getNestedColumn()).getData(); \ auto & res_null_map = res_nullable.getNullMapData(); \ - executeInstructionsColumnar(instructions, rows, res_data, &res_null_map); \ + executeInstructionsColumnar(instructions, rows, res_data, &res_null_map); \ } \ else \ { \ auto & res_data = assert_cast &>(*res).getData(); \ - executeInstructionsColumnar(instructions, rows, res_data, nullptr); \ + executeInstructionsColumnar(instructions, rows, res_data, nullptr); \ } \ return std::move(res); \ } @@ -409,7 +408,7 @@ private: } } - template + template static NO_INLINE void executeInstructionsColumnar( std::vector & instructions, size_t rows, @@ -419,50 +418,56 @@ private: PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); - if (!res_null_map) + res_data.resize_exact(rows); + if (res_null_map) + res_null_map->resize_exact(rows); + + std::vector data_cols(instructions.size(), nullptr); + std::vector null_map_cols(instructions.size(), nullptr); + for (size_t i = 0; i < instructions.size(); ++i) { - for (size_t row_i = 0; row_i < rows; ++row_i) + if (instructions[i].source->isNullable()) { - auto & instruction = instructions[inserts[row_i]]; - auto ref = instruction.source->getDataAt(row_i); - res_data[row_i] = *reinterpret_cast(ref.data); - } - } - else - { - std::vector data_cols(instructions.size()); - std::vector null_map_cols(instructions.size()); - PaddedPODArray shared_null_map(rows, 0); - for (size_t i = 0; i < instructions.size(); ++i) - { - if (instructions[i].source->isNullable()) - { - const ColumnNullable * nullable_col; - if (!instructions[i].source_is_constant) - nullable_col = assert_cast(instructions[i].source.get()); - else - { - const ColumnPtr data_column = assert_cast(*instructions[i].source).getDataColumnPtr(); - nullable_col = assert_cast(data_column.get()); - } - null_map_cols[i] = assert_cast(*nullable_col->getNullMapColumnPtr()).getData().data(); - data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); - } + const ColumnNullable * nullable_col; + if (!instructions[i].source_is_constant) + nullable_col = assert_cast(instructions[i].source.get()); else { - null_map_cols[i] = shared_null_map.data(); - data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); + const ColumnPtr data_column = assert_cast(*instructions[i].source).getDataColumnPtr(); + nullable_col = assert_cast(data_column.get()); + } + null_map_cols[i] = assert_cast(*nullable_col->getNullMapColumnPtr()).getData().data(); + data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); + } + else + { + data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); + } + } + + std::unique_ptr> shared_null_map; + if constexpr (nullable_result) + { + for (auto & col : null_map_cols) + { + if (!col) + { + if (!shared_null_map) + shared_null_map = std::make_unique>(rows, 0); + + col = shared_null_map->data(); } } + } - for (size_t row_i = 0; row_i < rows; ++row_i) - { - S insert = inserts[row_i]; - auto & instruction = instructions[insert]; - size_t index = instruction.source_is_constant ? 0 : row_i; - res_data[row_i] = *(data_cols[insert] + index); + for (size_t row_i = 0; row_i < rows; ++row_i) + { + S insert = inserts[row_i]; + auto & instruction = instructions[insert]; + size_t index = instruction.source_is_constant ? 0 : row_i; + res_data[row_i] = *(data_cols[insert] + index); + if constexpr (nullable_result) (*res_null_map)[row_i] = *(null_map_cols[insert] + index); - } } } From f7d173980806bb2219955013627dee3cd6dfdc42 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Feb 2024 15:00:09 +0800 Subject: [PATCH 062/197] fix failed uts --- src/Functions/multiIf.cpp | 30 +++++++++++++----------------- tests/performance/multiif.xml | 8 ++++++++ 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 3555c195bf0..cbadbcc0e98 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -418,6 +418,11 @@ private: PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); + for (size_t i=0; i(inserts[i]) << std::endl; + } + res_data.resize_exact(rows); if (res_null_map) res_null_map->resize_exact(rows); @@ -426,23 +431,14 @@ private: std::vector null_map_cols(instructions.size(), nullptr); for (size_t i = 0; i < instructions.size(); ++i) { - if (instructions[i].source->isNullable()) - { - const ColumnNullable * nullable_col; - if (!instructions[i].source_is_constant) - nullable_col = assert_cast(instructions[i].source.get()); - else - { - const ColumnPtr data_column = assert_cast(*instructions[i].source).getDataColumnPtr(); - nullable_col = assert_cast(data_column.get()); - } - null_map_cols[i] = assert_cast(*nullable_col->getNullMapColumnPtr()).getData().data(); - data_cols[i] = assert_cast &>(*nullable_col->getNestedColumnPtr()).getData().data(); - } - else - { - data_cols[i] = assert_cast &>(*instructions[i].source).getData().data(); - } + auto & instruction = instructions[i]; + const IColumn * non_const_col = instructions[i].source_is_constant + ? &assert_cast(*instruction.source).getDataColumn() + : instruction.source.get(); + const ColumnNullable * nullable_col = checkAndGetColumn(non_const_col); + data_cols[i] = nullable_col ? assert_cast &>(nullable_col->getNestedColumn()).getData().data() + : assert_cast &>(*non_const_col).getData().data(); + null_map_cols[i] = nullable_col ? assert_cast(nullable_col->getNullMapColumn()).getData().data() : nullptr; } std::unique_ptr> shared_null_map; diff --git a/tests/performance/multiif.xml b/tests/performance/multiif.xml index ad56ab3f5f2..0c2d95cc553 100644 --- a/tests/performance/multiif.xml +++ b/tests/performance/multiif.xml @@ -5,4 +5,12 @@ select count(1) from test_multiif_t where multiIf(d > 2, d-2, d > 1, d-1, d >0, d, 0) > 1 SETTINGS max_threads=1 DROP TABLE IF EXISTS test_multiif_t + + + + SELECT count() FROM zeros(10000000) WHERE NOT ignore(multiIf( rand(1) % 2 = 0, materialize(1::Nullable(Decimal256(3))), rand(2) % 2 = 0, materialize(2::Nullable(Decimal256(3))), rand(3) % 2 = 0, materialize(3::Nullable(Decimal256(3))), rand(4) % 2 = 0, materialize(4::Nullable(Decimal256(3))), rand(5) % 2 = 0, materialize(5::Nullable(Decimal256(3))), materialize(6::Nullable(Decimal256(3))))) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(multiIf( rand(1) % 2 = 0, materialize(1::Decimal256(3)), rand(2) % 2 = 0, materialize(2::Decimal256(3)), rand(3) % 2 = 0, materialize(3::Decimal256(3)), rand(4) % 2 = 0, materialize(4::Decimal256(3)), rand(5) % 2 = 0, materialize(5::Decimal256(3)), materialize(6::Decimal256(3)))) + + + SELECT count() FROM zeros(10000000) WHERE NOT ignore(multiIf(rand() % 2 = 0, rand()+2, rand() % 3 = 0, rand()+3, rand() % 4 = 0, rand()+4, rand() % 5 = 0, rand() + 5, rand() % 6 = 0, rand() + 6, rand())) From 656412c93ee87c5ba0656833b47d4796bb793c25 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Feb 2024 15:03:59 +0800 Subject: [PATCH 063/197] fix failed uts --- src/Functions/multiIf.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index cbadbcc0e98..a9584738a91 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -418,11 +418,6 @@ private: PaddedPODArray inserts(rows, static_cast(instructions.size())); calculateInserts(instructions, rows, inserts); - for (size_t i=0; i(inserts[i]) << std::endl; - } - res_data.resize_exact(rows); if (res_null_map) res_null_map->resize_exact(rows); From 46f9fddc4d9f823c8c2fa283aed20706f2d4af4d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Feb 2024 15:11:02 +0800 Subject: [PATCH 064/197] fix code style --- src/Functions/multiIf.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index a9584738a91..5bf9f2af420 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -368,11 +368,11 @@ private: /// We should read source from which instruction on each row? template - static NO_INLINE void calculateInserts(std::vector & instructions, size_t rows, PaddedPODArray & inserts) + static NO_INLINE void calculateInserts(const std::vector & instructions, size_t rows, PaddedPODArray & inserts) { for (S i = instructions.size() - 1; i != static_cast(-1); --i) { - auto & instruction = instructions[i]; + const auto & instruction = instructions[i]; if (instruction.condition_always_true) { for (size_t row_i = 0; row_i < rows; ++row_i) @@ -410,7 +410,7 @@ private: template static NO_INLINE void executeInstructionsColumnar( - std::vector & instructions, + const std::vector & instructions, size_t rows, PaddedPODArray & res_data, PaddedPODArray * res_null_map = nullptr) @@ -426,7 +426,7 @@ private: std::vector null_map_cols(instructions.size(), nullptr); for (size_t i = 0; i < instructions.size(); ++i) { - auto & instruction = instructions[i]; + const auto & instruction = instructions[i]; const IColumn * non_const_col = instructions[i].source_is_constant ? &assert_cast(*instruction.source).getDataColumn() : instruction.source.get(); @@ -454,7 +454,7 @@ private: for (size_t row_i = 0; row_i < rows; ++row_i) { S insert = inserts[row_i]; - auto & instruction = instructions[insert]; + const auto & instruction = instructions[insert]; size_t index = instruction.source_is_constant ? 0 : row_i; res_data[row_i] = *(data_cols[insert] + index); if constexpr (nullable_result) From 0b4622ebf640ffb4d31a97d39ec4034007528c31 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Feb 2024 09:45:08 +0000 Subject: [PATCH 065/197] Add more suppressions for utils --- programs/copier/ZooKeeperStaff.h | 2 +- src/Common/LoggingFormatStringHelpers.h | 8 ++++---- utils/memcpy-bench/FastMemcpy.h | 6 ++++-- utils/memcpy-bench/FastMemcpy_Avx.h | 4 ++-- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/programs/copier/ZooKeeperStaff.h b/programs/copier/ZooKeeperStaff.h index bbdec230d2d..c15db73f060 100644 --- a/programs/copier/ZooKeeperStaff.h +++ b/programs/copier/ZooKeeperStaff.h @@ -180,7 +180,7 @@ public: auto logger = getLogger("ClusterCopier"); if (rsp.error == Coordination::Error::ZOK) { - switch (rsp.type) + switch (rsp.type) /// NOLINT(bugprone-switch-missing-default-case) { case Coordination::CREATED: LOG_DEBUG(logger, "CleanStateClock change: CREATED, at {}", rsp.path); diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index b0f0a5cd716..c60c74ee663 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -26,7 +26,7 @@ struct FormatStringHelperImpl formatStringCheckArgsNumImpl(message_format_string, sizeof...(Args)); } template - FormatStringHelperImpl(fmt::basic_runtime && str) : message_format_string(), fmt_str(std::forward>(str)) {} + FormatStringHelperImpl(fmt::basic_runtime && str) : message_format_string(), fmt_str(std::forward>(str)) {} /// NOLINT PreformattedMessage format(Args && ...args) const; }; @@ -43,9 +43,9 @@ struct PreformattedMessage template static PreformattedMessage create(FormatStringHelper fmt, Args &&... args); - operator const std::string & () const { return text; } - operator std::string () && { return std::move(text); } - operator fmt::format_string<> () const { UNREACHABLE(); } + operator const std::string & () const { return text; } /// NOLINT + operator std::string () && { return std::move(text); } /// NOLINT + operator fmt::format_string<> () const { UNREACHABLE(); } /// NOLINT void apply(std::string & out_text, std::string_view & out_format_string) const & { diff --git a/utils/memcpy-bench/FastMemcpy.h b/utils/memcpy-bench/FastMemcpy.h index 85d09c5f53e..e2ac73a1b63 100644 --- a/utils/memcpy-bench/FastMemcpy.h +++ b/utils/memcpy-bench/FastMemcpy.h @@ -33,9 +33,11 @@ #endif #endif +/// NOLINTBEGIN(modernize-use-using) typedef __attribute__((__aligned__(1))) uint16_t uint16_unaligned_t; typedef __attribute__((__aligned__(1))) uint32_t uint32_unaligned_t; typedef __attribute__((__aligned__(1))) uint64_t uint64_unaligned_t; +/// NOLINTEND(modernize-use-using) //--------------------------------------------------------------------- // fast copy for different sizes @@ -98,7 +100,7 @@ __attribute__((__no_sanitize__("undefined"))) inline void *memcpy_tiny(void * __ unsigned char *dd = ((unsigned char*)dst) + size; const unsigned char *ss = ((const unsigned char*)src) + size; - switch (size) + switch (size) /// NOLINT(bugprone-switch-missing-default-case) { case 64: memcpy_sse2_64(dd - 64, ss - 64); @@ -652,7 +654,7 @@ __attribute__((__no_sanitize__("undefined"))) inline void *memcpy_tiny(void * __ //--------------------------------------------------------------------- // main routine //--------------------------------------------------------------------- -void* memcpy_fast_sse(void * __restrict destination, const void * __restrict source, size_t size) +inline void* memcpy_fast_sse(void * __restrict destination, const void * __restrict source, size_t size) { unsigned char *dst = (unsigned char*)destination; const unsigned char *src = (const unsigned char*)source; diff --git a/utils/memcpy-bench/FastMemcpy_Avx.h b/utils/memcpy-bench/FastMemcpy_Avx.h index ee7d4e19536..3271e10d237 100644 --- a/utils/memcpy-bench/FastMemcpy_Avx.h +++ b/utils/memcpy-bench/FastMemcpy_Avx.h @@ -103,7 +103,7 @@ static INLINE void *memcpy_tiny_avx(void * __restrict dst, const void * __restri unsigned char *dd = reinterpret_cast(dst) + size; const unsigned char *ss = reinterpret_cast(src) + size; - switch (size) + switch (size) /// NOLINT(bugprone-switch-missing-default-case) { case 128: memcpy_avx_128(dd - 128, ss - 128); [[fallthrough]]; case 0: break; @@ -371,7 +371,7 @@ static INLINE void *memcpy_tiny_avx(void * __restrict dst, const void * __restri //--------------------------------------------------------------------- // main routine //--------------------------------------------------------------------- -void* memcpy_fast_avx(void * __restrict destination, const void * __restrict source, size_t size) +inline void* memcpy_fast_avx(void * __restrict destination, const void * __restrict source, size_t size) { unsigned char *dst = reinterpret_cast(destination); const unsigned char *src = reinterpret_cast(source); From e27472c09dc94e1ddaa8e5972d74ffe3d3e07eb1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Feb 2024 09:47:44 +0000 Subject: [PATCH 066/197] Correct changes in LoggingFormatStringHelpers.h --- src/Common/LoggingFormatStringHelpers.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index c60c74ee663..73bf53a955b 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -26,7 +26,7 @@ struct FormatStringHelperImpl formatStringCheckArgsNumImpl(message_format_string, sizeof...(Args)); } template - FormatStringHelperImpl(fmt::basic_runtime && str) : message_format_string(), fmt_str(std::forward>(str)) {} /// NOLINT + explicit FormatStringHelperImpl(fmt::basic_runtime && str) : fmt_str(std::forward>(str)) {} PreformattedMessage format(Args && ...args) const; }; @@ -43,9 +43,9 @@ struct PreformattedMessage template static PreformattedMessage create(FormatStringHelper fmt, Args &&... args); - operator const std::string & () const { return text; } /// NOLINT - operator std::string () && { return std::move(text); } /// NOLINT - operator fmt::format_string<> () const { UNREACHABLE(); } /// NOLINT + explicit operator const std::string & () const { return text; } + explicit operator std::string () && { return std::move(text); } + explicit operator fmt::format_string<> () const { UNREACHABLE(); } void apply(std::string & out_text, std::string_view & out_format_string) const & { From 117764e53cc740522ff5cfbd2a6aec74ad4a53df Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Feb 2024 11:33:26 +0000 Subject: [PATCH 067/197] Revert "Correct changes in LoggingFormatStringHelpers.h" This reverts commit e27472c09dc94e1ddaa8e5972d74ffe3d3e07eb1. --- src/Common/LoggingFormatStringHelpers.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index 73bf53a955b..c60c74ee663 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -26,7 +26,7 @@ struct FormatStringHelperImpl formatStringCheckArgsNumImpl(message_format_string, sizeof...(Args)); } template - explicit FormatStringHelperImpl(fmt::basic_runtime && str) : fmt_str(std::forward>(str)) {} + FormatStringHelperImpl(fmt::basic_runtime && str) : message_format_string(), fmt_str(std::forward>(str)) {} /// NOLINT PreformattedMessage format(Args && ...args) const; }; @@ -43,9 +43,9 @@ struct PreformattedMessage template static PreformattedMessage create(FormatStringHelper fmt, Args &&... args); - explicit operator const std::string & () const { return text; } - explicit operator std::string () && { return std::move(text); } - explicit operator fmt::format_string<> () const { UNREACHABLE(); } + operator const std::string & () const { return text; } /// NOLINT + operator std::string () && { return std::move(text); } /// NOLINT + operator fmt::format_string<> () const { UNREACHABLE(); } /// NOLINT void apply(std::string & out_text, std::string_view & out_format_string) const & { From 1b8ae25153a06a630e8f3553d30494a497b9c449 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 29 Feb 2024 18:16:05 +0100 Subject: [PATCH 068/197] Use cancel instead of finish in case of exception --- src/Processors/Executors/PipelineExecutor.cpp | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 8477e011763..5b5880759e6 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -391,7 +391,9 @@ void PipelineExecutor::executeImpl(size_t num_threads, bool concurrency_control) SCOPE_EXIT_SAFE( if (!finished_flag) { - finish(); + /// If finished_flag is not set, there was an exception. + /// Cancel execution in this case. + cancel(); if (pool) pool->wait(); } @@ -399,18 +401,7 @@ void PipelineExecutor::executeImpl(size_t num_threads, bool concurrency_control) if (num_threads > 1) { - try - { - spawnThreads(); // start at least one thread - } - catch (...) - { - /// spawnThreads can throw an exception, for example CANNOT_SCHEDULE_TASK. - /// We should cancel execution properly before rethrow. - cancel(); - throw; - } - + spawnThreads(); // start at least one thread tasks.processAsyncTasks(); pool->wait(); } From 8a4e15536b86727003146c16e66bb8720463713e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Feb 2024 18:01:54 +0000 Subject: [PATCH 069/197] refactoring of virtual columns --- src/Interpreters/InterpreterDescribeQuery.cpp | 20 +- src/Interpreters/InterpreterDescribeQuery.h | 1 + src/Interpreters/MutationsInterpreter.cpp | 4 +- src/Storages/ColumnsDescription.cpp | 1 - src/Storages/ColumnsDescription.h | 7 - src/Storages/FileLog/FileLogSource.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 4 +- src/Storages/HDFS/StorageHDFS.cpp | 4 +- src/Storages/HDFS/StorageHDFSCluster.cpp | 4 +- src/Storages/Hive/StorageHive.cpp | 4 +- src/Storages/IStorage.h | 4 +- src/Storages/Kafka/StorageKafka.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 19 +- src/Storages/NATS/StorageNATS.cpp | 4 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 4 +- src/Storages/S3Queue/StorageS3Queue.cpp | 4 +- src/Storages/StorageAzureBlob.cpp | 4 +- src/Storages/StorageAzureBlobCluster.cpp | 4 +- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageFileCluster.cpp | 4 +- src/Storages/StorageMaterializedView.cpp | 10 +- src/Storages/StorageMaterializedView.h | 2 + src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageS3Cluster.cpp | 4 +- src/Storages/StorageURL.cpp | 4 +- src/Storages/StorageURLCluster.cpp | 4 +- src/Storages/VirtualColumnsDescription.cpp | 17 +- src/Storages/VirtualColumnsDescription.h | 19 +- .../02890_describe_table_options.reference | 388 ++++++++---------- .../02890_describe_table_options.sql | 2 + 31 files changed, 250 insertions(+), 314 deletions(-) diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 1aab72afcc1..11542931775 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -123,28 +123,29 @@ BlockIO InterpreterDescribeQuery::execute() void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression & table_expression) { - NamesAndTypesList names_and_types; + Block sample_block; auto select_query = table_expression.subquery->children.at(0); auto current_context = getContext(); if (settings.allow_experimental_analyzer) { SelectQueryOptions select_query_options; - names_and_types = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock().getNamesAndTypesList(); + sample_block = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock(); } else { - names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context).getNamesAndTypesList(); + sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context); } - for (auto && [name, type] : names_and_types) - columns.emplace_back(std::move(name), std::move(type)); + for (auto && column : sample_block) + columns.emplace_back(std::move(column.name), std::move(column.type)); } void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpression & table_expression) { auto current_context = getContext(); TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, current_context); + auto column_descriptions = table_function_ptr->getActualTableStructure(getContext(), /*is_insert_query*/ true); for (const auto & column : column_descriptions) columns.emplace_back(column); @@ -154,14 +155,15 @@ void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpres auto table = table_function_ptr->execute(table_expression.table_function, getContext(), table_function_ptr->getName()); if (table) { - for (const auto & column : table->getVirtuals()) + for (const auto & column : *table->getVirtualsDescription()) { if (!column_descriptions.has(column.name)) - virtual_columns.emplace_back(column.name, column.type); + virtual_columns.push_back(column); } } } } + void InterpreterDescribeQuery::fillColumnsFromTable(const ASTTableExpression & table_expression) { auto table_id = getContext()->resolveStorageID(table_expression.database_and_table_name); @@ -176,10 +178,10 @@ void InterpreterDescribeQuery::fillColumnsFromTable(const ASTTableExpression & t if (settings.describe_include_virtual_columns) { - for (const auto & column : table->getVirtuals()) + for (const auto & column : *table->getVirtualsDescription()) { if (!column_descriptions.has(column.name)) - virtual_columns.emplace_back(column.name, column.type); + virtual_columns.push_back(column); } } diff --git a/src/Interpreters/InterpreterDescribeQuery.h b/src/Interpreters/InterpreterDescribeQuery.h index 5d01745db6b..64573bcb93f 100644 --- a/src/Interpreters/InterpreterDescribeQuery.h +++ b/src/Interpreters/InterpreterDescribeQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include "Storages/VirtualColumnsDescription.h" namespace DB diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 8ac25d13bf0..7a0d68f7c87 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -346,7 +346,7 @@ bool MutationsInterpreter::Source::isCompactPart() const return part && part->getType() == MergeTreeDataPartType::Compact; } -static Names getAvailableColumnswithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) +static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) { auto all_columns = metadata_snapshot->getColumns().getNamesOfPhysical(); for (const auto & column : storage.getVirtuals()) @@ -363,7 +363,7 @@ MutationsInterpreter::MutationsInterpreter( : MutationsInterpreter( Source(storage_), metadata_snapshot_, std::move(commands_), - getAvailableColumnswithVirtuals(metadata_snapshot_, *storage_), + getAvailableColumnsWithVirtuals(metadata_snapshot_, *storage_), std::move(context_), std::move(settings_)) { if (settings.can_execute && dynamic_cast(source.getStorage().get())) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6ee87297cab..e08dac3a332 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -31,7 +31,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index cc6c7d0e1b9..2a5bbe3c78e 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -66,12 +66,6 @@ struct GetColumnsOptions return *this; } - GetColumnsOptions & withPersistentVirtuals(bool value = true) - { - with_persistent_virtuals = value; - return *this; - } - GetColumnsOptions & withExtendedObjects(bool value = true) { with_extended_objects = value; @@ -82,7 +76,6 @@ struct GetColumnsOptions VirtualsKind virtuals_kind = VirtualsKind::None; bool with_subcolumns = false; - bool with_persistent_virtuals = false; bool with_extended_objects = false; }; diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index b1192af4ced..eb3ff0436a5 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -31,7 +31,7 @@ FileLogSource::FileLogSource( , max_streams_number(max_streams_number_) , handle_error_mode(handle_error_mode_) , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) - , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtuals().getNames())) + , virtual_header(storage_snapshot->virtual_columns->getSampleBlock()) { consumer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 23e869bfa89..2911f9b268b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -148,9 +148,7 @@ StorageFileLog::StorageFileLog( storage_metadata.setColumns(columns_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - - auto virtuals = createVirtuals(filelog_settings->handle_error_mode); - setVirtuals(virtuals); + setVirtuals(createVirtuals(filelog_settings->handle_error_mode)); if (!fileOrSymlinkPathStartsWith(path, getContext()->getUserFilesPath())) { diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index ca5ff263876..4a481a6a7d2 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -241,9 +241,7 @@ StorageHDFS::StorageHDFS( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } namespace diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 80aec36999c..8c6d7ce5670 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -72,9 +72,7 @@ StorageHDFSCluster::StorageHDFSCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } void StorageHDFSCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index f8c10ea4249..88ab8e15e76 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -445,9 +445,7 @@ StorageHive::StorageHive( storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext()); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } void StorageHive::lazyInitialize() diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index bd64447a00e..e7ebb45cb46 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -11,13 +11,13 @@ #include #include #include +#include #include #include #include #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include @@ -230,7 +230,7 @@ public: /// /// By default return empty list of columns. VirtualsDescriptionPtr getVirtualsDescription() const { return virtuals.get(); } - virtual NamesAndTypesList getVirtuals() const { return virtuals.get()->getNamesAndTypesList(); } + NamesAndTypesList getVirtuals() const { return virtuals.get()->getNamesAndTypesList(); } Block getVirtualsHeader() const { return virtuals.get()->getSampleBlock(); } Names getAllRegisteredNames() const override; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 3656b599788..4b87b1f7b74 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -345,9 +345,7 @@ StorageKafka::StorageKafka( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - - auto virtuals = createVirtuals(kafka_settings->kafka_handle_error_mode); - setVirtuals(virtuals); + setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode)); auto task_count = thread_per_consumer ? num_consumers : 1; for (size_t i = 0; i < task_count; ++i) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cb319348b60..e52362ff76a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -435,23 +435,22 @@ MergeTreeData::MergeTreeData( VirtualColumnsDescription MergeTreeData::createVirtuals(const StorageInMemoryMetadata & metadata) { VirtualColumnsDescription desc; - auto low_cardinality_type = std::make_shared(std::make_shared()); - desc.addEphemeral("_part", low_cardinality_type, ""); - desc.addEphemeral("_part_index", std::make_shared(), ""); - desc.addEphemeral("_part_uuid", std::make_shared(), ""); - desc.addEphemeral("_partition_id", low_cardinality_type, ""); - desc.addEphemeral("_sample_factor", std::make_shared(), ""); - desc.addEphemeral("_part_offset", std::make_shared(), ""); + desc.addEphemeral("_part", std::make_shared(std::make_shared()), "Name of part"); + desc.addEphemeral("_part_index", std::make_shared(), "Sequential index of the part in the query result"); + desc.addEphemeral("_part_uuid", std::make_shared(), "Unique part identifier (if enabled MergeTree setting assign_part_uuids)"); + desc.addEphemeral("_partition_id", std::make_shared(std::make_shared()), "Name of partition"); + desc.addEphemeral("_sample_factor", std::make_shared(), "Sample factor (from the query)"); + desc.addEphemeral("_part_offset", std::make_shared(), "Number of row in the part"); if (metadata.hasPartitionKey()) { auto partition_types = metadata.partition_key.sample_block.getDataTypes(); - desc.addEphemeral("_partition_value", std::make_shared(std::move(partition_types)), ""); + desc.addEphemeral("_partition_value", std::make_shared(std::move(partition_types)), "Value (a tuple) of a PARTITION BY expression"); } - desc.addPersistent(RowExistsColumn::name, RowExistsColumn::type, nullptr, ""); - desc.addPersistent(BlockNumberColumn::name, BlockNumberColumn::type, BlockNumberColumn::codec, ""); + desc.addPersistent(RowExistsColumn::name, RowExistsColumn::type, nullptr, "Persisted mask created by lightweight delete that show whether row exists or is deleted"); + desc.addPersistent(BlockNumberColumn::name, BlockNumberColumn::type, BlockNumberColumn::codec, "Persisted original number of block that was assigned at insert"); return desc; } diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index a06221e1b26..0b88a9e8929 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -88,9 +88,7 @@ StorageNATS::StorageNATS( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - - auto virtuals = createVirtuals(nats_settings->nats_handle_error_mode); - setVirtuals(virtuals); + setVirtuals(createVirtuals(nats_settings->nats_handle_error_mode)); nats_context = addSettings(getContext()); nats_context->makeQueryContext(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index fb7b9c10d7f..c5df752fb7f 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -137,9 +137,7 @@ StorageRabbitMQ::StorageRabbitMQ( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - - auto virtuals = createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode); - setVirtuals(virtuals); + setVirtuals(createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode)); rabbitmq_context = addSettings(getContext()); rabbitmq_context->makeQueryContext(); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 724aa88e7e4..765fcbd9684 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -155,9 +155,7 @@ StorageS3Queue::StorageS3Queue( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); task = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 41ec5961ec3..a74e274e5d9 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -492,9 +492,7 @@ StorageAzureBlob::StorageAzureBlob( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); StoredObjects objects; for (const auto & key : configuration.blobs_paths) diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index ce318deada6..155a7220885 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -63,9 +63,7 @@ StorageAzureBlobCluster::StorageAzureBlobCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } void StorageAzureBlobCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 83eb3e55853..541b2934892 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -299,8 +299,8 @@ VirtualColumnsDescription StorageDistributed::createVirtuals() StorageInMemoryMetadata metadata; auto desc = MergeTreeData::createVirtuals(metadata); - desc.addEphemeral("_table", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_shard_num", std::make_shared(), "deprecated"); + desc.addEphemeral("_table", std::make_shared(std::make_shared()), "Name of a table"); + desc.addEphemeral("_shard_num", std::make_shared(), "Deprecated. Use function shardNum instead"); return desc; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index da52b960e8b..8eaf794f445 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1097,9 +1097,7 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setConstraints(args.constraints); storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index 7aeecf7095a..84691472809 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -61,9 +61,7 @@ StorageFileCluster::StorageFileCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 9e623001520..7e953889b34 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -21,6 +22,7 @@ #include #include +#include "Storages/StorageSnapshot.h" #include #include #include @@ -153,8 +155,6 @@ StorageMaterializedView::StorageMaterializedView( *query.refresh_strategy); refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; } - - setVirtuals(*getTargetTable()->getVirtualsDescription()); } QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( @@ -167,6 +167,12 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( return getTargetTable()->getQueryProcessingStage(local_context, to_stage, getTargetTable()->getStorageSnapshot(target_metadata, local_context), query_info); } +StorageSnapshotPtr StorageMaterializedView::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr) const +{ + /// We cannot set virtuals at table creation because target table may not exist at that time. + return std::make_shared(*this, metadata_snapshot, getTargetTable()->getVirtualsDescription()); +} + void StorageMaterializedView::read( QueryPlan & query_plan, const Names & column_names, diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index d716527b5ad..198b7a642ee 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -78,6 +78,8 @@ public: ActionLock getActionLock(StorageActionBlockType type) override; void onActionLockRemove(StorageActionBlockType action_type) override; + StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr) const override; + void read( QueryPlan & query_plan, const Names & column_names, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 75ba0635125..691d197aee6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1075,9 +1075,7 @@ StorageS3::StorageS3( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } static std::shared_ptr createFileIterator( diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index b9e671cdbb8..7641c66eefd 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -69,9 +69,7 @@ StorageS3Cluster::StorageS3Cluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } void StorageS3Cluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6aa719ec321..e0c5160a43b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -152,9 +152,7 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 2f492bd7950..334c2ca249b 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -75,9 +75,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - - auto virtuals = VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns()); - setVirtuals(virtuals); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) diff --git a/src/Storages/VirtualColumnsDescription.cpp b/src/Storages/VirtualColumnsDescription.cpp index 65223071dfe..d9f9cbe047e 100644 --- a/src/Storages/VirtualColumnsDescription.cpp +++ b/src/Storages/VirtualColumnsDescription.cpp @@ -19,11 +19,12 @@ VirtualColumnDescription::VirtualColumnDescription( { } -void VirtualColumnsDescription::add(VirtualColumnDescription desc_) +void VirtualColumnsDescription::add(VirtualColumnDescription desc) { - auto [it, inserted] = container.emplace(std::move(desc_)); - if (!inserted) - throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Virtual column {} already exists", it->name); + if (container.get<1>().contains(desc.name)) + throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Virtual column {} already exists", desc.name); + + container.get<0>().push_back(std::move(desc)); } void VirtualColumnsDescription::addEphemeral(String name, DataTypePtr type, String comment) @@ -47,8 +48,8 @@ NamesAndTypesList VirtualColumnsDescription::get(VirtualsKind kind) const std::optional VirtualColumnsDescription::tryGet(const String & name, VirtualsKind kind) const { - auto it = container.find(name); - if (it != container.end() && (static_cast(it->kind) & static_cast(kind))) + auto it = container.get<1>().find(name); + if (it != container.get<1>().end() && (static_cast(it->kind) & static_cast(kind))) return NameAndTypePair{it->name, it->type}; return {}; } @@ -63,8 +64,8 @@ NameAndTypePair VirtualColumnsDescription::get(const String & name, VirtualsKind std::optional VirtualColumnsDescription::tryGetDescription(const String & name, VirtualsKind kind) const { - auto it = container.find(name); - if (it != container.end() && (static_cast(it->kind) & static_cast(kind))) + auto it = container.get<1>().find(name); + if (it != container.get<1>().end() && (static_cast(it->kind) & static_cast(kind))) return *it; return {}; } diff --git a/src/Storages/VirtualColumnsDescription.h b/src/Storages/VirtualColumnsDescription.h index 2564e49a0e9..3d79167c623 100644 --- a/src/Storages/VirtualColumnsDescription.h +++ b/src/Storages/VirtualColumnsDescription.h @@ -16,20 +16,18 @@ public: bool isEphemeral() const { return kind == VirtualsKind::Ephemeral; } bool isPersistent() const { return kind == VirtualsKind::Persistent; } - - struct Comparator - { - using is_transparent = void; - bool operator()(const Self & lhs, const Self & rhs) const { return lhs.name < rhs.name; } - bool operator()(const Self & lhs, const String & rhs) const { return lhs.name < rhs; } - bool operator()(const String & lhs, const Self & rhs) const { return lhs < rhs.name; } - }; + const String & getName() const { return name; } }; class VirtualColumnsDescription { public: - using Container = std::set; + using Container = boost::multi_index_container< + VirtualColumnDescription, + boost::multi_index::indexed_by< + boost::multi_index::sequenced<>, + boost::multi_index::ordered_unique>>>; + using const_iterator = Container::const_iterator; const_iterator begin() const { return container.begin(); } @@ -41,8 +39,9 @@ public: void addEphemeral(String name, DataTypePtr type, String comment); void addPersistent(String name, DataTypePtr type, ASTPtr codec, String comment); + size_t size() const { return container.size(); } bool empty() const { return container.empty(); } - bool has(const String & name) const { return container.contains(name); } + bool has(const String & name) const { return container.get<1>().contains(name); } NameAndTypePair get(const String & name, VirtualsKind kind) const; std::optional tryGet(const String & name, VirtualsKind kind) const; diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 5d99df36bb4..ecbfe5a9f4d 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -2,237 +2,201 @@ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┠-│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ +┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┠+│ id │ UInt64 │ │ │ index column │ │ │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ +└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┠-│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ +┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┠+│ id │ UInt64 │ │ │ index column │ │ │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ +└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┠-│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ +┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┠+│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ +└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┠-│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ +┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┠+│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ +└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┠-│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _partition_value │ UInt8 │ │ │ │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ -└──────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┠+│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 1 │ +│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 1 │ +│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 1 │ +│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 1 │ +│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 1 │ +│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 1 │ +│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 1 │ +└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┠-│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ -│ _shard_num │ UInt32 │ │ │ │ │ │ 1 │ -└────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┠+│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 1 │ +│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 1 │ +│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 1 │ +│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 1 │ +│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 1 │ +│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 1 │ +│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 1 │ +│ _table │ LowCardinality(String) │ │ │ Name of a table │ │ │ 1 │ +│ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 1 │ +└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┠-│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _partition_value │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└──────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┠+│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 0 │ 1 │ +│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 0 │ 1 │ +│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 0 │ 1 │ +│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 0 │ 1 │ +│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 0 │ 1 │ +│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 0 │ 1 │ +│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┠-│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _shard_num │ UInt32 │ │ │ │ │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┠+│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ +│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 0 │ 1 │ +│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 0 │ 1 │ +│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 0 │ 1 │ +│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 0 │ 1 │ +│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 0 │ 1 │ +│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 0 │ 1 │ +│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 0 │ 1 │ +│ _table │ LowCardinality(String) │ │ │ Name of a table │ │ │ 0 │ 1 │ +│ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type─────────────────────────────┠-│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple( - a String, - b UInt64) │ -└──────┴──────────────────────────────────┘ +┌─name─┬─type──────────────────────┠+│ id │ UInt64 │ +│ arr │ Array(UInt64) │ +│ t │ Tuple(a String, b UInt64) │ +└──────┴───────────────────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type─────────────────────────────┠-│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple( - a String, - b UInt64) │ -└──────┴──────────────────────────────────┘ +┌─name─┬─type──────────────────────┠+│ id │ UInt64 │ +│ arr │ Array(UInt64) │ +│ t │ Tuple(a String, b UInt64) │ +└──────┴───────────────────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type─────────────────────────────┬─is_subcolumn─┠-│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴──────────────────────────────────┴──────────────┘ +┌─name──────┬─type──────────────────────┬─is_subcolumn─┠+│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple(a String, b UInt64) │ 0 │ +│ arr.size0 │ UInt64 │ 1 │ +│ t.a │ String │ 1 │ +│ t.b │ UInt64 │ 1 │ +└───────────┴───────────────────────────┴──────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type─────────────────────────────┬─is_subcolumn─┠-│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴──────────────────────────────────┴──────────────┘ +┌─name──────┬─type──────────────────────┬─is_subcolumn─┠+│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple(a String, b UInt64) │ 0 │ +│ arr.size0 │ UInt64 │ 1 │ +│ t.a │ String │ 1 │ +│ t.b │ UInt64 │ 1 │ +└───────────┴───────────────────────────┴──────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type─────────────────────────────┬─is_virtual─┠-│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ 0 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _partition_value │ UInt8 │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -└──────────────────┴──────────────────────────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─is_virtual─┠+│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple(a String, b UInt64) │ 0 │ +│ _part │ LowCardinality(String) │ 1 │ +│ _part_index │ UInt64 │ 1 │ +│ _part_uuid │ UUID │ 1 │ +│ _partition_id │ LowCardinality(String) │ 1 │ +│ _sample_factor │ Float64 │ 1 │ +│ _part_offset │ UInt64 │ 1 │ +│ _row_exists │ UInt8 │ 1 │ +│ _block_number │ UInt64 │ 1 │ +└────────────────┴───────────────────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type─────────────────────────────┬─is_virtual─┠-│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ 0 │ -│ _table │ LowCardinality(String) │ 1 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -│ _shard_num │ UInt32 │ 1 │ -└────────────────┴──────────────────────────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─is_virtual─┠+│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple(a String, b UInt64) │ 0 │ +│ _part │ LowCardinality(String) │ 1 │ +│ _part_index │ UInt64 │ 1 │ +│ _part_uuid │ UUID │ 1 │ +│ _partition_id │ LowCardinality(String) │ 1 │ +│ _sample_factor │ Float64 │ 1 │ +│ _part_offset │ UInt64 │ 1 │ +│ _row_exists │ UInt8 │ 1 │ +│ _block_number │ UInt64 │ 1 │ +│ _table │ LowCardinality(String) │ 1 │ +│ _shard_num │ UInt32 │ 1 │ +└────────────────┴───────────────────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type─────────────────────────────┬─is_subcolumn─┬─is_virtual─┠-│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _partition_value │ UInt8 │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└──────────────────┴──────────────────────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┠+│ id │ UInt64 │ 0 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ 0 │ +│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ 0 │ 1 │ +│ _part_index │ UInt64 │ 0 │ 1 │ +│ _part_uuid │ UUID │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ +│ _sample_factor │ Float64 │ 0 │ 1 │ +│ _part_offset │ UInt64 │ 0 │ 1 │ +│ _row_exists │ UInt8 │ 0 │ 1 │ +│ _block_number │ UInt64 │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ 1 │ 0 │ +│ t.a │ String │ 1 │ 0 │ +│ t.b │ UInt64 │ 1 │ 0 │ +└────────────────┴───────────────────────────┴──────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type─────────────────────────────┬─is_subcolumn─┬─is_virtual─┠-│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple( - a String, - b UInt64) │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ 0 │ 1 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ _shard_num │ UInt32 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└────────────────┴──────────────────────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┠+│ id │ UInt64 │ 0 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ 0 │ +│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ 0 │ 1 │ +│ _part_index │ UInt64 │ 0 │ 1 │ +│ _part_uuid │ UUID │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ +│ _sample_factor │ Float64 │ 0 │ 1 │ +│ _part_offset │ UInt64 │ 0 │ 1 │ +│ _row_exists │ UInt8 │ 0 │ 1 │ +│ _block_number │ UInt64 │ 0 │ 1 │ +│ _table │ LowCardinality(String) │ 0 │ 1 │ +│ _shard_num │ UInt32 │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ 1 │ 0 │ +│ t.a │ String │ 1 │ 0 │ +│ t.b │ UInt64 │ 1 │ 0 │ +└────────────────┴───────────────────────────┴──────────────┴────────────┘ diff --git a/tests/queries/0_stateless/02890_describe_table_options.sql b/tests/queries/0_stateless/02890_describe_table_options.sql index 236100148ff..63806c7ee3d 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.sql +++ b/tests/queries/0_stateless/02890_describe_table_options.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS t_describe_options; +SET print_pretty_type_names = 0; + CREATE TABLE t_describe_options ( id UInt64 COMMENT 'index column', arr Array(UInt64) DEFAULT [10, 20] CODEC(ZSTD), From d4702eaddf2a97165f672250afe49ea1f1193e53 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 29 Feb 2024 19:34:17 +0100 Subject: [PATCH 070/197] Fix test --- .../0_stateless/02998_analyzer_secret_args_tree_node.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql index f1d916eca80..f40b40b6c8c 100644 --- a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql @@ -1,4 +1,5 @@ --- Tags: no-fasttest: encrypt function doesn't exist in the fastest build +-- Tags: no-fasttest +-- encrypt function doesn't exist in the fastest build -- { echoOn } SET allow_experimental_analyzer = 1; From 621ea46b57be3c9422a1fe54db11dbe1e5d726ad Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 29 Feb 2024 13:36:56 -0500 Subject: [PATCH 071/197] Update 02998_analyzer_secret_args_tree_node.sql --- .../0_stateless/02998_analyzer_secret_args_tree_node.sql | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql index f40b40b6c8c..e5b7a9fe20a 100644 --- a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- encrypt function doesn't exist in the fastest build +-- Tag: no-fasttest: encrypt function doesn't exist in the fastest build -- { echoOn } SET allow_experimental_analyzer = 1; From caf50f4ea1613343a1e4fb5bed0bee8917afaa3c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Feb 2024 21:09:12 +0000 Subject: [PATCH 072/197] fix test --- src/Interpreters/InterpreterDescribeQuery.h | 1 - src/Interpreters/TreeRewriter.cpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterDescribeQuery.h b/src/Interpreters/InterpreterDescribeQuery.h index 64573bcb93f..5d01745db6b 100644 --- a/src/Interpreters/InterpreterDescribeQuery.h +++ b/src/Interpreters/InterpreterDescribeQuery.h @@ -4,7 +4,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" namespace DB diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 1a32b885f4d..896182c1f0f 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1119,7 +1119,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select optimize_trivial_count = true; for (const auto & required_column : required) { - if (partition_columns_set.contains(required_column)) + if (!partition_columns_set.contains(required_column)) { optimize_trivial_count = false; break; From 3b3a7c144bb93bd89eff401ed1596217c287837c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 29 Feb 2024 18:47:27 -0500 Subject: [PATCH 073/197] Update 02998_analyzer_secret_args_tree_node.sql --- .../0_stateless/02998_analyzer_secret_args_tree_node.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql index e5b7a9fe20a..f40b40b6c8c 100644 --- a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql @@ -1,4 +1,5 @@ --- Tag: no-fasttest: encrypt function doesn't exist in the fastest build +-- Tags: no-fasttest +-- encrypt function doesn't exist in the fastest build -- { echoOn } SET allow_experimental_analyzer = 1; From 845dcc95eca3e8664553c664f51432315f472d09 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 1 Mar 2024 11:43:54 +0000 Subject: [PATCH 074/197] Fix build --- utils/memcpy-bench/FastMemcpy.h | 2 +- utils/memcpy-bench/FastMemcpy_Avx.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/memcpy-bench/FastMemcpy.h b/utils/memcpy-bench/FastMemcpy.h index e2ac73a1b63..650a6761771 100644 --- a/utils/memcpy-bench/FastMemcpy.h +++ b/utils/memcpy-bench/FastMemcpy.h @@ -654,7 +654,7 @@ __attribute__((__no_sanitize__("undefined"))) inline void *memcpy_tiny(void * __ //--------------------------------------------------------------------- // main routine //--------------------------------------------------------------------- -inline void* memcpy_fast_sse(void * __restrict destination, const void * __restrict source, size_t size) +void* memcpy_fast_sse(void * __restrict destination, const void * __restrict source, size_t size) /// NOLINT(misc-definitions-in-headers) { unsigned char *dst = (unsigned char*)destination; const unsigned char *src = (const unsigned char*)source; diff --git a/utils/memcpy-bench/FastMemcpy_Avx.h b/utils/memcpy-bench/FastMemcpy_Avx.h index 3271e10d237..aecf8abbe03 100644 --- a/utils/memcpy-bench/FastMemcpy_Avx.h +++ b/utils/memcpy-bench/FastMemcpy_Avx.h @@ -371,7 +371,7 @@ static INLINE void *memcpy_tiny_avx(void * __restrict dst, const void * __restri //--------------------------------------------------------------------- // main routine //--------------------------------------------------------------------- -inline void* memcpy_fast_avx(void * __restrict destination, const void * __restrict source, size_t size) +void* memcpy_fast_avx(void * __restrict destination, const void * __restrict source, size_t size) /// NOLINT(misc-definitions-in-headers) { unsigned char *dst = reinterpret_cast(destination); const unsigned char *src = reinterpret_cast(source); From 447f0817d5dfe70070d0e72a09188cd2306cb88e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Mar 2024 12:08:51 +0000 Subject: [PATCH 075/197] Fixing build --- .../System/IStorageSystemOneBlock.cpp | 100 ++++++++++++++++++ 1 file changed, 100 insertions(+) create mode 100644 src/Storages/System/IStorageSystemOneBlock.cpp diff --git a/src/Storages/System/IStorageSystemOneBlock.cpp b/src/Storages/System/IStorageSystemOneBlock.cpp new file mode 100644 index 00000000000..20c8f3517ee --- /dev/null +++ b/src/Storages/System/IStorageSystemOneBlock.cpp @@ -0,0 +1,100 @@ +#include +// #include +// #include +// #include +// #include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class ReadFromSystemOneBlock : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemOneBlock"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemOneBlock( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , columns_mask(std::move(columns_mask_)) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::shared_ptr storage; + std::vector columns_mask; + const ActionsDAG::Node * predicate = nullptr; +}; + +void IStorageSystemOneBlock::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + size_t /*num_streams*/) + +{ + storage_snapshot->check(column_names); + Block sample_block = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + std::vector columns_mask; + + if (supportsColumnsMask()) + { + auto [columns_mask_, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + columns_mask = std::move(columns_mask_); + sample_block = std::move(header); + } + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(sample_block), std::move(this_ptr), std::move(columns_mask)); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemOneBlock::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + const auto & sample_block = getOutputStream().header; + MutableColumns res_columns = sample_block.cloneEmptyColumns(); + storage->fillData(res_columns, context, predicate, std::move(columns_mask)); + + UInt64 num_rows = res_columns.at(0)->size(); + Chunk chunk(std::move(res_columns), num_rows); + + pipeline.init(Pipe(std::make_shared(sample_block, std::move(chunk)))); +} + +void ReadFromSystemOneBlock::applyFilters(ActionDAGNodes added_filter_nodes) +{ + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); +} + +} From dc2d022cc49fd9bcf892a768c4063930f24e5104 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 1 Mar 2024 12:29:57 +0000 Subject: [PATCH 076/197] Docs: Follow-up to #58023 --- docs/en/sql-reference/statements/alter/column.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index f6d9668e628..902eae2d67d 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -272,10 +272,16 @@ ALTER TABLE table_name MODIFY COLUMN column_name RESET SETTING max_compress_bloc ## MATERIALIZE COLUMN -Materializes or updates a column with an expression for a default value (`DEFAULT` or `MATERIALIZED`). -It is used if it is necessary to add or update a column with a complicated expression, because evaluating such an expression directly on `SELECT` executing turns out to be expensive. +Materializes a column with a `DEFAULT` or `MATERIALIZED` value expression. +This statement can be used to rewrite existing column data after a `DEFAULT` or `MATERIALIZED` expression has been added or updated (which only updates the metadata but does not change existing data). Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). +For columns with a new or updated `MATERIALIZED` value expression, all existing rows are rewritten. + +For columns with a new or updated `DEFAULT` value expression, the behavior changed in ClickHouse v24.2: +- In ClickHouse < v24.2, all existing rows are changed to the new `DEFAULT` value expression. +- In ClickHouse >= v24.2, only rows containing the previous default value are changed to the new `DEFAULT` value expression. Rows with non-default values are kept as is. + Syntax: ```sql From b45ff56e4df3ed300a5d6c7e5b51979231f11fb7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 27 Feb 2024 18:27:06 +0100 Subject: [PATCH 077/197] Do not check to and from files existence in metadata_storage because it does not see uncommitted changes --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index de7a71e8dc1..66e8beb0637 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -595,12 +595,6 @@ void DiskObjectStorageTransaction::moveFile(const String & from_path, const Stri operations_to_execute.emplace_back( std::make_unique(object_storage, metadata_storage, [from_path, to_path, this](MetadataTransactionPtr tx) { - if (metadata_storage.exists(to_path)) - throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "File already exists: {}", to_path); - - if (!metadata_storage.exists(from_path)) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist, cannot move", from_path); - tx->moveFile(from_path, to_path); })); } From 590c30a23a0289b2e5333b54ebec204f00163209 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 28 Feb 2024 16:25:38 +0100 Subject: [PATCH 078/197] Update DiskObjectStorageTransaction.cpp --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 66e8beb0637..e1ac9f6fba7 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -23,7 +23,6 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; extern const int FILE_DOESNT_EXIST; extern const int BAD_FILE_TYPE; - extern const int FILE_ALREADY_EXISTS; extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; extern const int LOGICAL_ERROR; } From 2dd83a09e3e150f041482953bb05a3e2c7ac6cec Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 28 Feb 2024 16:55:23 +0100 Subject: [PATCH 079/197] No need to capture this --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index e1ac9f6fba7..d25add625e8 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -592,7 +592,7 @@ void DiskObjectStorageTransaction::moveDirectory(const std::string & from_path, void DiskObjectStorageTransaction::moveFile(const String & from_path, const String & to_path) { operations_to_execute.emplace_back( - std::make_unique(object_storage, metadata_storage, [from_path, to_path, this](MetadataTransactionPtr tx) + std::make_unique(object_storage, metadata_storage, [from_path, to_path](MetadataTransactionPtr tx) { tx->moveFile(from_path, to_path); })); From 2811b24a4706eaa490ac6db26e0c59a26dd1f4d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Mar 2024 14:42:46 +0000 Subject: [PATCH 080/197] Fixing build --- src/Storages/System/StorageSystemDNSCache.cpp | 2 +- src/Storages/System/StorageSystemDNSCache.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/System/StorageSystemDNSCache.cpp b/src/Storages/System/StorageSystemDNSCache.cpp index 867d1fd55b9..6fb83597908 100644 --- a/src/Storages/System/StorageSystemDNSCache.cpp +++ b/src/Storages/System/StorageSystemDNSCache.cpp @@ -32,7 +32,7 @@ ColumnsDescription StorageSystemDNSCache::getColumnsDescription() }; } -void StorageSystemDNSCache::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemDNSCache::fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const { using HostIPPair = std::pair; std::unordered_set>> reported_elements; diff --git a/src/Storages/System/StorageSystemDNSCache.h b/src/Storages/System/StorageSystemDNSCache.h index 340060335b8..dd24d2f35f6 100644 --- a/src/Storages/System/StorageSystemDNSCache.h +++ b/src/Storages/System/StorageSystemDNSCache.h @@ -9,7 +9,7 @@ namespace DB class Context; /// system.dns_cache table. -class StorageSystemDNSCache final : public IStorageSystemOneBlock +class StorageSystemDNSCache final : public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemDNSCache"; } @@ -19,7 +19,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; } From 47aa33408bee0ffc7f5857bd4859bc8d0157f51e Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 1 Mar 2024 12:06:12 +0000 Subject: [PATCH 081/197] Resubmit "Analyzer: compute ALIAS columns right after reading" This reverts commit 8141e1c3d19072d3aefd5bfc162b97bd6655ce96, reversing changes made to fb099bbd62b012a3b0c22ab7bd94078ea4883182. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 +- .../getHeaderForProcessingStage.cpp | 7 +- src/Planner/CollectTableExpressionData.cpp | 166 +++++++++++------- src/Planner/CollectTableExpressionData.h | 2 +- src/Planner/PlannerActionsVisitor.cpp | 5 +- src/Planner/PlannerJoinTree.cpp | 96 +++------- src/Planner/TableExpressionData.h | 115 +++++------- src/Planner/Utils.cpp | 11 +- src/Planner/Utils.h | 5 + .../QueryPlan/ReadFromMergeTree.cpp | 5 + src/Storages/StorageDistributed.cpp | 28 +++ tests/analyzer_integration_broken_tests.txt | 1 - .../test_row_policy.py | 2 +- tests/integration/test_row_policy/test.py | 3 +- .../02514_analyzer_drop_join_on.reference | 12 +- ..._support_alias_column_in_indices.reference | 13 +- 16 files changed, 245 insertions(+), 231 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 907a732493d..1f81ac54078 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6651,7 +6651,6 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table if (column_default && column_default->kind == ColumnDefaultKind::Alias) { auto alias_expression = buildQueryTree(column_default->expression, scope.context); - alias_expression = buildCastFunction(alias_expression, column_name_and_type.type, scope.context, false /*resolve*/); auto column_node = std::make_shared(column_name_and_type, std::move(alias_expression), table_expression_node); column_name_to_column_node.emplace(column_name_and_type.name, column_node); alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node); @@ -6684,7 +6683,9 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table alias_column_resolve_scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - + auto & resolved_expression = alias_column_to_resolve->getExpression(); + if (!resolved_expression->getResultType()->equals(*alias_column_to_resolve->getResultType())) + resolved_expression = buildCastFunction(resolved_expression, alias_column_to_resolve->getResultType(), scope.context, true); column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node); column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; } diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 21739298036..75b0e710fbe 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -121,7 +121,12 @@ Block getHeaderForProcessingStage( auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(left_table_expression); const auto & query_context = query_info.planner_context->getQueryContext(); - auto columns = table_expression_data.getColumns(); + + NamesAndTypes columns; + const auto & column_name_to_column = table_expression_data.getColumnNameToColumn(); + for (const auto & column_name : table_expression_data.getSelectedColumnsNames()) + columns.push_back(column_name_to_column.at(column_name)); + auto new_query_node = buildSubqueryToReadColumnsFromTableExpression(columns, left_table_expression, query_context); query = new_query_node->toAST(); } diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 78a7c7074c3..385381f1355 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -29,34 +29,13 @@ namespace class CollectSourceColumnsVisitor : public InDepthQueryTreeVisitor { public: - explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_) + explicit CollectSourceColumnsVisitor(PlannerContextPtr & planner_context_, bool keep_alias_columns_ = true) : planner_context(planner_context_) + , keep_alias_columns(keep_alias_columns_) {} void visitImpl(QueryTreeNodePtr & node) { - /// Special case for USING clause which contains references to ALIAS columns. - /// We can not modify such ColumnNode. - if (auto * join_node = node->as()) - { - if (!join_node->isUsingJoinExpression()) - return; - - auto & using_list = join_node->getJoinExpression()->as(); - for (auto & using_element : using_list) - { - auto & column_node = using_element->as(); - /// This list contains column nodes from left and right tables. - auto & columns_from_subtrees = column_node.getExpressionOrThrow()->as().getNodes(); - - /// Visit left table column node. - visitUsingColumn(columns_from_subtrees[0]); - /// Visit right table column node. - visitUsingColumn(columns_from_subtrees[1]); - } - return; - } - auto * column_node = node->as(); if (!column_node) return; @@ -72,22 +51,55 @@ public: /// JOIN using expression if (column_node->hasExpression() && column_source_node_type == QueryTreeNodeType::JOIN) - return; - - auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_source_node); - - if (column_node->hasExpression() && column_source_node_type != QueryTreeNodeType::ARRAY_JOIN) { - /// Replace ALIAS column with expression + auto & columns_from_subtrees = column_node->getExpression()->as().getNodes(); + if (columns_from_subtrees.size() != 2) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected two columns in JOIN using expression for column {}", column_node->dumpTree()); + + visit(columns_from_subtrees[0]); + visit(columns_from_subtrees[1]); + return; + } + + auto & table_expression_data = planner_context->getOrCreateTableExpressionData(column_source_node); + + if (isAliasColumn(node)) + { + /// Column is an ALIAS column with expression bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName()); if (!column_already_exists) { - auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node); - table_expression_data.addAliasColumnName(column_node->getColumnName(), column_identifier); + CollectSourceColumnsVisitor visitor_for_alias_column(planner_context); + /// While we are processing expression of ALIAS columns we should not add source columns to selected. + /// See also comment for `select_added_columns` + visitor_for_alias_column.select_added_columns = false; + visitor_for_alias_column.keep_alias_columns = keep_alias_columns; + visitor_for_alias_column.visit(column_node->getExpression()); + + if (!keep_alias_columns) + { + /// For PREWHERE we can just replace ALIAS column with it's expression, + /// because ActionsDAG for PREWHERE applied right on top of table expression + /// and cannot affect subqueries or other table expressions. + node = column_node->getExpression(); + return; + } + + auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node); + + ActionsDAGPtr alias_column_actions_dag = std::make_shared(); + PlannerActionsVisitor actions_visitor(planner_context, false); + auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression()); + if (outputs.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size()); + const auto & column_name = column_node->getColumnName(); + const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name); + alias_column_actions_dag->addOrReplaceInOutputs(alias_node); + table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, alias_column_actions_dag, select_added_columns); } - node = column_node->getExpression(); - visitImpl(node); return; } @@ -102,45 +114,58 @@ public: bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName()); if (column_already_exists) + { + /// Column may be added when we collected data for ALIAS column + /// But now we see it directly in the query, so make sure it's marked as selected + if (select_added_columns) + table_expression_data.markSelectedColumn(column_node->getColumnName()); return; + } - auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node); - table_expression_data.addColumn(column_node->getColumn(), column_identifier); + auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node); + table_expression_data.addColumn(column_node->getColumn(), column_identifier, select_added_columns); } - static bool needChildVisit(const QueryTreeNodePtr & parent, const QueryTreeNodePtr & child_node) + static bool isAliasColumn(const QueryTreeNodePtr & node) + { + const auto * column_node = node->as(); + if (!column_node || !column_node->hasExpression()) + return false; + const auto & column_source = column_node->getColumnSourceOrNull(); + if (!column_source) + return false; + return column_source->getNodeType() != QueryTreeNodeType::JOIN && + column_source->getNodeType() != QueryTreeNodeType::ARRAY_JOIN; + } + + static bool needChildVisit(const QueryTreeNodePtr & parent_node, const QueryTreeNodePtr & child_node) { - if (auto * join_node = parent->as()) - { - if (join_node->getJoinExpression() == child_node && join_node->isUsingJoinExpression()) - return false; - } auto child_node_type = child_node->getNodeType(); - return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION); + return !(child_node_type == QueryTreeNodeType::QUERY || + child_node_type == QueryTreeNodeType::UNION || + isAliasColumn(parent_node)); + } + + void setKeepAliasColumns(bool keep_alias_columns_) + { + keep_alias_columns = keep_alias_columns_; } private: + PlannerContextPtr & planner_context; - void visitUsingColumn(QueryTreeNodePtr & node) - { - auto & column_node = node->as(); - if (column_node.hasExpression()) - { - auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_node.getColumnSource()); - bool column_already_exists = table_expression_data.hasColumn(column_node.getColumnName()); - if (column_already_exists) - return; + /// Replace ALIAS columns with their expressions or register them in table expression data. + /// Usually we can replace them when we build some "local" actions DAG + /// (for example Row Policy or PREWHERE) that is applied on top of the table expression. + /// In other cases, we keep ALIAS columns as ColumnNode with an expression child node, + /// and handle them in the Planner by inserting ActionsDAG to compute them after reading from storage. + bool keep_alias_columns = true; - auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node); - table_expression_data.addAliasColumnName(column_node.getColumnName(), column_identifier); - - visitImpl(column_node.getExpressionOrThrow()); - } - else - visitImpl(node); - } - - PlannerContext & planner_context; + /// Flag `select_added_columns` indicates if we should mark column as explicitly selected. + /// For example, for table with columns (a Int32, b ALIAS a+1) and query SELECT b FROM table + /// Column `b` is selected explicitly by user, but not `a` (that is also read though). + /// Distinguishing such columns is important for checking access rights for ALIAS columns. + bool select_added_columns = true; }; class CollectPrewhereTableExpressionVisitor : public ConstInDepthQueryTreeVisitor @@ -274,7 +299,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr } } - CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context); + CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context); for (auto & node : query_node_typed.getChildren()) { if (!node || node == query_node_typed.getPrewhere()) @@ -300,21 +325,26 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr } auto & table_expression_data = planner_context->getOrCreateTableExpressionData(prewhere_table_expression); - const auto & column_names = table_expression_data.getColumnNames(); - NameSet required_column_names_without_prewhere(column_names.begin(), column_names.end()); + const auto & read_column_names = table_expression_data.getColumnNames(); + NameSet required_column_names_without_prewhere(read_column_names.begin(), read_column_names.end()); + const auto & selected_column_names = table_expression_data.getSelectedColumnsNames(); + required_column_names_without_prewhere.insert(selected_column_names.begin(), selected_column_names.end()); + collect_source_columns_visitor.setKeepAliasColumns(false); collect_source_columns_visitor.visit(query_node_typed.getPrewhere()); auto prewhere_actions_dag = std::make_shared(); + QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere(); + PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = visitor.visit(prewhere_actions_dag, query_node_typed.getPrewhere()); + auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node); if (expression_nodes.size() != 1) throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "Invalid PREWHERE. Expected single boolean expression. In query {}", query_node->formatASTForErrorMessage()); - prewhere_actions_dag->getOutputs().push_back(expression_nodes[0]); + prewhere_actions_dag->getOutputs().push_back(expression_nodes.back()); for (const auto & prewhere_input_node : prewhere_actions_dag->getInputs()) if (required_column_names_without_prewhere.contains(prewhere_input_node->result_name)) @@ -324,9 +354,9 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr } } -void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context) +void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context, bool keep_alias_columns) { - CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context); + CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context, keep_alias_columns); collect_source_columns_visitor.visit(expression_node); } diff --git a/src/Planner/CollectTableExpressionData.h b/src/Planner/CollectTableExpressionData.h index ed3f0ff7a47..b0cebc15682 100644 --- a/src/Planner/CollectTableExpressionData.h +++ b/src/Planner/CollectTableExpressionData.h @@ -19,6 +19,6 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr * * ALIAS table column nodes are registered in table expression data and replaced in query tree with inner alias expression. */ -void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context); +void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context, bool keep_alias_columns = true); } diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 511e9396a35..c417d463c73 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -451,6 +451,7 @@ private: std::unordered_map node_to_node_name; const PlannerContextPtr planner_context; ActionNodeNameHelper action_node_name_helper; + bool use_column_identifier_as_action_node_name; }; PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, @@ -458,6 +459,7 @@ PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, bool use_column_identifier_as_action_node_name_) : planner_context(planner_context_) , action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_) + , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) { actions_stack.emplace_back(std::move(actions_dag), nullptr); } @@ -503,7 +505,8 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi { auto column_node_name = action_node_name_helper.calculateActionNodeName(node); const auto & column_node = node->as(); - + if (column_node.hasExpression() && !use_column_identifier_as_action_node_name) + return visitImpl(column_node.getExpression()); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); for (Int64 i = actions_stack_size; i >= 0; --i) { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 59da88f4e45..7b3fb0c5c91 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -86,7 +86,7 @@ namespace /// Check if current user has privileges to SELECT columns from table /// Throws an exception if access to any column from `column_names` is not granted /// If `column_names` is empty, check access to any columns and return names of accessible columns -NameSet checkAccessRights(const TableNode & table_node, Names & column_names, const ContextPtr & query_context) +NameSet checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context) { /// StorageDummy is created on preliminary stage, ignore access check for it. if (typeid_cast(table_node.getStorage().get())) @@ -353,9 +353,7 @@ void prepareBuildQueryPlanForTableExpression(const QueryTreeNodePtr & table_expr NameSet columns_names_allowed_to_select; if (table_node) { - auto column_names_with_aliases = columns_names; - const auto & alias_columns_names = table_expression_data.getAliasColumnsNames(); - column_names_with_aliases.insert(column_names_with_aliases.end(), alias_columns_names.begin(), alias_columns_names.end()); + const auto & column_names_with_aliases = table_expression_data.getSelectedColumnsNames(); columns_names_allowed_to_select = checkAccessRights(*table_node, column_names_with_aliases, query_context); } @@ -864,6 +862,28 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres max_block_size, max_streams); + const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); + if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns) + { + ActionsDAGPtr merged_alias_columns_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs(); + + for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) + { + const auto & current_outputs = alias_column_actions_dag->getOutputs(); + action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end()); + merged_alias_columns_actions_dag->mergeNodes(std::move(*alias_column_actions_dag)); + } + + for (const auto * output_node : action_dag_outputs) + merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node); + merged_alias_columns_actions_dag->removeUnusedActions(false); + + auto alias_column_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(merged_alias_columns_actions_dag)); + alias_column_step->setStepDescription("Compute alias columns"); + query_plan.addStep(std::move(alias_column_step)); + } + for (const auto & filter_info_and_description : where_filters) { const auto & [filter_info, description] = filter_info_and_description; @@ -907,7 +927,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres else { /// Create step which reads from empty source if storage has no data. - auto source_header = storage_snapshot->getSampleBlockForColumns(table_expression_data.getColumnNames()); + const auto & column_names = table_expression_data.getSelectedColumnsNames(); + auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); Pipe pipe(std::make_shared(source_header)); auto read_from_pipe = std::make_unique(std::move(pipe)); read_from_pipe->setStepDescription("Read from NullSource"); @@ -1024,57 +1045,6 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP plan_to_add_cast.addStep(std::move(cast_join_columns_step)); } -/// Actions to calculate table columns that have a functional representation (ALIASes and subcolumns) -/// and used in USING clause of JOIN expression. -struct UsingAliasKeyActions -{ - UsingAliasKeyActions( - const ColumnsWithTypeAndName & left_plan_output_columns, - const ColumnsWithTypeAndName & right_plan_output_columns - ) - : left_alias_columns_keys(std::make_shared(left_plan_output_columns)) - , right_alias_columns_keys(std::make_shared(right_plan_output_columns)) - {} - - void addLeftColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context) - { - addColumnImpl(left_alias_columns_keys, node, plan_output_columns, planner_context); - } - - void addRightColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context) - { - addColumnImpl(right_alias_columns_keys, node, plan_output_columns, planner_context); - } - - ActionsDAGPtr getLeftActions() - { - left_alias_columns_keys->projectInput(); - return std::move(left_alias_columns_keys); - } - - ActionsDAGPtr getRightActions() - { - right_alias_columns_keys->projectInput(); - return std::move(right_alias_columns_keys); - } - -private: - void addColumnImpl(ActionsDAGPtr & alias_columns_keys, QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context) - { - auto & column_node = node->as(); - if (column_node.hasExpression()) - { - auto dag = buildActionsDAGFromExpressionNode(column_node.getExpressionOrThrow(), plan_output_columns, planner_context); - const auto & left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(node); - dag->addOrReplaceInOutputs(dag->addAlias(*dag->getOutputs().front(), left_inner_column_identifier)); - alias_columns_keys->mergeInplace(std::move(*dag)); - } - } - - ActionsDAGPtr left_alias_columns_keys; - ActionsDAGPtr right_alias_columns_keys; -}; - JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan, @@ -1143,8 +1113,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_node.isUsingJoinExpression()) { - UsingAliasKeyActions using_alias_key_actions{left_plan_output_columns, right_plan_output_columns}; - auto & join_node_using_columns_list = join_node.getJoinExpression()->as(); for (auto & join_node_using_node : join_node_using_columns_list.getNodes()) { @@ -1154,13 +1122,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto & left_inner_column_node = inner_columns_list.getNodes().at(0); auto & left_inner_column = left_inner_column_node->as(); - using_alias_key_actions.addLeftColumn(left_inner_column_node, left_plan_output_columns, planner_context); - auto & right_inner_column_node = inner_columns_list.getNodes().at(1); auto & right_inner_column = right_inner_column_node->as(); - using_alias_key_actions.addRightColumn(right_inner_column_node, right_plan_output_columns, planner_context); - const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) { @@ -1174,14 +1138,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); } } - - auto left_alias_columns_keys_step = std::make_unique(left_plan.getCurrentDataStream(), using_alias_key_actions.getLeftActions()); - left_alias_columns_keys_step->setStepDescription("Actions for left table alias column keys"); - left_plan.addStep(std::move(left_alias_columns_keys_step)); - - auto right_alias_columns_keys_step = std::make_unique(right_plan.getCurrentDataStream(), using_alias_key_actions.getRightActions()); - right_alias_columns_keys_step->setStepDescription("Actions for right table alias column keys"); - right_plan.addStep(std::move(right_alias_columns_keys_step)); } auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 20c4f05ea7e..9ab7a8e64fe 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -55,7 +55,7 @@ public: /// Return true if column with name exists, false otherwise bool hasColumn(const std::string & column_name) const { - return alias_columns_names.contains(column_name) || column_name_to_column.contains(column_name); + return column_name_to_column.contains(column_name); } /** Add column in table expression data. @@ -63,37 +63,40 @@ public: * * Logical error exception is thrown if column already exists. */ - void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, bool is_selected_column = true) { if (hasColumn(column.name)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists", column.name); - addColumnImpl(column, column_identifier); + column_names.push_back(column.name); + addColumnImpl(column, column_identifier, is_selected_column); } - /** Add column if it does not exists in table expression data. - * Column identifier must be created using global planner context. - */ - void addColumnIfNotExists(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + /// Add alias column + void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAGPtr actions_dag, bool is_selected_column = true) { - if (hasColumn(column.name)) - return; - - addColumnImpl(column, column_identifier); + alias_column_expressions.emplace(column.name, std::move(actions_dag)); + addColumnImpl(column, column_identifier, is_selected_column); } - /// Add alias column name - void addAliasColumnName(const std::string & column_name, const ColumnIdentifier & column_identifier) + /// Mark existing column as selected + void markSelectedColumn(const std::string & column_name) { - alias_columns_names.insert(column_name); - - column_name_to_column_identifier.emplace(column_name, column_identifier); + auto [_, inserted] = selected_column_names_set.emplace(column_name); + if (inserted) + selected_column_names.push_back(column_name); } - /// Get alias columns names - const NameSet & getAliasColumnsNames() const + /// Get columns that are requested from table expression, including ALIAS columns + const Names & getSelectedColumnsNames() const { - return alias_columns_names; + return selected_column_names; + } + + /// Get ALIAS columns names mapped to expressions + const std::unordered_map & getAliasColumnExpressions() const + { + return alias_column_expressions; } /// Get column name to column map @@ -102,7 +105,7 @@ public: return column_name_to_column; } - /// Get column names + /// Get column names that are read from table expression const Names & getColumnNames() const { return column_names; @@ -119,23 +122,6 @@ public: return result; } - ColumnIdentifiers getColumnIdentifiers() const - { - ColumnIdentifiers result; - result.reserve(column_identifier_to_column_name.size()); - - for (const auto & [column_identifier, _] : column_identifier_to_column_name) - result.push_back(column_identifier); - - return result; - } - - /// Get column name to column identifier map - const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const - { - return column_name_to_column_identifier; - } - /// Get column identifier to column name map const ColumnNameToColumnIdentifier & getColumnIdentifierToColumnName() const { @@ -159,18 +145,6 @@ public: return it->second; } - /** Get column for column name. - * Null is returned if there are no column for column name. - */ - const NameAndTypePair * getColumnOrNull(const std::string & column_name) const - { - auto it = column_name_to_column.find(column_name); - if (it == column_name_to_column.end()) - return nullptr; - - return &it->second; - } - /** Get column identifier for column name. * Exception is thrown if there are no column identifier for column name. */ @@ -200,24 +174,6 @@ public: return &it->second; } - /** Get column name for column identifier. - * Exception is thrown if there are no column name for column identifier. - */ - const std::string & getColumnNameOrThrow(const ColumnIdentifier & column_identifier) const - { - auto it = column_identifier_to_column_name.find(column_identifier); - if (it == column_identifier_to_column_name.end()) - { - auto column_identifiers = getColumnIdentifiers(); - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Column name for column identifier {} does not exists. There are only column identifiers: {}", - column_identifier, - fmt::join(column_identifiers.begin(), column_identifiers.end(), ", ")); - } - - return it->second; - } - /** Get column name for column identifier. * Null is returned if there are no column name for column identifier. */ @@ -296,23 +252,36 @@ public: } private: - void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, bool add_to_selected_columns) { - column_names.push_back(column.name); + if (add_to_selected_columns) + markSelectedColumn(column.name); + column_name_to_column.emplace(column.name, column); column_name_to_column_identifier.emplace(column.name, column_identifier); column_identifier_to_column_name.emplace(column_identifier, column.name); } - /// Valid for table, table function, array join, query, union nodes + /// Set of columns that are physically read from table expression + /// In case of ALIAS columns it contains source column names that are used to calculate alias + /// This source column may be not used by user Names column_names; + /// Set of columns that are SELECTed from table expression + /// It may contain ALIAS columns. + /// Mainly it's used to determine access to which columns to check + /// For example user may have an access to column `a ALIAS x + y` but not to `x` and `y` + /// In that case we can read `x` and `y` and calculate `a`, but not return `x` and `y` to user + Names selected_column_names; + /// To deduplicate columns in `selected_column_names` + NameSet selected_column_names_set; + + /// Expression to calculate ALIAS columns + std::unordered_map alias_column_expressions; + /// Valid for table, table function, array join, query, union nodes ColumnNameToColumn column_name_to_column; - /// Valid only for table node - NameSet alias_columns_names; - /// Valid for table, table function, array join, query, union nodes ColumnNameToColumnIdentifier column_name_to_column_identifier; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 5f5875b8019..bd0b831ee58 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -469,12 +469,19 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); - auto filter_query_tree = buildQueryTree(filter_expression, query_context); QueryAnalysisPass query_analysis_pass(table_expression); query_analysis_pass.run(filter_query_tree, query_context); + return buildFilterInfo(std::move(filter_query_tree), table_expression, planner_context, std::move(table_expression_required_names_without_filter)); +} + +FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, + const QueryTreeNodePtr & table_expression, + PlannerContextPtr & planner_context, + NameSet table_expression_required_names_without_filter) +{ if (table_expression_required_names_without_filter.empty()) { auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); @@ -482,7 +489,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, table_expression_required_names_without_filter.insert(table_expression_names.begin(), table_expression_names.end()); } - collectSourceColumns(filter_query_tree, planner_context); + collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/); collectSets(filter_query_tree, *planner_context); auto filter_actions_dag = std::make_shared(); diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 3060b1c2711..bf45770552b 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -89,6 +89,11 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, PlannerContextPtr & planner_context, NameSet table_expression_required_names_without_filter = {}); +FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, + const QueryTreeNodePtr & table_expression, + PlannerContextPtr & planner_context, + NameSet table_expression_required_names_without_filter = {}); + ASTPtr parseAdditionalResultFilter(const Settings & settings); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index c7b9eb72d4d..331bd46f909 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1434,8 +1434,13 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) if (query_info.planner_context) { const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); + const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) { + /// ALIAS columns cannot be used in the filter expression without being calculated in ActionsDAG, + /// so they should not be added to the input nodes. + if (alias_column_expressions.contains(column_name)) + continue; const auto & column = table_expression_data.getColumnOrThrow(column_name); node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 92e7dcdf4f2..7370bd3ab8f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -744,6 +744,32 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( namespace { +class ReplaseAliasColumnsVisitor : public InDepthQueryTreeVisitor +{ + static QueryTreeNodePtr getColumnNodeAliasExpression(const QueryTreeNodePtr & node) + { + const auto * column_node = node->as(); + if (!column_node || !column_node->hasExpression()) + return nullptr; + + const auto & column_source = column_node->getColumnSourceOrNull(); + if (!column_source || column_source->getNodeType() == QueryTreeNodeType::JOIN + || column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN) + return nullptr; + + auto column_expression = column_node->getExpression(); + column_expression->setAlias(column_node->getColumnName()); + return column_expression; + } + +public: + void visitImpl(QueryTreeNodePtr & node) + { + if (auto column_expression = getColumnNodeAliasExpression(node)) + node = column_expression; + } +}; + QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, @@ -796,6 +822,8 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, replacement_table_expression->setAlias(query_info.table_expression->getAlias()); auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); + ReplaseAliasColumnsVisitor replase_alias_columns_visitor; + replase_alias_columns_visitor.visit(query_tree_to_modify); return buildQueryTreeForShard(query_info.planner_context, query_tree_to_modify); } diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 796ca6bca22..8f72fcd4050 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -4,4 +4,3 @@ test_distributed_type_object/test.py::test_distributed_type_object test_merge_table_over_distributed/test.py::test_global_in test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster -test_select_access_rights/test_main.py::test_alias_columns diff --git a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py index b620e88e7eb..c09a80cea06 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py +++ b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py @@ -41,7 +41,7 @@ def started_cluster(): CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0); - CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; + CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, bb ALIAS b + 1, c UInt16 ALIAS a + bb - 1) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1); CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 98653bf6106..8260be78e82 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -60,7 +60,7 @@ def started_cluster(): CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0); - CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; + CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, bb ALIAS b + 1, c UInt16 ALIAS a + bb - 1) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1); CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; @@ -113,6 +113,7 @@ def test_smoke(): assert node.query("SELECT a FROM mydb.filtered_table3") == TSV([[0], [1]]) assert node.query("SELECT b FROM mydb.filtered_table3") == TSV([[1], [0]]) + assert node.query("SELECT bb FROM mydb.filtered_table3") == TSV([[2], [1]]) assert node.query("SELECT c FROM mydb.filtered_table3") == TSV([[1], [1]]) assert node.query("SELECT a + b FROM mydb.filtered_table3") == TSV([[1], [1]]) assert node.query("SELECT a FROM mydb.filtered_table3 WHERE c = 1") == TSV( diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index a5a71560d00..2c62e278050 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -55,33 +55,33 @@ Header: a2 String Header: __table1.a2 String __table1.k UInt64 __table4.d2 String - Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) + Expression (DROP unused columns after JOIN) Header: __table1.a2 String __table1.k UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 - Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) + Expression (DROP unused columns after JOIN) Header: __table1.a2 String __table1.k UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 - Expression ((Actions for left table alias column keys + Change column names to column identifiers)) + Expression (Change column names to column identifiers) Header: __table1.a2 String __table1.k UInt64 ReadFromMemoryStorage Header: a2 String k UInt64 - Expression ((Actions for right table alias column keys + Change column names to column identifiers)) + Expression (Change column names to column identifiers) Header: __table2.k UInt64 ReadFromMemoryStorage Header: k UInt64 - Expression ((Actions for right table alias column keys + Change column names to column identifiers)) + Expression (Change column names to column identifiers) Header: __table3.k UInt64 ReadFromMemoryStorage Header: k UInt64 - Expression ((Actions for right table alias column keys + Change column names to column identifiers)) + Expression (Change column names to column identifiers) Header: __table4.d2 String __table4.k UInt64 ReadFromMemoryStorage diff --git a/tests/queries/0_stateless/02911_support_alias_column_in_indices.reference b/tests/queries/0_stateless/02911_support_alias_column_in_indices.reference index 883966ce6b5..b867a31dcc3 100644 --- a/tests/queries/0_stateless/02911_support_alias_column_in_indices.reference +++ b/tests/queries/0_stateless/02911_support_alias_column_in_indices.reference @@ -14,13 +14,13 @@ Expression ((Projection + Before ORDER BY)) Parts: 1/1 Granules: 1/1 Expression ((Project names + Projection)) - Filter ((WHERE + Change column names to column identifiers)) + Filter ((WHERE + (Change column names to column identifiers + Compute alias columns))) ReadFromMergeTree (02911_support_alias_column_in_indices.test1) Indexes: PrimaryKey Keys: c - Condition: (_CAST(plus(c, \'UInt64\'), 1) in [11, +Inf)) + Condition: (plus(c, 1) in [11, +Inf)) Parts: 1/2 Granules: 1/2 Skip @@ -44,12 +44,17 @@ Expression ((Projection + Before ORDER BY)) Parts: 1/1 Granules: 1/1 Expression ((Project names + Projection)) - Filter ((WHERE + Change column names to column identifiers)) + Filter ((WHERE + (Change column names to column identifiers + Compute alias columns))) ReadFromMergeTree (02911_support_alias_column_in_indices.test2) Indexes: PrimaryKey Keys: c - Condition: (_CAST(plus(_CAST(plus(c, \'UInt64\'), 1), \'UInt64\'), 1) in [16, +Inf)) + Condition: (plus(plus(c, 1), 1) in [16, +Inf)) Parts: 1/2 Granules: 1/2 + Skip + Name: i + Description: minmax GRANULARITY 1 + Parts: 1/1 + Granules: 1/1 From 361ec17c871d69cccb0d2994ea312533b7d0272a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 1 Mar 2024 15:31:25 +0000 Subject: [PATCH 082/197] gdb... --- docker/test/fuzzer/run-fuzzer.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index b4376fe2409..1a1888861af 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -247,6 +247,9 @@ quit fuzzer_pid=$! echo "Fuzzer pid is $fuzzer_pid" + echo "Attaching gdb to the fuzzer itself" + gdb -batch -command script.gdb -p $fuzzer_pid & + # Wait for the fuzzer to complete. # Note that the 'wait || ...' thing is required so that the script doesn't # exit because of 'set -e' when 'wait' returns nonzero code. From 0b10612c863bf5b62bcf90028daa57275a966b6a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 1 Mar 2024 17:42:57 +0000 Subject: [PATCH 083/197] fix --- src/Functions/FunctionBinaryArithmetic.h | 23 ++++++++++--------- src/Functions/IsOperation.h | 6 ++--- ...02_int_div_decimal_with_date_bug.reference | 0 .../03002_int_div_decimal_with_date_bug.sql | 5 ++++ 4 files changed, 20 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/03002_int_div_decimal_with_date_bug.reference create mode 100644 tests/queries/0_stateless/03002_int_div_decimal_with_date_bug.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index d253095ca01..9ad74f6332f 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -170,7 +170,8 @@ public: /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). using ResultDataType = Switch< /// Result must be Integer - Case::div_int || IsOperation::div_int_or_zero, DataTypeFromFieldType>, + Case::int_div || IsOperation::int_div_or_zero, + std::conditional_t && IsDataTypeNumber, DataTypeFromFieldType, InvalidType>>, /// Decimal cases Case || IsDataTypeDecimal, DecimalResultDataType>, Case< @@ -672,8 +673,8 @@ private: IsOperation::minus; static constexpr bool is_multiply = IsOperation::multiply; static constexpr bool is_float_division = IsOperation::div_floating; - static constexpr bool is_int_division = IsOperation::div_int || - IsOperation::div_int_or_zero; + static constexpr bool is_int_division = IsOperation::int_div || + IsOperation::int_div_or_zero; static constexpr bool is_division = is_float_division || is_int_division; static constexpr bool is_compare = IsOperation::least || IsOperation::greatest; @@ -781,8 +782,8 @@ class FunctionBinaryArithmetic : public IFunction static constexpr bool is_division = IsOperation::division; static constexpr bool is_bit_hamming_distance = IsOperation::bit_hamming_distance; static constexpr bool is_modulo = IsOperation::modulo; - static constexpr bool is_div_int = IsOperation::div_int; - static constexpr bool is_div_int_or_zero = IsOperation::div_int_or_zero; + static constexpr bool is_int_div = IsOperation::int_div; + static constexpr bool is_int_div_or_zero = IsOperation::int_div_or_zero; ContextPtr context; bool check_decimal_overflow = true; @@ -1007,11 +1008,11 @@ class FunctionBinaryArithmetic : public IFunction { function_name = "tupleModuloByNumber"; } - else if constexpr (is_div_int) + else if constexpr (is_int_div) { function_name = "tupleIntDivByNumber"; } - else if constexpr (is_div_int_or_zero) + else if constexpr (is_int_div_or_zero) { function_name = "tupleIntDivOrZeroByNumber"; } @@ -1466,7 +1467,7 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override { - return ((IsOperation::div_int || IsOperation::modulo || IsOperation::positive_modulo) && !arguments[1].is_const) + return ((IsOperation::int_div || IsOperation::modulo || IsOperation::positive_modulo) && !arguments[1].is_const) || (IsOperation::div_floating && (isDecimalOrNullableDecimal(arguments[0].type) || isDecimalOrNullableDecimal(arguments[1].type))); } @@ -1690,7 +1691,7 @@ public: if constexpr (!std::is_same_v) { - if constexpr (is_div_int || is_div_int_or_zero) + if constexpr (is_int_div || is_int_div_or_zero) type_res = std::make_shared(); else if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) { @@ -2086,7 +2087,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A right_nullmap); } /// Here we check if we have `intDiv` or `intDivOrZero` and at least one of the arguments is decimal, because in this case originally we had result as decimal, so we need to convert result into integer after calculations - else if constexpr (!decimal_with_float && (is_div_int || is_div_int_or_zero) && (IsDataTypeDecimal || IsDataTypeDecimal)) + else if constexpr (!decimal_with_float && (is_int_div || is_int_div_or_zero) && (IsDataTypeDecimal || IsDataTypeDecimal)) { if constexpr (!std::is_same_v) @@ -2624,7 +2625,7 @@ public: /// Check the case when operation is divide, intDiv or modulo and denominator is Nullable(Something). /// For divide operation we should check only Nullable(Decimal), because only this case can throw division by zero error. bool division_by_nullable = !arguments[0].type->onlyNull() && !arguments[1].type->onlyNull() && arguments[1].type->isNullable() - && (IsOperation::div_int || IsOperation::modulo || IsOperation::positive_modulo + && (IsOperation::int_div || IsOperation::modulo || IsOperation::positive_modulo || (IsOperation::div_floating && (isDecimalOrNullableDecimal(arguments[0].type) || isDecimalOrNullableDecimal(arguments[1].type)))); diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index b2c7a27d375..a74df8f4dd9 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -51,8 +51,8 @@ struct IsOperation static constexpr bool minus = IsSameOperation::value; static constexpr bool multiply = IsSameOperation::value; static constexpr bool div_floating = IsSameOperation::value; - static constexpr bool div_int = IsSameOperation::value; - static constexpr bool div_int_or_zero = IsSameOperation::value; + static constexpr bool int_div = IsSameOperation::value; + static constexpr bool int_div_or_zero = IsSameOperation::value; static constexpr bool modulo = IsSameOperation::value; static constexpr bool positive_modulo = IsSameOperation::value; static constexpr bool least = IsSameOperation::value; @@ -60,7 +60,7 @@ struct IsOperation static constexpr bool bit_hamming_distance = IsSameOperation::value; - static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo; + static constexpr bool division = div_floating || int_div || int_div_or_zero || modulo; // NOTE: allow_decimal should not fully contain `division` because of divInt static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; }; diff --git a/tests/queries/0_stateless/03002_int_div_decimal_with_date_bug.reference b/tests/queries/0_stateless/03002_int_div_decimal_with_date_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03002_int_div_decimal_with_date_bug.sql b/tests/queries/0_stateless/03002_int_div_decimal_with_date_bug.sql new file mode 100644 index 00000000000..1668821200c --- /dev/null +++ b/tests/queries/0_stateless/03002_int_div_decimal_with_date_bug.sql @@ -0,0 +1,5 @@ +SELECT intDiv(CAST('1.0', 'Decimal256(3)'), today()); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT intDiv(CAST('1.0', 'Decimal256(3)'), toDate('2023-01-02')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT intDiv(CAST('1.0', 'Decimal256(2)'), toDate32('2023-01-02 12:12:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT intDiv(CAST('1.0', 'Decimal256(2)'), toDateTime('2023-01-02 12:12:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT intDiv(CAST('1.0', 'Decimal256(2)'), toDateTime64('2023-01-02 12:12:12.002', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 95611889dc0f4b877c6c9bb914831041cd26f2c0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 1 Mar 2024 18:44:33 +0000 Subject: [PATCH 084/197] fix tests --- src/Interpreters/TreeRewriter.cpp | 23 ++++++++++++------- src/Interpreters/inplaceBlockConversions.cpp | 1 - src/Storages/AlterCommands.cpp | 1 - src/Storages/FileLog/StorageFileLog.cpp | 1 - src/Storages/FileLog/StorageFileLog.h | 1 - src/Storages/HDFS/StorageHDFS.cpp | 5 ---- src/Storages/HDFS/StorageHDFS.h | 2 -- src/Storages/IStorage.cpp | 1 - src/Storages/Kafka/StorageKafka.cpp | 1 - src/Storages/Kafka/StorageKafka.h | 1 - src/Storages/LiveView/StorageLiveView.cpp | 1 - src/Storages/MergeTree/IMergeTreeReader.cpp | 4 ---- .../MergeTree/MergeTreeBlockReadUtils.cpp | 3 +-- src/Storages/MergeTree/MergeTreeData.cpp | 2 -- src/Storages/MergeTree/MergeTreeData.h | 2 -- .../MergeTree/MergeTreeDataPartCompact.cpp | 1 - .../MergeTree/MergeTreeDataPartWriterWide.cpp | 1 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - .../MergeTree/MergeTreeSelectProcessor.cpp | 1 - .../MergeTree/MergeTreeSelectProcessor.h | 1 - src/Storages/MergeTreeVirtualColumns.cpp | 2 +- .../StorageMaterializedPostgreSQL.cpp | 1 - .../StorageMaterializedPostgreSQL.h | 1 - src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 1 - src/Storages/RabbitMQ/StorageRabbitMQ.h | 1 - src/Storages/StorageAzureBlob.cpp | 5 ---- src/Storages/StorageAzureBlob.h | 2 -- src/Storages/StorageDistributed.cpp | 3 --- src/Storages/StorageDistributed.h | 1 - src/Storages/StorageFile.cpp | 5 ---- src/Storages/StorageFile.h | 2 -- src/Storages/StorageKeeperMap.cpp | 1 - src/Storages/StorageLog.cpp | 1 - src/Storages/StorageMaterializedView.cpp | 2 -- src/Storages/StorageMerge.cpp | 2 -- src/Storages/StorageMerge.h | 1 - src/Storages/StorageS3.cpp | 5 ---- src/Storages/StorageS3.h | 2 -- src/Storages/StorageSnapshot.cpp | 1 - src/Storages/StorageURL.cpp | 5 ---- src/Storages/StorageURL.h | 2 -- .../System/StorageSystemDictionaries.cpp | 3 --- .../System/StorageSystemDictionaries.h | 1 - .../System/StorageSystemPartsBase.cpp | 1 - src/Storages/VirtualColumnUtils.cpp | 2 +- src/Storages/VirtualColumnUtils.h | 3 +-- src/Storages/VirtualColumnsDescription.cpp | 3 --- src/TableFunctions/ITableFunctionFileLike.cpp | 6 +++++ src/TableFunctions/ITableFunctionFileLike.h | 3 +++ .../TableFunctionAzureBlobStorage.cpp | 4 ++-- src/TableFunctions/TableFunctionFile.cpp | 6 ----- src/TableFunctions/TableFunctionFile.h | 2 -- src/TableFunctions/TableFunctionHDFS.cpp | 6 ----- src/TableFunctions/TableFunctionHDFS.h | 2 -- src/TableFunctions/TableFunctionS3.cpp | 4 ++-- src/TableFunctions/TableFunctionURL.cpp | 6 ----- src/TableFunctions/TableFunctionURL.h | 2 -- 57 files changed, 32 insertions(+), 122 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 896182c1f0f..5588fc55a64 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -992,7 +992,6 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) auto options = GetColumnsOptions(add_special ? GetColumnsOptions::All : GetColumnsOptions::AllPhysical); options.withExtendedObjects(); options.withSubcolumns(storage->supportsSubcolumns()); - options.withVirtuals(); auto columns_from_storage = storage_snapshot->getColumns(options); @@ -1002,8 +1001,7 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) source_columns.insert(source_columns.end(), columns_from_storage.begin(), columns_from_storage.end()); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto metadata_column_descriptions = metadata_snapshot->getColumns(); - source_columns_ordinary = metadata_column_descriptions.getOrdinary(); + source_columns_ordinary = metadata_snapshot->getColumns().getOrdinary(); } source_columns_set = removeDuplicateColumns(source_columns); @@ -1142,16 +1140,25 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select } has_virtual_shard_num = false; - if (is_remote_storage) + /// If there are virtual columns among the unknown columns. Remove them from the list of unknown and add + /// in columns list, so that when further processing they are also considered. + if (storage_snapshot) { - for (const auto & column : *storage_snapshot->virtual_columns) + const auto & virtuals = storage_snapshot->virtual_columns; + for (auto it = unknown_required_source_columns.begin(); it != unknown_required_source_columns.end();) { - if (column.name == "_shard_num" && storage->isVirtualColumn("_shard_num", storage_snapshot->getMetadataForQuery())) + if (auto column = virtuals->tryGet(*it)) { - has_virtual_shard_num = true; - break; + source_columns.push_back(*column); + it = unknown_required_source_columns.erase(it); + } + else + { + ++it; } } + + has_virtual_shard_num = is_remote_storage && storage->isVirtualColumn("_shard_num", storage_snapshot->getMetadataForQuery()) && virtuals->has("_shard_num"); } /// Collect missed object subcolumns diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 061156c56db..239cce5b427 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -20,7 +20,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index acd95a2b8d7..e1c80112c72 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -31,7 +31,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 2911f9b268b..a5f2331a068 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -27,7 +27,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ede36600f92..91d58540c94 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -9,7 +9,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 4a481a6a7d2..19b2817510d 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -1178,11 +1178,6 @@ void registerStorageHDFS(StorageFactory & factory) }); } -Names StorageHDFS::getVirtualColumnNames() -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); -} - SchemaCache & StorageHDFS::getSchemaCache(const ContextPtr & ctx) { static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_hdfs", DEFAULT_SCHEMA_CACHE_ELEMENTS)); diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index e62fc180a6b..47e5addccb4 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -69,8 +69,6 @@ public: ContextPtr local_context, TableExclusiveLockHolder &) override; - static Names getVirtualColumnNames(); - bool supportsPartitionBy() const override { return true; } /// Check if the format is column-oriented. diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 18a934af767..9852220241f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -1,7 +1,6 @@ #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include #include diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 4b87b1f7b74..c63ebdfa28f 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -52,7 +52,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #if USE_KRB5 diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 6a14bd64cc6..605291fdd8c 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -7,7 +7,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 631c88a4dad..cef385c6f98 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -27,7 +27,6 @@ limitations under the License. */ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index dd55a6acb2c..8bef26ca240 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -2,11 +2,7 @@ #include #include #include -#include #include -#include -#include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index f97e07751e0..7d54d3867ac 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -6,8 +6,7 @@ #include #include #include -#include "Storages/ColumnsDescription.h" -#include "Storages/MergeTreeVirtualColumns.h" +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e52362ff76a..0b7ac39aa1b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,7 +8,6 @@ #include #include #include -#include "Common/logger_useful.h" #include #include #include @@ -23,7 +22,6 @@ #include #include #include -#include "Storages/ProjectionsDescription.h" #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index f7bde252fb9..3a685a4c397 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -37,8 +37,6 @@ #include #include #include -#include "Storages/ProjectionsDescription.h" -#include "Storages/VirtualColumnsDescription.h" #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index ee34a02b0b3..9f201ab3b81 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 3a646e0b85d..d79590ded21 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ac0b104687d..428c8f92931 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -46,7 +46,6 @@ #include #include -#include #include namespace CurrentMetrics diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index e3900ccdd73..8a24e150bae 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -6,7 +6,6 @@ #include #include #include -#include "Storages/StorageSnapshot.h" #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index e25dfba4327..01bb3851e04 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -5,7 +5,6 @@ #include #include #include -#include "Storages/StorageSnapshot.h" namespace DB diff --git a/src/Storages/MergeTreeVirtualColumns.cpp b/src/Storages/MergeTreeVirtualColumns.cpp index 1a576bef017..885e46c6828 100644 --- a/src/Storages/MergeTreeVirtualColumns.cpp +++ b/src/Storages/MergeTreeVirtualColumns.cpp @@ -1,8 +1,8 @@ #include +#include #include #include #include -#include namespace DB { diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 2914c17b117..c9866146a8b 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -1,5 +1,4 @@ #include "StorageMaterializedPostgreSQL.h" -#include "Storages/VirtualColumnsDescription.h" #if USE_LIBPQXX #include diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index 41f72ea79e1..af2f13bb880 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -1,6 +1,5 @@ #pragma once -#include "Storages/VirtualColumnsDescription.h" #include "config.h" #if USE_LIBPQXX diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index c5df752fb7f..980fccd307e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -30,7 +30,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" namespace DB { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 603a1f1d68f..e14741d9636 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -9,7 +9,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include #include diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index a74e274e5d9..92dbc1025db 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -886,11 +886,6 @@ SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMeta } } -Names StorageAzureBlob::getVirtualColumnNames() -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); -} - bool StorageAzureBlob::supportsPartitionBy() const { return true; diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 5571ec6d902..ae70b111fb0 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -94,8 +94,6 @@ public: void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; - static Names getVirtualColumnNames(); - bool supportsPartitionBy() const override; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 541b2934892..ce695d5f009 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -31,7 +31,6 @@ #include #include #include -#include "Storages/StorageInMemoryMetadata.h" #include #include @@ -106,8 +105,6 @@ #include #include -#include - #include #include #include diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 51e4ccd4da3..cf5b78305a5 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -9,7 +9,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8eaf794f445..a734d3967eb 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2255,9 +2255,4 @@ StorageFile::ArchiveInfo StorageFile::getArchiveInfo( return archive_info; } -Names StorageFile::getVirtualColumnNames() -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); -} - } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 5535dba36e5..93c263008a6 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -80,8 +80,6 @@ public: bool storesDataOnDisk() const override; Strings getDataPaths() const override; - static Names getVirtualColumnNames(); - static Strings getPathsList(const String & table_path, const String & user_files_path, const ContextPtr & context, size_t & total_bytes_to_read); /// Check if the format supports reading only some subset of columns. diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 6c7a8e06cc3..69e6085bab9 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -41,7 +41,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 25cc38f7834..7d959b05c94 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -22,7 +22,6 @@ #include #include "StorageLogSettings.h" -#include "Storages/StorageSnapshot.h" #include #include #include diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 7e953889b34..aeba01fdf8a 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -22,7 +21,6 @@ #include #include -#include "Storages/StorageSnapshot.h" #include #include #include diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0ddfc2a6bb4..00513d1ec46 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include @@ -61,7 +60,6 @@ #include #include #include -#include "Storages/StorageSnapshot.h" #include #include diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 661750fb6dd..556649f622d 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -5,7 +5,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" namespace DB diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 691d197aee6..afd34792335 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1975,11 +1975,6 @@ void registerStorageOSS(StorageFactory & factory) return registerStorageS3Impl("OSS", factory); } -Names StorageS3::getVirtualColumnNames() -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); -} - bool StorageS3::supportsPartitionBy() const { return true; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 29a4e46eb9a..7c5db76aadf 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -336,8 +336,6 @@ public: void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; - static Names getVirtualColumnNames(); - bool supportsPartitionBy() const override; static void processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection); diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 71d2809e18a..1893d65a64c 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -5,7 +5,6 @@ #include #include #include -#include "Storages/ColumnsDescription.h" namespace DB { diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index e0c5160a43b..ec5b6008c17 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1302,11 +1302,6 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad } } -Names IStorageURLBase::getVirtualColumnNames() -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); -} - SchemaCache & IStorageURLBase::getSchemaCache(const ContextPtr & context) { static SchemaCache schema_cache(context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_url", DEFAULT_SCHEMA_CACHE_ELEMENTS)); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 36cfa97463c..294b1f828bb 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -48,8 +48,6 @@ public: bool supportsPartitionBy() const override { return true; } - static Names getVirtualColumnNames(); - static ColumnsDescription getTableStructureFromData( const String & format, const String & uri, diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index 353c61e6347..d60cfcafc13 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -16,9 +16,6 @@ #include #include -#include "Storages/System/IStorageSystemOneBlock.h" -#include "Storages/VirtualColumnsDescription.h" -#include namespace DB { diff --git a/src/Storages/System/StorageSystemDictionaries.h b/src/Storages/System/StorageSystemDictionaries.h index e4f07e3c4bf..60b17af4a45 100644 --- a/src/Storages/System/StorageSystemDictionaries.h +++ b/src/Storages/System/StorageSystemDictionaries.h @@ -1,7 +1,6 @@ #pragma once #include -#include "Interpreters/StorageID.h" namespace DB diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index b4900d26470..075e1c62323 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -2,7 +2,6 @@ #include #include #include -#include "Storages/VirtualColumnsDescription.h" #include #include #include diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index c2b61c49e4a..21f05953714 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -343,7 +343,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex } } -Names getVirtualNamesForFileLikeStorage() +NameSet getVirtualNamesForFileLikeStorage() { return {"_path", "_file", "_size"}; } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index a3aa3b6adc6..3a83874c13e 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -5,7 +5,6 @@ #include #include #include -#include "Storages/ColumnsDescription.h" #include @@ -58,7 +57,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) return res; } -Names getVirtualNamesForFileLikeStorage(); +NameSet getVirtualNamesForFileLikeStorage(); VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); diff --git a/src/Storages/VirtualColumnsDescription.cpp b/src/Storages/VirtualColumnsDescription.cpp index d9f9cbe047e..5dd249c5190 100644 --- a/src/Storages/VirtualColumnsDescription.cpp +++ b/src/Storages/VirtualColumnsDescription.cpp @@ -1,6 +1,3 @@ -#include "Core/NamesAndTypes.h" -#include "DataTypes/Serializations/ISerialization.h" -#include "base/types.h" #include namespace DB diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index b697f3df925..1a58be4f75b 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -7,6 +7,7 @@ #include #include +#include #include @@ -37,6 +38,11 @@ bool ITableFunctionFileLike::supportsReadingSubsetOfColumns(const ContextPtr & c return format != "auto" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format, context); } +NameSet ITableFunctionFileLike::getVirtualsToCheckBeforeUsingStructureHint() const +{ + return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); +} + void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context) { /// Parse args diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index c8412905e44..ba1b7d2bb3f 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -1,6 +1,7 @@ #pragma once #include +#include "Core/Names.h" #include "Parsers/IAST_fwd.h" namespace DB @@ -29,6 +30,8 @@ public: bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; + NameSet getVirtualsToCheckBeforeUsingStructureHint() const override; + static size_t getMaxNumberOfArguments() { return 4; } static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr &); diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 066d6338b6a..f9645b8323d 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include "registerTableFunctions.h" #include @@ -348,8 +349,7 @@ bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns(const Context std::unordered_set TableFunctionAzureBlobStorage::getVirtualsToCheckBeforeUsingStructureHint() const { - auto virtual_column_names = StorageAzureBlob::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; + return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); } StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index b481076e9b6..28bf72e07fb 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -112,12 +112,6 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context return parseColumnsListFromString(structure, context); } -std::unordered_set TableFunctionFile::getVirtualsToCheckBeforeUsingStructureHint() const -{ - auto virtual_column_names = StorageFile::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; -} - void registerTableFunctionFile(TableFunctionFactory & factory) { factory.registerFunction(); diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index c1924028b49..aaf5ba8873a 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -22,8 +22,6 @@ public: ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - protected: int fd = -1; String path_to_archive; diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index 2dac4398144..45829245551 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -41,12 +41,6 @@ ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context return parseColumnsListFromString(structure, context); } -std::unordered_set TableFunctionHDFS::getVirtualsToCheckBeforeUsingStructureHint() const -{ - auto virtual_column_names = StorageHDFS::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; -} - void registerTableFunctionHDFS(TableFunctionFactory & factory) { factory.registerFunction(); diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index 3a719496b26..f1c0b8a7eae 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -36,8 +36,6 @@ public: ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - private: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 3fedd38277c..7af1675e19e 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include "registerTableFunctions.h" #include @@ -401,8 +402,7 @@ bool TableFunctionS3::supportsReadingSubsetOfColumns(const ContextPtr & context) std::unordered_set TableFunctionS3::getVirtualsToCheckBeforeUsingStructureHint() const { - auto virtual_column_names = StorageS3::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; + return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); } StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool /*is_insert_query*/) const diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index a78b2affa9a..2bdc0b449e0 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -159,12 +159,6 @@ ColumnsDescription TableFunctionURL::getActualTableStructure(ContextPtr context, return parseColumnsListFromString(structure, context); } -std::unordered_set TableFunctionURL::getVirtualsToCheckBeforeUsingStructureHint() const -{ - auto virtual_column_names = StorageURL::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; -} - std::optional TableFunctionURL::tryGetFormatFromFirstArgument() { return FormatFactory::instance().tryGetFormatFromFileName(Poco::URI(filename).getPath()); diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 54e223283ba..a1efddb84c6 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -36,8 +36,6 @@ public: static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context); - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - protected: void parseArguments(const ASTPtr & ast, ContextPtr context) override; void parseArgumentsImpl(ASTs & args, const ContextPtr & context) override; From 1a1e78cf63b994dfb837a00602378c78c3e4c011 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 1 Mar 2024 18:52:14 +0000 Subject: [PATCH 085/197] temporary remove virtual column --- src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 10 ---------- src/Storages/MergeTreeVirtualColumns.cpp | 4 ---- src/Storages/MergeTreeVirtualColumns.h | 7 ------- 5 files changed, 2 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 8bef26ca240..30be1aa1c56 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -93,7 +93,7 @@ void IMergeTreeReader::fillVirtualColumns(Columns & columns, size_t rows) const it->name, it->type->getName(), virtual_column->type->getName()); } - if (it->name == "_part_offset" || it->name == BlockOffsetColumn::name) + if (it->name == "_part_offset") throw Exception(ErrorCodes::LOGICAL_ERROR, "Virtual column {} must be filled by range reader", it->name); Field field; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 7d54d3867ac..e84ed0a8068 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -278,7 +278,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( .withVirtuals() .withSubcolumns(with_subcolumns); - static const NameSet columns_to_read_at_first_step = {"_part_offset", BlockOffsetColumn::name}; + static const NameSet columns_to_read_at_first_step = {"_part_offset"}; NameSet columns_from_previous_steps; auto add_step = [&](const PrewhereExprStep & step) diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index fb0bc617aa4..2eb010c54ec 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -210,16 +210,6 @@ static void fillBlockNumberColumns( { res_columns[i] = BlockNumberColumn::type->createColumnConst(num_rows, block_number)->convertToFullColumnIfConst(); } - else if (it->name == BlockOffsetColumn::name) - { - auto column = BlockOffsetColumn::type->createColumn(); - auto & block_offset_data = assert_cast(*column).getData(); - - block_offset_data.resize(num_rows); - std::iota(block_offset_data.begin(), block_offset_data.end(), block_offset); - - res_columns[i] = std::move(column); - } } } diff --git a/src/Storages/MergeTreeVirtualColumns.cpp b/src/Storages/MergeTreeVirtualColumns.cpp index 885e46c6828..8250ceda7fa 100644 --- a/src/Storages/MergeTreeVirtualColumns.cpp +++ b/src/Storages/MergeTreeVirtualColumns.cpp @@ -26,10 +26,6 @@ const String BlockNumberColumn::name = "_block_number"; const DataTypePtr BlockNumberColumn::type = std::make_shared(); const ASTPtr BlockNumberColumn::codec = getCompressionCodecDeltaLZ4(); -const String BlockOffsetColumn::name = "_block_offset"; -const DataTypePtr BlockOffsetColumn::type = std::make_shared(); -const ASTPtr BlockOffsetColumn::codec = getCompressionCodecDeltaLZ4(); - Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part) { if (column_name == RowExistsColumn::name) diff --git a/src/Storages/MergeTreeVirtualColumns.h b/src/Storages/MergeTreeVirtualColumns.h index cd9fe544ed8..24721bf1ad1 100644 --- a/src/Storages/MergeTreeVirtualColumns.h +++ b/src/Storages/MergeTreeVirtualColumns.h @@ -21,13 +21,6 @@ struct BlockNumberColumn static const ASTPtr codec; }; -struct BlockOffsetColumn -{ - static const String name; - static const DataTypePtr type; - static const ASTPtr codec; -}; - Field getFieldForConstVirtualColumn(const String & column_name, const IMergeTreeDataPart & part); } From 754ae8792dcfd48219e479a71afc6f23a260d3ec Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 1 Mar 2024 19:29:42 +0000 Subject: [PATCH 086/197] Always apply first minmax index among available skip indices --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 13 +++++++++++++ src/Storages/IndicesDescription.h | 1 - src/Storages/MergeTree/MergeTreeIndexHypothesis.h | 1 - src/Storages/MergeTree/MergeTreeIndexMinMax.h | 2 -- src/Storages/MergeTree/MergeTreeIndexSet.h | 3 --- src/Storages/MergeTree/MergeTreeIndices.h | 3 --- src/Storages/MergeTree/MergeTreeReaderStream.h | 1 - .../03000_minmax_index_first.reference | 2 ++ .../0_stateless/03000_minmax_index_first.sql | 15 +++++++++++++++ 9 files changed, 30 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/03000_minmax_index_first.reference create mode 100644 tests/queries/0_stateless/03000_minmax_index_first.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index c7b9eb72d4d..8f0db044d8f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -43,6 +43,7 @@ #include #include #include +#include #include #include @@ -1421,6 +1422,18 @@ static void buildIndexes( } } + // move minmax indices to first positions, so they will be applied first as cheapest ones + std::sort(begin(skip_indexes.useful_indices), end(skip_indexes.useful_indices), [](const auto & l, const auto & r) + { + if (typeid_cast(l.index.get())) + return true; // left is min max + + if (typeid_cast(r.index.get())) + return false; // right is min max but left is not + + return true; + }); + indexes->skip_indexes = std::move(skip_indexes); } diff --git a/src/Storages/IndicesDescription.h b/src/Storages/IndicesDescription.h index e56642b8c76..21ba5fb632e 100644 --- a/src/Storages/IndicesDescription.h +++ b/src/Storages/IndicesDescription.h @@ -2,7 +2,6 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index 2296e1b717d..130e708d76f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.h b/src/Storages/MergeTree/MergeTreeIndexMinMax.h index 1e2abe6983f..dca26fb7b28 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.h +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.h @@ -4,8 +4,6 @@ #include #include -#include - namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index ea9f7ddef3d..7c66ba1a867 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -5,9 +5,6 @@ #include -#include -#include - namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 4749470bedd..8fdadb4e5eb 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -1,12 +1,9 @@ #pragma once #include -#include #include #include #include -#include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.h b/src/Storages/MergeTree/MergeTreeReaderStream.h index baf8ec713f9..49ce3103434 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.h +++ b/src/Storages/MergeTree/MergeTreeReaderStream.h @@ -1,5 +1,4 @@ #pragma once -#include #include #include #include diff --git a/tests/queries/0_stateless/03000_minmax_index_first.reference b/tests/queries/0_stateless/03000_minmax_index_first.reference new file mode 100644 index 00000000000..7cf792d8ed4 --- /dev/null +++ b/tests/queries/0_stateless/03000_minmax_index_first.reference @@ -0,0 +1,2 @@ +Name: v_mm +Name: v_set diff --git a/tests/queries/0_stateless/03000_minmax_index_first.sql b/tests/queries/0_stateless/03000_minmax_index_first.sql new file mode 100644 index 00000000000..ee2cb2f4dfa --- /dev/null +++ b/tests/queries/0_stateless/03000_minmax_index_first.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS skip_table; + +CREATE TABLE skip_table +( + k UInt64, + v UInt64, + INDEX v_set v TYPE set(100) GRANULARITY 2, + INDEX v_mm v TYPE minmax GRANULARITY 2 +) +ENGINE = MergeTree +PRIMARY KEY k; + +INSERT INTO skip_table SELECT number, intDiv(number, 4096) FROM numbers(1000000); + +SELECT trim(explain) FROM ( EXPLAIN indexes = 1 SELECT * FROM skip_table WHERE v = 125) WHERE explain ilike '%Name%'; From 4baa60d5ffd94dffba89394d38d7514782a9cb07 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 1 Mar 2024 19:36:05 +0000 Subject: [PATCH 087/197] Comment in test --- tests/queries/0_stateless/03000_minmax_index_first.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03000_minmax_index_first.sql b/tests/queries/0_stateless/03000_minmax_index_first.sql index ee2cb2f4dfa..4db232880de 100644 --- a/tests/queries/0_stateless/03000_minmax_index_first.sql +++ b/tests/queries/0_stateless/03000_minmax_index_first.sql @@ -4,7 +4,7 @@ CREATE TABLE skip_table ( k UInt64, v UInt64, - INDEX v_set v TYPE set(100) GRANULARITY 2, + INDEX v_set v TYPE set(100) GRANULARITY 2, -- set index is declared before minmax intentionally INDEX v_mm v TYPE minmax GRANULARITY 2 ) ENGINE = MergeTree From aa42d5a88da0c16706b95631285db77d2dca9bdf Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 1 Mar 2024 20:58:22 +0000 Subject: [PATCH 088/197] separate FunctionSecretArgumentsFinder for AST and TreeNode to avoid binary bloat due to linkage dependencies --- .../FunctionSecretArgumentsFinderTreeNode.h | 1 + src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 ++-- src/Parsers/ASTFunction.cpp | 6 +++--- src/Parsers/FunctionSecretArgumentsFinder.cpp | 19 ------------------- src/Parsers/FunctionSecretArgumentsFinder.h | 6 +----- 5 files changed, 7 insertions(+), 29 deletions(-) delete mode 100644 src/Parsers/FunctionSecretArgumentsFinder.cpp diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 7e9a31868b2..439ddffe5e5 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -7,6 +7,7 @@ #include #include #include +#include #include diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 819a3758e26..10866e5eed4 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include @@ -5127,7 +5127,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi /// Mask arguments if needed if (!scope.context->getSettingsRef().format_display_secrets_in_show_and_select) { - if (FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinder::find(*function_node_ptr); secret_arguments.count) + if (FunctionSecretArgumentsFinder::Result secret_arguments = FunctionSecretArgumentsFinderTreeNode(*function_node_ptr).getResult(); secret_arguments.count) { auto & argument_nodes = function_node_ptr->getArgumentsNode()->as().getNodes(); diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 4dac9c090f6..4537ced14cf 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include @@ -693,7 +693,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format FunctionSecretArgumentsFinder::Result secret_arguments; if (!settings.show_secrets) - secret_arguments = FunctionSecretArgumentsFinder::find(*this); + secret_arguments = FunctionSecretArgumentsFinderAST(*this).getResult(); for (size_t i = 0, size = arguments->children.size(); i < size; ++i) { @@ -757,7 +757,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format bool ASTFunction::hasSecretParts() const { - return (FunctionSecretArgumentsFinder::find(*this).hasSecrets()) || childrenHaveSecretParts(); + return (FunctionSecretArgumentsFinderAST(*this).getResult().hasSecrets()) || childrenHaveSecretParts(); } String getFunctionName(const IAST * ast) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.cpp b/src/Parsers/FunctionSecretArgumentsFinder.cpp deleted file mode 100644 index bdeb29a37ba..00000000000 --- a/src/Parsers/FunctionSecretArgumentsFinder.cpp +++ /dev/null @@ -1,19 +0,0 @@ -#include -#include -#include - - -namespace DB -{ - -FunctionSecretArgumentsFinder::Result FunctionSecretArgumentsFinder::find(const ASTFunction & function) -{ - return FunctionSecretArgumentsFinderAST(function).getResult(); -} - -FunctionSecretArgumentsFinder::Result FunctionSecretArgumentsFinder::find(const FunctionNode & function) -{ - return FunctionSecretArgumentsFinderTreeNode(function).getResult(); -} - -} diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 950d913fe20..002ad94f6ea 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -1,7 +1,6 @@ #pragma once -#include -#include +#include namespace DB { @@ -24,9 +23,6 @@ public: return count != 0 || !nested_maps.empty(); } }; - - static Result find(const ASTFunction & function); - static Result find(const FunctionNode & function); }; } From 1b9e6c936e7e8123fd1d0c712c3a051a5bbbe89a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 1 Mar 2024 22:29:56 +0000 Subject: [PATCH 089/197] better interfaces for virtual columns --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 6 ++- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../InterpreterShowColumnsQuery.cpp | 2 +- src/Interpreters/JoinedTables.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 3 +- src/Interpreters/getTableExpressions.cpp | 4 +- .../processColumnTransformers.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 4 +- src/Storages/AlterCommands.cpp | 6 +-- src/Storages/HDFS/StorageHDFS.cpp | 6 +-- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/Storages/IStorage.h | 4 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +-- .../MergeTree/MergeTreeSequentialSource.cpp | 3 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/NATS/NATSSource.cpp | 6 +-- .../StorageMaterializedPostgreSQL.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQSource.cpp | 6 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 6 +-- src/Storages/StorageAzureBlob.cpp | 6 +-- src/Storages/StorageAzureBlobCluster.cpp | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageFileCluster.cpp | 2 +- src/Storages/StorageMaterializedMySQL.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageSnapshot.cpp | 12 ++--- src/Storages/StorageURL.cpp | 6 +-- src/Storages/StorageURLCluster.cpp | 2 +- src/Storages/System/IStorageSystemOneBlock.h | 2 +- src/Storages/System/StorageSystemJemalloc.cpp | 2 +- .../System/StorageSystemZooKeeper.cpp | 2 +- src/Storages/VirtualColumnsDescription.cpp | 44 +++++++++---------- src/Storages/VirtualColumnsDescription.h | 14 +++--- src/Storages/prepareReadingFromFormat.cpp | 6 +-- src/Storages/prepareReadingFromFormat.h | 2 +- 41 files changed, 99 insertions(+), 99 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a1b63960d40..3ae7a9d9f11 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -904,7 +904,7 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat void validateVirtualColumns(const IStorage & storage) { - auto virtual_columns = storage.getVirtualsDescription(); + auto virtual_columns = storage.getVirtualsPtr(); for (const auto & storage_column : storage.getInMemoryMetadataPtr()->getColumns()) { if (virtual_columns->tryGet(storage_column.name, VirtualsKind::Persistent)) diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 11542931775..04d44e34fff 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -155,7 +155,8 @@ void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpres auto table = table_function_ptr->execute(table_expression.table_function, getContext(), table_function_ptr->getName()); if (table) { - for (const auto & column : *table->getVirtualsDescription()) + auto virtuals = table->getVirtualsPtr(); + for (const auto & column : *virtuals) { if (!column_descriptions.has(column.name)) virtual_columns.push_back(column); @@ -178,7 +179,8 @@ void InterpreterDescribeQuery::fillColumnsFromTable(const ASTTableExpression & t if (settings.describe_include_virtual_columns) { - for (const auto & column : *table->getVirtualsDescription()) + auto virtuals = table->getVirtualsPtr(); + for (const auto & column : *virtuals) { if (!column_descriptions.has(column.name)) virtual_columns.push_back(column); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 724cfca6a80..87013151a1a 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -133,7 +133,7 @@ Block InterpreterInsertQuery::getSampleBlock( if (auto * window_view = dynamic_cast(table.get())) return window_view->getInputHeader(); else if (no_destination) - return metadata_snapshot->getSampleBlockWithVirtuals(table->getVirtuals()); + return metadata_snapshot->getSampleBlockWithVirtuals(table->getVirtualsList()); else return metadata_snapshot->getSampleBlockNonMaterialized(); } diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 149ba6d7575..f32ebceaa63 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -107,7 +107,7 @@ SELECT '' AS extra )"; // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see - // IStorage::getVirtuals(). We can't easily do that via SQL. + // IStorage::getVirtualsList(). We can't easily do that via SQL. if (query.full) { diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 9be8bf178a1..49693332280 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -272,7 +272,7 @@ void JoinedTables::makeFakeTable(StoragePtr storage, const StorageMetadataPtr & auto & table = tables_with_columns.back(); table.addHiddenColumns(storage_columns.getMaterialized()); table.addHiddenColumns(storage_columns.getAliases()); - table.addHiddenColumns(storage->getVirtuals()); + table.addHiddenColumns(storage->getVirtualsList()); } else tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, source_header.getNamesAndTypesList()); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 7a0d68f7c87..6641c6b740c 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -349,7 +349,8 @@ bool MutationsInterpreter::Source::isCompactPart() const static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) { auto all_columns = metadata_snapshot->getColumns().getNamesOfPhysical(); - for (const auto & column : storage.getVirtuals()) + auto virtuals = storage.getVirtualsPtr(); + for (const auto & column : *virtuals) all_columns.push_back(column.name); return all_columns; } diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 70e38526648..2853be4c05e 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -99,7 +99,7 @@ static NamesAndTypesList getColumnsFromTableExpression( names_and_type_list = columns.getOrdinary(); materialized = columns.getMaterialized(); aliases = columns.getAliases(); - virtuals = function_storage->getVirtuals(); + virtuals = function_storage->getVirtualsList(); } else if (table_expression.database_and_table_name) { @@ -110,7 +110,7 @@ static NamesAndTypesList getColumnsFromTableExpression( names_and_type_list = columns.getOrdinary(); materialized = columns.getMaterialized(); aliases = columns.getAliases(); - virtuals = table->getVirtuals(); + virtuals = table->getVirtualsList(); } return names_and_type_list; diff --git a/src/Interpreters/processColumnTransformers.cpp b/src/Interpreters/processColumnTransformers.cpp index 2a704d4a937..5ef331eb119 100644 --- a/src/Interpreters/processColumnTransformers.cpp +++ b/src/Interpreters/processColumnTransformers.cpp @@ -32,7 +32,7 @@ ASTPtr processColumnTransformers( tables_with_columns[0].addHiddenColumns(columns.getMaterialized()); tables_with_columns[0].addHiddenColumns(columns.getAliases()); - tables_with_columns[0].addHiddenColumns(table->getVirtuals()); + tables_with_columns[0].addHiddenColumns(table->getVirtualsList()); NameSet source_columns_set; for (const auto & identifier : query_columns->children) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index c4b707a0ce0..f051029c3dc 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -214,7 +214,7 @@ Chain buildPushingToViewsChain( /// If we don't write directly to the destination /// then expect that we're inserting with precalculated virtual columns - auto storage_header = no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()) + auto storage_header = no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtualsList()) : metadata_snapshot->getSampleBlock(); /** TODO This is a very important line. At any insertion into the table one of chains should own lock. @@ -574,7 +574,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat views_data.source_storage_id, views_data.source_metadata_snapshot->getColumns(), std::move(block), - *views_data.source_storage->getVirtualsDescription())); + *views_data.source_storage->getVirtualsPtr())); QueryPipelineBuilder pipeline; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index e1c80112c72..086e7152fda 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1244,7 +1244,7 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata) void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { const auto & metadata = table->getInMemoryMetadata(); - const auto & virtuals = *table->getVirtualsDescription(); + auto virtuals = table->getVirtualsPtr(); auto all_columns = metadata.columns; /// Default expression for all added/modified columns @@ -1281,7 +1281,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const if (command.data_type->hasDynamicSubcolumns()) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Adding a new column of a type which has dynamic subcolumns to an existing table is not allowed. It has known bugs"); - if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) + if (virtuals->tryGet(column_name, VirtualsKind::Persistent)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: this column name is reserved for persistent virtual column", backQuote(column_name)); @@ -1495,7 +1495,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Cannot rename to {}: column with this name already exists", backQuote(command.rename_to)); - if (virtuals.tryGet(command.rename_to, VirtualsKind::Persistent)) + if (virtuals->tryGet(command.rename_to, VirtualsKind::Persistent)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: this column name is reserved for persistent virtual column", backQuote(command.rename_to)); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 19b2817510d..c574f57fc6a 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -974,7 +974,7 @@ void StorageHDFS::read( size_t max_block_size, size_t num_streams) { - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_)); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && context_->getSettingsRef().optimize_count_from_files; @@ -1010,7 +1010,7 @@ void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) else if (storage->is_path_with_globs) { /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(storage->uris[0], predicate, storage->getVirtuals(), context); + auto glob_iterator = std::make_shared(storage->uris[0], predicate, storage->getVirtualsList(), context); iterator_wrapper = std::make_shared([glob_iterator]() { return glob_iterator->next(); @@ -1018,7 +1018,7 @@ void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) } else { - auto uris_iterator = std::make_shared(storage->uris, predicate, storage->getVirtuals(), context); + auto uris_iterator = std::make_shared(storage->uris, predicate, storage->getVirtualsList(), context); iterator_wrapper = std::make_shared([uris_iterator]() { return uris_iterator->next(); diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 8c6d7ce5670..bde8b84e349 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -88,7 +88,7 @@ void StorageHDFSCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB: RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { - auto iterator = std::make_shared(uri, predicate, getVirtuals(), context); + auto iterator = std::make_shared(uri, predicate, getVirtualsList(), context); auto callback = std::make_shared>([iter = std::move(iterator)]() mutable -> String { return iter->next().path; }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e7ebb45cb46..f8d73038e09 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -229,8 +229,8 @@ public: /// virtual column will be overridden and inaccessible. /// /// By default return empty list of columns. - VirtualsDescriptionPtr getVirtualsDescription() const { return virtuals.get(); } - NamesAndTypesList getVirtuals() const { return virtuals.get()->getNamesAndTypesList(); } + VirtualsDescriptionPtr getVirtualsPtr() const { return virtuals.get(); } + NamesAndTypesList getVirtualsList() const { return virtuals.get()->getNamesAndTypesList(); } Block getVirtualsHeader() const { return virtuals.get()->getSampleBlock(); } Names getAllRegisteredNames() const override; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 4621314cb98..aa38198334e 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1074,7 +1074,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() if (global_ctx->deduplicate) { - const auto & virtuals = *global_ctx->data->getVirtualsDescription(); + const auto & virtuals = *global_ctx->data->getVirtualsPtr(); /// We don't want to deduplicate by virtual persistent column. /// If deduplicate_by_columns is empty, add all columns except virtuals. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0b7ac39aa1b..7b1d08642e2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1031,7 +1031,7 @@ const Names MergeTreeData::virtuals_useful_for_filter = {"_part", "_partition_id Block MergeTreeData::getHeaderWithVirtualsForFilter() const { Block header; - auto virtuals_desc = getVirtualsDescription(); + auto virtuals_desc = getVirtualsPtr(); for (const auto & name : virtuals_useful_for_filter) if (auto column = virtuals_desc->tryGet(name)) header.insert({column->type->createColumn(), column->type, name}); @@ -3648,7 +3648,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts { auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & columns = metadata_snapshot->getColumns(); - const auto & virtuals = *getVirtualsDescription(); + auto virtuals = getVirtualsPtr(); if (!hasDynamicSubcolumns(columns)) return; @@ -3656,7 +3656,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts const auto & part_columns = part->getColumns(); for (const auto & part_column : part_columns) { - if (virtuals.has(part_column.name)) + if (virtuals->has(part_column.name)) continue; auto storage_column = columns.getPhysical(part_column.name); diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 2eb010c54ec..0d6b1d88075 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -195,7 +195,6 @@ static void fillBlockNumberColumns( Columns & res_columns, const NamesAndTypesList & columns_list, UInt64 block_number, - UInt64 block_offset, UInt64 num_rows) { chassert(res_columns.size() == columns_list.size()); @@ -231,7 +230,7 @@ try if (rows_read) { - fillBlockNumberColumns(columns, sample, data_part->info.min_block, current_row, rows_read); + fillBlockNumberColumns(columns, sample, data_part->info.min_block, rows_read); reader->fillVirtualColumns(columns, rows_read); current_row += rows_read; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b3c36f7180b..56f832630b7 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -168,7 +168,7 @@ static void splitAndModifyMutationCommands( { if (!mutated_columns.contains(column.name)) { - if (!metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtuals().contains(column.name)) + if (!metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtualsPtr()->has(column.name)) { /// We cannot add the column because there's no such column in table. /// It's okay if the column was dropped. It may also absent in dropped_columns diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index da4405dfd40..ca8ed9abdb5 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -34,7 +34,7 @@ public: , partition_id(part_->info.partition_id) { setInMemoryMetadata(storage.getInMemoryMetadata()); - setVirtuals(*storage.getVirtualsDescription()); + setVirtuals(*storage.getVirtualsPtr()); } /// Used in queries with projection. diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 3fc01eacb22..54f479faacc 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -9,10 +9,10 @@ namespace DB { -static std::pair getHeaders(StorageNATS & storage, const StorageSnapshotPtr & storage_snapshot) +static std::pair getHeaders(const StorageSnapshotPtr & storage_snapshot) { auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized(); - auto virtual_header = storage_snapshot->getSampleBlockForColumns(storage.getVirtuals().getNames()); + auto virtual_header = storage_snapshot->virtual_columns->getSampleBlock(); return {non_virtual_header, virtual_header}; } @@ -33,7 +33,7 @@ NATSSource::NATSSource( const Names & columns, size_t max_block_size_, StreamingHandleErrorMode handle_error_mode_) - : NATSSource(storage_, storage_snapshot_, getHeaders(storage_, storage_snapshot_), context_, columns, max_block_size_, handle_error_mode_) + : NATSSource(storage_, storage_snapshot_, getHeaders(storage_snapshot_), context_, columns, max_block_size_, handle_error_mode_) { } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index c9866146a8b..64d329f74b2 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -128,7 +128,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( , nested_table_id(nested_storage_->getStorageID()) { setInMemoryMetadata(nested_storage_->getInMemoryMetadata()); - setVirtuals(*nested_storage_->getVirtualsDescription()); + setVirtuals(*nested_storage_->getVirtualsPtr()); } VirtualColumnsDescription StorageMaterializedPostgreSQL::createVirtuals() diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 72196e7dd3c..4dc257074f3 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -11,10 +11,10 @@ namespace DB { -static std::pair getHeaders(StorageRabbitMQ & storage_, const StorageSnapshotPtr & storage_snapshot) +static std::pair getHeaders(const StorageSnapshotPtr & storage_snapshot) { auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized(); - auto virtual_header = storage_snapshot->getSampleBlockForColumns(storage_.getVirtuals().getNames()); + auto virtual_header = storage_snapshot->virtual_columns->getSampleBlock(); return {non_virtual_header, virtual_header}; } @@ -40,7 +40,7 @@ RabbitMQSource::RabbitMQSource( : RabbitMQSource( storage_, storage_snapshot_, - getHeaders(storage_, storage_snapshot_), + getHeaders(storage_snapshot_), context_, columns, max_block_size_, diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 765fcbd9684..6e7ac2b47b8 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -314,7 +314,7 @@ void StorageS3Queue::read( } auto this_ptr = std::static_pointer_cast(shared_from_this()); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); auto reading = std::make_unique( column_names, @@ -492,7 +492,7 @@ bool StorageS3Queue::streamToViews() auto block_io = interpreter.execute(); auto file_iterator = createFileIterator(s3queue_context, nullptr); - auto read_from_format_info = prepareReadingFromFormat(block_io.pipeline.getHeader().getNames(), storage_snapshot, supportsSubsetOfColumns(s3queue_context), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(block_io.pipeline.getHeader().getNames(), storage_snapshot, supportsSubsetOfColumns(s3queue_context)); Pipes pipes; pipes.reserve(s3queue_settings->s3queue_processing_threads_num); @@ -601,7 +601,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate) { auto glob_iterator = std::make_unique( - *configuration.client, configuration.url, predicate, getVirtuals(), local_context, + *configuration.client, configuration.url, predicate, getVirtualsList(), local_context, /* read_keys */nullptr, configuration.request_settings); return std::make_shared(files_metadata, std::move(glob_iterator), s3queue_settings->s3queue_current_shard_num, shutdown_called); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 92dbc1025db..bd88620c55e 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -736,7 +736,7 @@ void StorageAzureBlob::read( auto this_ptr = std::static_pointer_cast(shared_from_this()); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; @@ -772,13 +772,13 @@ void ReadFromAzureBlob::createIterator(const ActionsDAG::Node * predicate) /// Iterate through disclosed globs and make a source for each file iterator_wrapper = std::make_shared( storage->object_storage.get(), configuration.container, configuration.blob_path, - predicate, storage->getVirtuals(), context, nullptr, context->getFileProgressCallback()); + predicate, storage->getVirtualsList(), context, nullptr, context->getFileProgressCallback()); } else { iterator_wrapper = std::make_shared( storage->object_storage.get(), configuration.container, configuration.blobs_paths, - predicate, storage->getVirtuals(), context, nullptr, context->getFileProgressCallback()); + predicate, storage->getVirtualsList(), context, nullptr, context->getFileProgressCallback()); } } diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index 155a7220885..a80d121567a 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -80,7 +80,7 @@ RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension { auto iterator = std::make_shared( object_storage.get(), configuration.container, configuration.blob_path, - predicate, getVirtuals(), context, nullptr); + predicate, getVirtualsList(), context, nullptr); auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next().relative_path; }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index a734d3967eb..0d220f2fd5d 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1580,7 +1580,7 @@ void StorageFile::read( auto this_ptr = std::static_pointer_cast(shared_from_this()); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context)); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && context->getSettingsRef().optimize_count_from_files; @@ -1608,7 +1608,7 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) storage->paths, storage->archive_info, predicate, - storage->getVirtuals(), + storage->getVirtualsList(), context, storage->distributed_processing); } diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index 84691472809..d43e242f70c 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -76,7 +76,7 @@ void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const Sto RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { - auto iterator = std::make_shared(paths, std::nullopt, predicate, getVirtuals(), context); + auto iterator = std::make_shared(paths, std::nullopt, predicate, getVirtualsList(), context); auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index 1651b0499ad..887c58ff816 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -23,7 +23,7 @@ StorageMaterializedMySQL::StorageMaterializedMySQL(const StoragePtr & nested_sto : StorageProxy(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) { setInMemoryMetadata(nested_storage->getInMemoryMetadata()); - setVirtuals(*nested_storage->getVirtualsDescription()); + setVirtuals(*nested_storage->getVirtualsPtr()); } bool StorageMaterializedMySQL::needRewriteQueryWithFinal(const Names & column_names) const diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index aeba01fdf8a..668dc84481d 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -168,7 +168,7 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( StorageSnapshotPtr StorageMaterializedView::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr) const { /// We cannot set virtuals at table creation because target table may not exist at that time. - return std::make_shared(*this, metadata_snapshot, getTargetTable()->getVirtualsDescription()); + return std::make_shared(*this, metadata_snapshot, getTargetTable()->getVirtualsPtr()); } void StorageMaterializedView::read( diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 00513d1ec46..3947ae6c1f3 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -325,7 +325,7 @@ StorageSnapshotPtr StorageMerge::getStorageSnapshot(const StorageMetadataPtr & m auto virtuals = common_virtuals; if (auto first_table = getFirstTable([](auto && table) { return table; })) { - auto table_virtuals = first_table->getVirtualsDescription(); + auto table_virtuals = first_table->getVirtualsPtr(); for (const auto & column : *table_virtuals) { if (virtuals.has(column.name)) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index afd34792335..c7cb0163a0a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -165,7 +165,7 @@ public: , num_streams(num_streams_) { query_configuration = storage.updateConfigurationAndGetCopy(context); - virtual_columns = storage.getVirtuals(); + virtual_columns = storage.getVirtualsList(); } private: @@ -1142,7 +1142,7 @@ void StorageS3::read( size_t max_block_size, size_t num_streams) { - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 7641c66eefd..8a13c2c6ca5 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -93,7 +93,7 @@ void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { auto iterator = std::make_shared( - *s3_configuration.client, s3_configuration.url, predicate, getVirtuals(), context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); + *s3_configuration.client, s3_configuration.url, predicate, getVirtualsList(), context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 1893d65a64c..e5c1d3d1dea 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -22,7 +22,7 @@ StorageSnapshot::StorageSnapshot( StorageMetadataPtr metadata_) : storage(storage_) , metadata(std::move(metadata_)) - , virtual_columns(storage_.getVirtualsDescription()) + , virtual_columns(storage_.getVirtualsPtr()) { } @@ -42,7 +42,7 @@ StorageSnapshot::StorageSnapshot( ColumnsDescription object_columns_) : storage(storage_) , metadata(std::move(metadata_)) - , virtual_columns(storage_.getVirtualsDescription()) + , virtual_columns(storage_.getVirtualsPtr()) , object_columns(std::move(object_columns_)) { } @@ -54,7 +54,7 @@ StorageSnapshot::StorageSnapshot( DataPtr data_) : storage(storage_) , metadata(std::move(metadata_)) - , virtual_columns(storage_.getVirtualsDescription()) + , virtual_columns(storage_.getVirtualsPtr()) , object_columns(std::move(object_columns_)) , data(std::move(data_)) { @@ -83,7 +83,7 @@ NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) for (const auto & column : all_columns) column_names.insert(column.name); - auto virtuals_list = virtual_columns->get(options.virtuals_kind); + auto virtuals_list = virtual_columns->getNamesAndTypesList(options.virtuals_kind); for (const auto & column : virtuals_list) { if (column_names.contains(column.name)) @@ -150,7 +150,7 @@ CompressionCodecPtr StorageSnapshot::getCodecOrDefault(const String & column_nam if (const auto * column_desc = columns.tryGet(column_name)) return get_codec_or_default(*column_desc); - if (const auto virtual_desc = virtual_columns->tryGetDescription(column_name)) + if (const auto * virtual_desc = virtual_columns->tryGetDescription(column_name)) return get_codec_or_default(*virtual_desc); return default_codec; @@ -172,7 +172,7 @@ ASTPtr StorageSnapshot::getCodecDescOrDefault(const String & column_name, Compre if (const auto * column_desc = columns.tryGet(column_name)) return get_codec_or_default(*column_desc); - if (const auto virtual_desc = virtual_columns->tryGetDescription(column_name)) + if (const auto * virtual_desc = virtual_columns->tryGetDescription(column_name)) return get_codec_or_default(*virtual_desc); return default_codec->getFullCodecDesc(); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index ec5b6008c17..039be222e7e 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1058,7 +1058,7 @@ void IStorageURLBase::read( size_t num_streams) { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; @@ -1126,7 +1126,7 @@ void ReadFromURL::createIterator(const ActionsDAG::Node * predicate) else if (is_url_with_globs) { /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(storage->uri, max_addresses, predicate, storage->getVirtuals(), context); + auto glob_iterator = std::make_shared(storage->uri, max_addresses, predicate, storage->getVirtualsList(), context); /// check if we filtered out all the paths if (glob_iterator->size() == 0) @@ -1229,7 +1229,7 @@ void StorageURLWithFailover::read( size_t num_streams) { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 334c2ca249b..2e7c63d0097 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -90,7 +90,7 @@ void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageS RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { - auto iterator = std::make_shared(uri, context->getSettingsRef().glob_expansion_max_elements, predicate, getVirtuals(), context); + auto iterator = std::make_shared(uri, context->getSettingsRef().glob_expansion_max_elements, predicate, getVirtualsList(), context); auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 3b2807965a4..910bbaa13c2 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -52,7 +52,7 @@ public: size_t /*num_streams*/) override { storage_snapshot->check(column_names); - Block sample_block = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + Block sample_block = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtualsList()); if (supportsColumnsMask()) { diff --git a/src/Storages/System/StorageSystemJemalloc.cpp b/src/Storages/System/StorageSystemJemalloc.cpp index 15543208dd9..b55e32c479c 100644 --- a/src/Storages/System/StorageSystemJemalloc.cpp +++ b/src/Storages/System/StorageSystemJemalloc.cpp @@ -115,7 +115,7 @@ Pipe StorageSystemJemallocBins::read( { storage_snapshot->check(column_names); - auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtualsList()); MutableColumns res_columns = header.cloneEmptyColumns(); fillJemallocBins(res_columns); diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index f2a4de87efd..d1bf86ba8ef 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -248,7 +248,7 @@ void StorageSystemZooKeeper::read( size_t max_block_size, size_t /*num_streams*/) { - auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtualsList()); auto read_step = std::make_unique( column_names, query_info, diff --git a/src/Storages/VirtualColumnsDescription.cpp b/src/Storages/VirtualColumnsDescription.cpp index 5dd249c5190..64097224ed9 100644 --- a/src/Storages/VirtualColumnsDescription.cpp +++ b/src/Storages/VirtualColumnsDescription.cpp @@ -34,15 +34,6 @@ void VirtualColumnsDescription::addPersistent(String name, DataTypePtr type, AST add({std::move(name), std::move(type), std::move(codec), std::move(comment), VirtualsKind::Persistent}); } -NamesAndTypesList VirtualColumnsDescription::get(VirtualsKind kind) const -{ - NamesAndTypesList result; - for (const auto & column : container) - if (static_cast(column.kind) & static_cast(kind)) - result.emplace_back(column.name, column.type); - return result; -} - std::optional VirtualColumnsDescription::tryGet(const String & name, VirtualsKind kind) const { auto it = container.get<1>().find(name); @@ -59,30 +50,22 @@ NameAndTypePair VirtualColumnsDescription::get(const String & name, VirtualsKind return *column; } -std::optional VirtualColumnsDescription::tryGetDescription(const String & name, VirtualsKind kind) const +const VirtualColumnDescription * VirtualColumnsDescription::tryGetDescription(const String & name, VirtualsKind kind) const { auto it = container.get<1>().find(name); if (it != container.get<1>().end() && (static_cast(it->kind) & static_cast(kind))) - return *it; - return {}; + return &(*it); + return nullptr; } -VirtualColumnDescription VirtualColumnsDescription::getDescription(const String & name, VirtualsKind kind) const +const VirtualColumnDescription & VirtualColumnsDescription::getDescription(const String & name, VirtualsKind kind) const { - auto column = tryGetDescription(name, kind); + const auto * column = tryGetDescription(name, kind); if (!column) throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no virtual column {}", name); return *column; } -NamesAndTypesList VirtualColumnsDescription::getNamesAndTypesList() const -{ - NamesAndTypesList result; - for (const auto & desc : container) - result.emplace_back(desc.name, desc.type); - return result; -} - Block VirtualColumnsDescription::getSampleBlock() const { Block result; @@ -91,4 +74,21 @@ Block VirtualColumnsDescription::getSampleBlock() const return result; } +NamesAndTypesList VirtualColumnsDescription::getNamesAndTypesList() const +{ + NamesAndTypesList result; + for (const auto & desc : container) + result.emplace_back(desc.name, desc.type); + return result; +} + +NamesAndTypesList VirtualColumnsDescription::getNamesAndTypesList(VirtualsKind kind) const +{ + NamesAndTypesList result; + for (const auto & column : container) + if (static_cast(column.kind) & static_cast(kind)) + result.emplace_back(column.name, column.type); + return result; +} + } diff --git a/src/Storages/VirtualColumnsDescription.h b/src/Storages/VirtualColumnsDescription.h index 3d79167c623..2f46bbcab82 100644 --- a/src/Storages/VirtualColumnsDescription.h +++ b/src/Storages/VirtualColumnsDescription.h @@ -49,17 +49,15 @@ public: NameAndTypePair get(const String & name) const { return get(name, VirtualsKind::All); } std::optional tryGet(const String & name) const { return tryGet(name, VirtualsKind::All); } - std::optional tryGetDescription(const String & name, VirtualsKind kind) const; - VirtualColumnDescription getDescription(const String & name, VirtualsKind kind) const; + const VirtualColumnDescription * tryGetDescription(const String & name, VirtualsKind kind) const; + const VirtualColumnDescription & getDescription(const String & name, VirtualsKind kind) const; - std::optional tryGetDescription(const String & name) const { return tryGetDescription(name, VirtualsKind::All); } - VirtualColumnDescription getDescription(const String & name) const { return getDescription(name, VirtualsKind::All); } - - NamesAndTypesList get(VirtualsKind kind) const; - NamesAndTypesList getNamesAndTypesList() const; + const VirtualColumnDescription * tryGetDescription(const String & name) const { return tryGetDescription(name, VirtualsKind::All); } + const VirtualColumnDescription & getDescription(const String & name) const { return getDescription(name, VirtualsKind::All); } Block getSampleBlock() const; - Block getSampleBlock(const Names & names) const; + NamesAndTypesList getNamesAndTypesList() const; + NamesAndTypesList getNamesAndTypesList(VirtualsKind kind) const; private: Container container; diff --git a/src/Storages/prepareReadingFromFormat.cpp b/src/Storages/prepareReadingFromFormat.cpp index 6be4213ec6b..406b7f379f9 100644 --- a/src/Storages/prepareReadingFromFormat.cpp +++ b/src/Storages/prepareReadingFromFormat.cpp @@ -4,7 +4,7 @@ namespace DB { -ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, bool supports_subset_of_columns, const NamesAndTypesList & virtuals) +ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, bool supports_subset_of_columns) { ReadFromFormatInfo info; /// Collect requested virtual columns and remove them from requested columns. @@ -12,11 +12,11 @@ ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, c for (const auto & column_name : requested_columns) { bool is_virtual = false; - for (const auto & virtual_column : virtuals) + for (const auto & virtual_column : *storage_snapshot->virtual_columns) { if (column_name == virtual_column.name) { - info.requested_virtual_columns.push_back(virtual_column); + info.requested_virtual_columns.emplace_back(virtual_column.name, virtual_column.type); is_virtual = true; break; } diff --git a/src/Storages/prepareReadingFromFormat.h b/src/Storages/prepareReadingFromFormat.h index c5f3959a550..e4d62c29ec6 100644 --- a/src/Storages/prepareReadingFromFormat.h +++ b/src/Storages/prepareReadingFromFormat.h @@ -22,5 +22,5 @@ namespace DB }; /// Get all needed information for reading from data in some input format. - ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, bool supports_subset_of_columns, const NamesAndTypesList & virtuals); + ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, bool supports_subset_of_columns); } From 4735eb3ab419e9a9c21bdc58ef2bcc4ee2823353 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 1 Mar 2024 23:29:17 +0100 Subject: [PATCH 090/197] wait for pool jobs in case of exception on scheduling --- src/Interpreters/AsynchronousInsertQueue.cpp | 51 +++++++++++--------- 1 file changed, 29 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index c7a39ad610b..e25cedb916c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -214,34 +214,41 @@ AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t poo AsynchronousInsertQueue::~AsynchronousInsertQueue() { - LOG_TRACE(log, "Shutting down the asynchronous insertion queue"); - shutdown = true; - - for (size_t i = 0; i < pool_size; ++i) + try { - auto & shard = queue_shards[i]; + LOG_TRACE(log, "Shutting down the asynchronous insertion queue"); + shutdown = true; - shard.are_tasks_available.notify_one(); - assert(dump_by_first_update_threads[i].joinable()); - dump_by_first_update_threads[i].join(); - - if (flush_on_shutdown) - { - for (auto & [_, elem] : shard.queue) - scheduleDataProcessingJob(elem.key, std::move(elem.data), getContext(), i); - } - else + for (size_t i = 0; i < pool_size; ++i) { + auto & shard = queue_shards[i]; - for (auto & [_, elem] : shard.queue) - for (const auto & entry : elem.data->entries) - entry->finish(std::make_exception_ptr(Exception( - ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout exceeded)"))); + shard.are_tasks_available.notify_one(); + assert(dump_by_first_update_threads[i].joinable()); + dump_by_first_update_threads[i].join(); + + if (flush_on_shutdown) + { + for (auto & [_, elem] : shard.queue) + scheduleDataProcessingJob(elem.key, std::move(elem.data), getContext(), i); + } + else + { + for (auto & [_, elem] : shard.queue) + for (const auto & entry : elem.data->entries) + entry->finish( + std::make_exception_ptr(Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout exceeded)"))); + } } + + pool.wait(); + LOG_TRACE(log, "Asynchronous insertion queue finished"); + } + catch (...) + { + tryLogCurrentException(log); + pool.wait(); } - - pool.wait(); - LOG_TRACE(log, "Asynchronous insertion queue finished"); } void AsynchronousInsertQueue::scheduleDataProcessingJob( From 420d98295daa84ef86f5a63b73a3c6dc9ff7ed7f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 1 Mar 2024 23:24:35 +0000 Subject: [PATCH 091/197] Fix: reorder indices only if necessary --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8f0db044d8f..d3b1a324de9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1423,15 +1423,17 @@ static void buildIndexes( } // move minmax indices to first positions, so they will be applied first as cheapest ones - std::sort(begin(skip_indexes.useful_indices), end(skip_indexes.useful_indices), [](const auto & l, const auto & r) + std::stable_sort(begin(skip_indexes.useful_indices), end(skip_indexes.useful_indices), [](const auto & l, const auto & r) { - if (typeid_cast(l.index.get())) - return true; // left is min max + const bool l_min_max = (typeid_cast(l.index.get())); + const bool r_min_max = (typeid_cast(r.index.get())); + if (l_min_max == r_min_max) + return false; - if (typeid_cast(r.index.get())) - return false; // right is min max but left is not + if (l_min_max) + return true; // left is min max but right is not - return true; + return false; // right is min max but left is not }); indexes->skip_indexes = std::move(skip_indexes); From 57482de0f0858f5a03e60a7310227106487ce438 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sat, 2 Mar 2024 01:16:47 +0100 Subject: [PATCH 092/197] Update FunctionBinaryArithmetic.h --- src/Functions/FunctionBinaryArithmetic.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 9ad74f6332f..79e5ee442c2 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -171,7 +171,7 @@ public: using ResultDataType = Switch< /// Result must be Integer Case::int_div || IsOperation::int_div_or_zero, - std::conditional_t && IsDataTypeNumber, DataTypeFromFieldType, InvalidType>>, + std::conditional_t && IsDataTypeDecimalOrNumber, DataTypeFromFieldType, InvalidType>>, /// Decimal cases Case || IsDataTypeDecimal, DecimalResultDataType>, Case< From cbad19956cd2a078bb9304f90e0675c9c081403e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 4 Mar 2024 10:09:19 +0000 Subject: [PATCH 093/197] Fix test with analyzer --- tests/queries/0_stateless/03000_minmax_index_first.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03000_minmax_index_first.sql b/tests/queries/0_stateless/03000_minmax_index_first.sql index 4db232880de..5dae245a0a0 100644 --- a/tests/queries/0_stateless/03000_minmax_index_first.sql +++ b/tests/queries/0_stateless/03000_minmax_index_first.sql @@ -12,4 +12,4 @@ PRIMARY KEY k; INSERT INTO skip_table SELECT number, intDiv(number, 4096) FROM numbers(1000000); -SELECT trim(explain) FROM ( EXPLAIN indexes = 1 SELECT * FROM skip_table WHERE v = 125) WHERE explain ilike '%Name%'; +SELECT trim(explain) FROM ( EXPLAIN indexes = 1 SELECT * FROM skip_table WHERE v = 125) WHERE explain like '%Name%'; From 41ff818a20c8b196f08155445cc29504346d998b Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 4 Mar 2024 12:53:41 +0100 Subject: [PATCH 094/197] fix toStartOfInterval --- src/Functions/DateTimeTransforms.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 5467cf77085..2c5d8add0db 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -538,7 +538,7 @@ struct ToStartOfInterval { Int64 scale_diff = scale_multiplier / static_cast(1000000); if (t >= 0) [[likely]] /// When we divide the `t` value we should round the result - return (t / microseconds + scale_diff / 2) / scale_diff * microseconds; + return (t + scale_diff / 2) / (microseconds * scale_diff) * microseconds; else return ((t + 1) / microseconds / scale_diff - 1) * microseconds; } @@ -581,7 +581,7 @@ struct ToStartOfInterval { Int64 scale_diff = scale_multiplier / static_cast(1000); if (t >= 0) [[likely]] /// When we divide the `t` value we should round the result - return (t / milliseconds + scale_diff / 2) / scale_diff * milliseconds; + return (t + scale_diff / 2) / (milliseconds * scale_diff) * milliseconds; else return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds; } From 4e55f2c90dae6b3c52e6af0107993613cd8b563a Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 4 Mar 2024 13:00:10 +0100 Subject: [PATCH 095/197] update test for toStartof... --- .../02956_fix_to_start_of_milli_microsecond.reference | 3 +++ .../0_stateless/02956_fix_to_start_of_milli_microsecond.sql | 5 ++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference index d3a002c4fd4..413c79828c7 100644 --- a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference +++ b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference @@ -2,3 +2,6 @@ 2023-10-09 10:11:12.001 2023-10-09 10:11:12.000 2023-10-09 10:11:12.000 +2023-10-09 00:00:00.000000 +2023-10-09 00:00:00.000 +2023-10-09 00:00:00 diff --git a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.sql b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.sql index 178f21a9e63..15753d4532c 100644 --- a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.sql +++ b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.sql @@ -1,4 +1,7 @@ SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.000999', 6), toIntervalMillisecond(1)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.000500', 6), toIntervalMillisecond(1)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.000499', 6), toIntervalMillisecond(1)); -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.000999', 6), toIntervalMillisecond(10)); \ No newline at end of file +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.000999', 6), toIntervalMillisecond(10)); +select toStartOfInterval(toDateTime64('2023-10-09 00:01:34', 9), toIntervalMicrosecond(100000000)); +select toStartOfInterval(toDateTime64('2023-10-09 00:01:34', 9), toIntervalMillisecond(100000)); +select toStartOfInterval(toDateTime64('2023-10-09 00:01:34', 9), toIntervalSecond(100)); \ No newline at end of file From c634012dbf8fa4570f9f60ff3e17b14d457f3c3d Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 4 Mar 2024 09:26:25 +0100 Subject: [PATCH 096/197] Reproducer for insert-select + insert_deduplication_token bug --- ...001_insert_threads_deduplication.reference | 10 +++ .../03001_insert_threads_deduplication.sh | 82 +++++++++++++++++++ 2 files changed, 92 insertions(+) create mode 100644 tests/queries/0_stateless/03001_insert_threads_deduplication.reference create mode 100755 tests/queries/0_stateless/03001_insert_threads_deduplication.sh diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.reference b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference new file mode 100644 index 00000000000..0c6a5a55576 --- /dev/null +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference @@ -0,0 +1,10 @@ +This bug has been there forever. Present in 22.2 +- When using multiple threads the insert produces 3 parts causing undesired deduplication. +- When using a single thread the insert produces 1 part without deduplication. +1 +4 +This bug has been introduced in CH 24.2+. See https://github.com/ClickHouse/ClickHouse/pull/59448 +- When using remote function and multiple threads the insert produces 3 parts causing undesired deduplication. +- When using remote function and a single thread the insert produces 1 part without deduplication. +1 +4 diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.sh b/tests/queries/0_stateless/03001_insert_threads_deduplication.sh new file mode 100755 index 00000000000..cf87f7c2c67 --- /dev/null +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.sh @@ -0,0 +1,82 @@ +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo " +DROP TABLE IF EXISTS landing SYNC; +CREATE TABLE landing +( + timestamp DateTime64(3), + status String, + id String +) +ENGINE = MergeTree() +ORDER BY timestamp; + +SYSTEM STOP MERGES landing; -- Stopping merges to force 3 parts + +INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; +INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; +INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; + +DROP TABLE IF EXISTS ds SYNC; + +CREATE TABLE ds +( + timestamp DateTime64(3), + status String, + id String +) +ENGINE = MergeTree() +ORDER BY timestamp +SETTINGS non_replicated_deduplication_window=1000; + +SELECT 'This bug has been there forever. Present in 22.2'; +SELECT '- When using multiple threads the insert produces 3 parts causing undesired deduplication.'; +SELECT '- When using a single thread the insert produces 1 part without deduplication.'; + +INSERT INTO ds SELECT * FROM landing +SETTINGS insert_deduplicate=1, insert_deduplication_token='token1', + max_insert_threads=5; + +SELECT count() FROM ds; + +INSERT INTO ds SELECT * FROM landing +SETTINGS insert_deduplicate=1, insert_deduplication_token='token2', + max_insert_threads=1; + +SELECT count() FROM ds; +" | $CLICKHOUSE_CLIENT -n + +echo " +CREATE TABLE ds_remote +( + timestamp DateTime64(3), + status String, + id String +) +ENGINE = MergeTree() +ORDER BY timestamp +SETTINGS non_replicated_deduplication_window=1000; + +SELECT 'This bug has been introduced in CH 24.2+. See https://github.com/ClickHouse/ClickHouse/pull/59448'; +SELECT '- When using remote function and multiple threads the insert produces 3 parts causing undesired deduplication.'; +SELECT '- When using remote function and a single thread the insert produces 1 part without deduplication.'; + +INSERT INTO ds_remote SELECT * FROM remote('localhost:$CLICKHOUSE_PORT_TCP', $CLICKHOUSE_DATABASE, landing) +SETTINGS insert_deduplicate=1, insert_deduplication_token='token1', + max_insert_threads=5; + +SELECT count() FROM ds_remote; + +INSERT INTO ds_remote SELECT * FROM remote('localhost:$CLICKHOUSE_PORT_TCP', $CLICKHOUSE_DATABASE, landing) +SETTINGS insert_deduplicate=1, insert_deduplication_token='token2', + max_insert_threads=1; + +SELECT count() FROM ds_remote; +" | $CLICKHOUSE_LOCAL -n + +echo " +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS ds SYNC; +" | $CLICKHOUSE_CLIENT -n From 63747271e8b57076467811f5a537d63c60e71cb3 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 4 Mar 2024 09:53:09 +0100 Subject: [PATCH 097/197] Fix test --- tests/queries/0_stateless/03001_insert_threads_deduplication.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.sh b/tests/queries/0_stateless/03001_insert_threads_deduplication.sh index cf87f7c2c67..154e578a7a8 100755 --- a/tests/queries/0_stateless/03001_insert_threads_deduplication.sh +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.sh @@ -1,3 +1,5 @@ +#!/bin/bash + CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh From 1da1bbeae2aee280fca052acd2b19672188a1ce1 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 4 Mar 2024 14:35:57 +0100 Subject: [PATCH 098/197] Set streams to 1 when using insert_deduplication_token --- src/Interpreters/InterpreterInsertQuery.cpp | 6 ++++++ .../03001_insert_threads_deduplication.reference | 8 ++++---- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e27a8bd414b..df833803970 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -513,6 +513,12 @@ BlockIO InterpreterInsertQuery::execute() const bool resize_to_max_insert_threads = !table->isView() && views.empty(); pre_streams_size = resize_to_max_insert_threads ? settings.max_insert_threads : std::min(settings.max_insert_threads, pipeline.getNumStreams()); + + /// Deduplication when passing insert_deduplication_token breaks if using more than one thread + const String & deduplication_token = settings.insert_deduplication_token; + if (!deduplication_token.empty()) + pre_streams_size = 1; + if (table->supportsParallelInsert()) sink_streams_size = pre_streams_size; } diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.reference b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference index 0c6a5a55576..0791b98cc09 100644 --- a/tests/queries/0_stateless/03001_insert_threads_deduplication.reference +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference @@ -1,10 +1,10 @@ This bug has been there forever. Present in 22.2 - When using multiple threads the insert produces 3 parts causing undesired deduplication. - When using a single thread the insert produces 1 part without deduplication. -1 -4 +3 +6 This bug has been introduced in CH 24.2+. See https://github.com/ClickHouse/ClickHouse/pull/59448 - When using remote function and multiple threads the insert produces 3 parts causing undesired deduplication. - When using remote function and a single thread the insert produces 1 part without deduplication. -1 -4 +3 +6 From 9f5fe176ada41ba7bf72f19df53d79f65290a401 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 4 Mar 2024 14:41:01 +0100 Subject: [PATCH 099/197] Catch exceptions on finalize --- src/Server/InterserverIOHTTPHandler.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index c41d68bab02..d2e0ed93667 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -117,7 +117,17 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe if (auto [message, success] = checkAuthentication(request); success) { processQuery(request, response, used_output); - used_output.out->finalize(); + + try + { + used_output.out->finalize(); + } + catch (...) + { + tryLogCurrentException(log, "Failed to finalize response write buffer"); + return; + } + LOG_DEBUG(log, "Done processing query"); } else From f2387262ce9b4e440bf39130e4c08620a523371a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 4 Mar 2024 15:32:15 +0100 Subject: [PATCH 100/197] Fix crash with different allow_experimental_analyzer value in subqueries --- src/Interpreters/executeQuery.cpp | 31 +++++++++++++++++++ .../03003_analyzer_setting.reference | 0 .../0_stateless/03003_analyzer_setting.sql | 9 ++++++ 3 files changed, 40 insertions(+) create mode 100644 tests/queries/0_stateless/03003_analyzer_setting.reference create mode 100644 tests/queries/0_stateless/03003_analyzer_setting.sql diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 53fd5088372..18f0cd6601f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -651,6 +651,36 @@ static void setQuerySpecificSettings(ASTPtr & ast, ContextMutablePtr context) } } +void validateAnalyzerSettings(ASTPtr ast, bool context_value) +{ + if (ast->as()) + return; + + bool top_level = context_value; + + std::vector nodes_to_process{ ast }; + while (!nodes_to_process.empty()) + { + auto node = nodes_to_process.back(); + nodes_to_process.pop_back(); + + if (auto * set_query = node->as()) + { + if (auto * value = set_query->changes.tryGet("allow_experimental_analyzer")) + { + if (top_level != value->safeGet()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting 'allow_experimental_analyzer' is changed in the subquery. Top level value: {}", top_level); + } + } + + for (auto child : node->children) + { + if (child) + nodes_to_process.push_back(std::move(child)); + } + } +} + static std::tuple executeQueryImpl( const char * begin, const char * end, @@ -861,6 +891,7 @@ static std::tuple executeQueryImpl( /// Interpret SETTINGS clauses as early as possible (before invoking the corresponding interpreter), /// to allow settings to take effect. InterpreterSetQuery::applySettingsFromQuery(ast, context); + validateAnalyzerSettings(ast, context->getSettingsRef().allow_experimental_analyzer); if (auto * insert_query = ast->as()) insert_query->tail = istr; diff --git a/tests/queries/0_stateless/03003_analyzer_setting.reference b/tests/queries/0_stateless/03003_analyzer_setting.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03003_analyzer_setting.sql b/tests/queries/0_stateless/03003_analyzer_setting.sql new file mode 100644 index 00000000000..43e1bd0d955 --- /dev/null +++ b/tests/queries/0_stateless/03003_analyzer_setting.sql @@ -0,0 +1,9 @@ +CREATE TABLE test (dummy Int8) ENGINE = Distributed(test_cluster_two_shards, 'system', 'one'); + +SET allow_experimental_analyzer = 0; + +SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 1); -- { serverError LOGICAL_ERROR} + +SET allow_experimental_analyzer = 1; + +SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 0); -- { serverError LOGICAL_ERROR} From 8d106f6c7011e4a938dbc6b6dfa5d5970b8eb5be Mon Sep 17 00:00:00 2001 From: Yong Wang Date: Mon, 4 Mar 2024 06:38:13 -0800 Subject: [PATCH 101/197] Fix_max_query_size_for_kql_compound_operator : update test case for better readability --- tests/queries/0_stateless/02366_kql_mvexpand.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02366_kql_mvexpand.sql b/tests/queries/0_stateless/02366_kql_mvexpand.sql index ac1a6d9a8f4..af336a19638 100644 --- a/tests/queries/0_stateless/02366_kql_mvexpand.sql +++ b/tests/queries/0_stateless/02366_kql_mvexpand.sql @@ -35,5 +35,5 @@ print '-- mv_expand_test_table | mv-expand c to typeof(bool) --'; mv_expand_test_table | mv-expand c to typeof(bool); SET max_query_size = 28; SET dialect='kusto'; -mv_expand_test_table | mv-expand c, d; -- { serverError 62 } +mv_expand_test_table | mv-expand c, d; -- { serverError SYNTAX_ERROR } SET max_query_size=262144; From b72507fdf6fa74e40e42b2e8bc1af56b7bf93725 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Mon, 4 Mar 2024 15:48:38 +0100 Subject: [PATCH 102/197] Simplify test now that the cause is clear --- ...001_insert_threads_deduplication.reference | 8 -- .../03001_insert_threads_deduplication.sh | 84 ------------------- .../03001_insert_threads_deduplication.sql | 42 ++++++++++ 3 files changed, 42 insertions(+), 92 deletions(-) delete mode 100755 tests/queries/0_stateless/03001_insert_threads_deduplication.sh create mode 100644 tests/queries/0_stateless/03001_insert_threads_deduplication.sql diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.reference b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference index 0791b98cc09..2559e5c49e7 100644 --- a/tests/queries/0_stateless/03001_insert_threads_deduplication.reference +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference @@ -1,10 +1,2 @@ -This bug has been there forever. Present in 22.2 -- When using multiple threads the insert produces 3 parts causing undesired deduplication. -- When using a single thread the insert produces 1 part without deduplication. -3 -6 -This bug has been introduced in CH 24.2+. See https://github.com/ClickHouse/ClickHouse/pull/59448 -- When using remote function and multiple threads the insert produces 3 parts causing undesired deduplication. -- When using remote function and a single thread the insert produces 1 part without deduplication. 3 6 diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.sh b/tests/queries/0_stateless/03001_insert_threads_deduplication.sh deleted file mode 100755 index 154e578a7a8..00000000000 --- a/tests/queries/0_stateless/03001_insert_threads_deduplication.sh +++ /dev/null @@ -1,84 +0,0 @@ -#!/bin/bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -echo " -DROP TABLE IF EXISTS landing SYNC; -CREATE TABLE landing -( - timestamp DateTime64(3), - status String, - id String -) -ENGINE = MergeTree() -ORDER BY timestamp; - -SYSTEM STOP MERGES landing; -- Stopping merges to force 3 parts - -INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; -INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; -INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; - -DROP TABLE IF EXISTS ds SYNC; - -CREATE TABLE ds -( - timestamp DateTime64(3), - status String, - id String -) -ENGINE = MergeTree() -ORDER BY timestamp -SETTINGS non_replicated_deduplication_window=1000; - -SELECT 'This bug has been there forever. Present in 22.2'; -SELECT '- When using multiple threads the insert produces 3 parts causing undesired deduplication.'; -SELECT '- When using a single thread the insert produces 1 part without deduplication.'; - -INSERT INTO ds SELECT * FROM landing -SETTINGS insert_deduplicate=1, insert_deduplication_token='token1', - max_insert_threads=5; - -SELECT count() FROM ds; - -INSERT INTO ds SELECT * FROM landing -SETTINGS insert_deduplicate=1, insert_deduplication_token='token2', - max_insert_threads=1; - -SELECT count() FROM ds; -" | $CLICKHOUSE_CLIENT -n - -echo " -CREATE TABLE ds_remote -( - timestamp DateTime64(3), - status String, - id String -) -ENGINE = MergeTree() -ORDER BY timestamp -SETTINGS non_replicated_deduplication_window=1000; - -SELECT 'This bug has been introduced in CH 24.2+. See https://github.com/ClickHouse/ClickHouse/pull/59448'; -SELECT '- When using remote function and multiple threads the insert produces 3 parts causing undesired deduplication.'; -SELECT '- When using remote function and a single thread the insert produces 1 part without deduplication.'; - -INSERT INTO ds_remote SELECT * FROM remote('localhost:$CLICKHOUSE_PORT_TCP', $CLICKHOUSE_DATABASE, landing) -SETTINGS insert_deduplicate=1, insert_deduplication_token='token1', - max_insert_threads=5; - -SELECT count() FROM ds_remote; - -INSERT INTO ds_remote SELECT * FROM remote('localhost:$CLICKHOUSE_PORT_TCP', $CLICKHOUSE_DATABASE, landing) -SETTINGS insert_deduplicate=1, insert_deduplication_token='token2', - max_insert_threads=1; - -SELECT count() FROM ds_remote; -" | $CLICKHOUSE_LOCAL -n - -echo " -DROP TABLE IF EXISTS landing SYNC; -DROP TABLE IF EXISTS ds SYNC; -" | $CLICKHOUSE_CLIENT -n diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.sql b/tests/queries/0_stateless/03001_insert_threads_deduplication.sql new file mode 100644 index 00000000000..5b5cb1d6845 --- /dev/null +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.sql @@ -0,0 +1,42 @@ +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS ds SYNC; + +CREATE TABLE landing +( + timestamp DateTime64(3), + status String, + id String +) +ENGINE = MergeTree() +ORDER BY timestamp; + +SYSTEM STOP MERGES landing; -- Stopping merges to force 3 parts + +INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; +INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; +INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; + +CREATE TABLE ds +( + timestamp DateTime64(3), + status String, + id String +) +ENGINE = MergeTree() +ORDER BY timestamp +SETTINGS non_replicated_deduplication_window=1000; + +INSERT INTO ds SELECT * FROM landing +SETTINGS insert_deduplicate=1, insert_deduplication_token='token1', + max_insert_threads=5; + +SELECT count() FROM ds; + +INSERT INTO ds SELECT * FROM landing +SETTINGS insert_deduplicate=1, insert_deduplication_token='token2', + max_insert_threads=1; + +SELECT count() FROM ds; + +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS ds SYNC; From fb8241c6519029882371b14d8ba0b1a61108c18d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 4 Mar 2024 16:36:46 +0100 Subject: [PATCH 103/197] Better --- src/Server/InterserverIOHTTPHandler.cpp | 33 +++++++++++++------------ 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index d2e0ed93667..28045380cd7 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -91,24 +91,35 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe used_output.out = std::make_shared( response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); + auto finalize_output = [&] + { + try + { + used_output.out->finalize(); + } + catch (...) + { + tryLogCurrentException(log, "Failed to finalize response write buffer"); + } + }; + auto write_response = [&](const std::string & message) { - auto & out = *used_output.out; if (response.sent()) { - out.finalize(); + finalize_output(); return; } try { - writeString(message, out); - out.finalize(); + writeString(message, *used_output.out); + finalize_output(); } catch (...) { tryLogCurrentException(log); - out.finalize(); + finalize_output(); } }; @@ -117,17 +128,7 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe if (auto [message, success] = checkAuthentication(request); success) { processQuery(request, response, used_output); - - try - { - used_output.out->finalize(); - } - catch (...) - { - tryLogCurrentException(log, "Failed to finalize response write buffer"); - return; - } - + finalize_output(); LOG_DEBUG(log, "Done processing query"); } else From 70acf7a930f5991aebccfccaf36bf4106300e588 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Mar 2024 16:16:58 +0000 Subject: [PATCH 104/197] Support files without format extension in Filesystem database --- src/Databases/DatabaseFilesystem.cpp | 15 +-------------- ...database_filesystem_format_detection.reference | 2 ++ .../03003_database_filesystem_format_detection.sh | 15 +++++++++++++++ 3 files changed, 18 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/03003_database_filesystem_format_detection.reference create mode 100755 tests/queries/0_stateless/03003_database_filesystem_format_detection.sh diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index bc6714a6471..5af1e1ae0d2 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -146,20 +146,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont if (!checkTableFilePath(table_path, context_, throw_on_error)) return {}; - String format; - if (throw_on_error) - { - format = FormatFactory::instance().getFormatFromFileName(table_path); - } - else - { - auto format_maybe = FormatFactory::instance().tryGetFormatFromFileName(table_path); - if (!format_maybe) - return {}; - format = *format_maybe; - } - - auto ast_function_ptr = makeASTFunction("file", std::make_shared(table_path), std::make_shared(format)); + auto ast_function_ptr = makeASTFunction("file", std::make_shared(table_path)); auto table_function = TableFunctionFactory::instance().get(ast_function_ptr, context_); if (!table_function) diff --git a/tests/queries/0_stateless/03003_database_filesystem_format_detection.reference b/tests/queries/0_stateless/03003_database_filesystem_format_detection.reference new file mode 100644 index 00000000000..87d15e83593 --- /dev/null +++ b/tests/queries/0_stateless/03003_database_filesystem_format_detection.reference @@ -0,0 +1,2 @@ +a Nullable(Int64) +42 diff --git a/tests/queries/0_stateless/03003_database_filesystem_format_detection.sh b/tests/queries/0_stateless/03003_database_filesystem_format_detection.sh new file mode 100755 index 00000000000..8b476a1f82e --- /dev/null +++ b/tests/queries/0_stateless/03003_database_filesystem_format_detection.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.data + +echo '{"a" : 42}' > $DATA_FILE +$CLICKHOUSE_LOCAL -q "desc table \`$DATA_FILE\`" +$CLICKHOUSE_LOCAL -q "select * from \`$DATA_FILE\`" + +rm $DATA_FILE + From cc1c3f2da3038e78bc5f2fca3477c71968bae06f Mon Sep 17 00:00:00 2001 From: Brokenice0415 Date: Tue, 5 Mar 2024 00:54:25 +0800 Subject: [PATCH 105/197] add a keeper setting leadership_expiry --- docs/ru/operations/clickhouse-keeper.md | 2 ++ docs/zh/operations/clickhouse-keeper.md | 2 ++ src/Coordination/CoordinationSettings.cpp | 2 ++ src/Coordination/CoordinationSettings.h | 1 + src/Coordination/KeeperServer.cpp | 15 +++++++++++++++ .../test_keeper_four_word_command/test.py | 1 + 6 files changed, 23 insertions(+) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 9f1301d817d..e1d21dd537c 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -38,6 +38,7 @@ ClickHouse Keeper может иÑпользоватьÑÑ ÐºÐ°Ðº равноце - `dead_session_check_period_ms` — чаÑтота, Ñ ÐºÐ¾Ñ‚Ð¾Ñ€Ð¾Ð¹ ClickHouse Keeper проверÑет мертвые ÑеÑÑии и удалÑет их, в миллиÑекундах (по умолчанию: 500). - `election_timeout_lower_bound_ms` — времÑ, поÑле которого поÑледователь может инициировать перевыбор лидера, еÑли не получил от него контрольный Ñигнал (по умолчанию: 1000). - `election_timeout_upper_bound_ms` — времÑ, поÑле которого поÑледователь должен инициировать перевыбор лидера, еÑли не получил от него контрольный Ñигнал (по умолчанию: 2000). +- `leadership_expiry_ms` — ЕÑли лидер не получает ответа от доÑтаточного количеÑтва поÑледователей в течение Ñтого промежутка времени, он добровольно отказываетÑÑ Ð¾Ñ‚ Ñвоего руководÑтва. При наÑтройке 0 автоматичеÑки уÑтанавливаетÑÑ 20 - кратное значение `heart_beat_interval_ms`, а при наÑтройке меньше 0 лидер не отказываетÑÑ Ð¾Ñ‚ лидерÑтва (по умолчанию 0). - `force_sync` — вызывать `fsync` при каждой запиÑи в журнал координации (по умолчанию: true). - `four_letter_word_white_list` — ÑпиÑок разрешенных 4-Ñ… буквенных команд (по умолчанию: "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro"). - `fresh_log_gap` — минимальное отÑтавание от лидера в количеÑтве запиÑей журнала поÑле которого поÑледователь Ñчитает ÑÐµÐ±Ñ Ð°ÐºÑ‚ÑƒÐ°Ð»ÑŒÐ½Ñ‹Ð¼ (по умолчанию: 200). @@ -209,6 +210,7 @@ dead_session_check_period_ms=500 heart_beat_interval_ms=500 election_timeout_lower_bound_ms=1000 election_timeout_upper_bound_ms=2000 +leadership_expiry_ms=0 reserved_log_items=1000000000000000 snapshot_distance=10000 auto_forwarding=true diff --git a/docs/zh/operations/clickhouse-keeper.md b/docs/zh/operations/clickhouse-keeper.md index 6d8a570aa12..e4412be2e30 100644 --- a/docs/zh/operations/clickhouse-keeper.md +++ b/docs/zh/operations/clickhouse-keeper.md @@ -45,6 +45,7 @@ ClickHouse Keeper 完全å¯ä»¥ä½œä¸ºZooKeeper的独立替代å“或者作为Click - `heart_beat_interval_ms` — ClickHouse Keeperçš„leaderå‘é€å¿ƒè·³é¢‘率(毫秒)(默认为500)。 - `election_timeout_lower_bound_ms` — 如果follower在此间隔内没有收到leader的心跳,那么它å¯ä»¥å¯åŠ¨leader选举(默认为1000). - `election_timeout_upper_bound_ms` — 如果follower在此间隔内没有收到leader的心跳,那么它必须å¯åŠ¨leader选举(默认为2000)。 +- `leadership_expiry_ms` — 如果leader在此间隔内没有收到足够的follower回å¤ï¼Œé‚£ä¹ˆä»–会主动放弃领导æƒã€‚当被设置为0时会自动设置为`heart_beat_interval_ms`çš„20å€ï¼Œå½“被设置å°äºŽ0æ—¶leaderä¸ä¼šä¸»åŠ¨æ”¾å¼ƒé¢†å¯¼æƒï¼ˆé»˜è®¤ä¸º0)。 - `rotate_log_storage_interval` — å•ä¸ªæ–‡ä»¶ä¸­å­˜å‚¨çš„日志记录数é‡(默认100000æ¡)。 - `reserved_log_items` — 在压缩之å‰éœ€è¦å­˜å‚¨å¤šå°‘å调日志记录(默认100000)。 - `snapshot_distance` — ClickHouse Keeper创建新快照的频率(以日志记录的数é‡ä¸ºå•ä½)(默认100000)。 @@ -214,6 +215,7 @@ dead_session_check_period_ms=500 heart_beat_interval_ms=500 election_timeout_lower_bound_ms=1000 election_timeout_upper_bound_ms=2000 +leadership_expiry_ms=0 reserved_log_items=1000000000000000 snapshot_distance=10000 auto_forwarding=true diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index ea1acf02450..05f691ca76b 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -114,6 +114,8 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const write_int(static_cast(coordination_settings->election_timeout_lower_bound_ms)); writeText("election_timeout_upper_bound_ms=", buf); write_int(static_cast(coordination_settings->election_timeout_upper_bound_ms)); + writeText("leadership_expiry_ms=", buf); + write_int(static_cast(coordination_settings->leadership_expiry_ms)); writeText("reserved_log_items=", buf); write_int(coordination_settings->reserved_log_items); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index ed0490681b7..3c8f1d271d0 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -26,6 +26,7 @@ struct Settings; M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \ M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \ M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Upper bound of election timer (avoid too often leader elections)", 0) \ + M(Milliseconds, leadership_expiry_ms, 0, "How long will a leader expire after not getting enough peer responses. Set it lower or equal to election_timeout_lower_bound_ms to avoid multiple leaders.", 0) \ M(UInt64, reserved_log_items, 100000, "How many log items to store (don't remove during compaction)", 0) \ M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \ M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index cf4399461ab..80d61518e19 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -316,6 +316,21 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co } } + params.leadership_expiry_ = getValueOrMaxInt32AndLogWarning( + coordination_settings->leadership_expiry_ms.totalMilliseconds(), "leadership_expiry_ms", log); + + if (params.leadership_expiry_ > 0) + { + if (params.leadership_expiry_ < params.election_timeout_lower_bound_) + { + LOG_WARNING( + log, + "leadership_expiry_ is smaller than election_timeout_lower_bound_ms. " + "Notice that too small leadership_expiry_ may make Raft group " + "sensitive to network status."); + } + } + params.reserved_log_items_ = getValueOrMaxInt32AndLogWarning(coordination_settings->reserved_log_items, "reserved_log_items", log); params.snapshot_distance_ = getValueOrMaxInt32AndLogWarning(coordination_settings->snapshot_distance, "snapshot_distance", log); diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 84dd2a2fd93..44b2b50673a 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -266,6 +266,7 @@ def test_cmd_conf(started_cluster): assert result["heart_beat_interval_ms"] == "500" assert result["election_timeout_lower_bound_ms"] == "1000" assert result["election_timeout_upper_bound_ms"] == "2000" + assert result["leadership_expiry_ms"] == "0" assert result["reserved_log_items"] == "100000" assert result["snapshot_distance"] == "75" From 70abdf7a414ee57d59df51f6cf5ec435e2830f9e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Mar 2024 17:32:22 +0000 Subject: [PATCH 106/197] Small improvements in JSON schema inference --- docs/en/interfaces/schema-inference.md | 64 +++++++++++++ src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.h | 3 + src/Formats/EscapingRuleUtils.cpp | 9 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Formats/SchemaInferenceUtils.cpp | 91 +++++++++++-------- ...02982_dont_infer_exponent_floats.reference | 1 + .../02982_dont_infer_exponent_floats.sql | 3 + ...erence_ambiguous_paths_as_string.reference | 3 + ...es_inference_ambiguous_paths_as_string.sql | 4 + 11 files changed, 142 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.reference create mode 100644 tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.sql diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 39ae69eaef4..f2e9136d1db 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -549,6 +549,48 @@ Result: └───────┴─────────────────────────────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` +##### input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects + +Enabling this setting allows to use String type for ambiguous paths during named tuples inference from JSON objects (when `input_format_json_try_infer_named_tuples_from_objects` is enabled) instead of an exception. +It allows to read JSON objects as named Tuples even if there are ambiguous paths. + +Disabled by default. + +**Examples** + +With disabled setting: +```sql +SET input_format_json_try_infer_named_tuples_from_objects = 1; +SET input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects = 0; +DESC format(JSONEachRow, '{"obj" : {"a" : 42}}, {"obj" : {"a" : {"b" : "Hello"}}}'); +``` +Result: + +```text +Code: 636. DB::Exception: The table structure cannot be extracted from a JSONEachRow format file. Error: +Code: 117. DB::Exception: JSON objects have ambiguous paths: 'a' (with type Int64) and 'a.b'. You can enable setting input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects to use String type for path 'a'. (INCORRECT_DATA) (version 24.3.1.1). +You can specify the structure manually. (CANNOT_EXTRACT_TABLE_STRUCTURE) +``` + +With enabled setting: +```sql +SET input_format_json_try_infer_named_tuples_from_objects = 1; +SET input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects = 1; +DESC format(JSONEachRow, '{"obj" : "a" : 42}, {"obj" : {"a" : {"b" : "Hello"}}}'); +SELECT * FROM format(JSONEachRow, '{"obj" : {"a" : 42}}, {"obj" : {"a" : {"b" : "Hello"}}}'); +``` + +Result: +```text +┌─name─┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠+│ obj │ Tuple(a Nullable(String)) │ │ │ │ │ │ +└──────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +┌─obj─────────────────┠+│ ('42') │ +│ ('{"b" : "Hello"}') │ +└─────────────────────┘ +``` + ##### input_format_json_read_objects_as_strings Enabling this setting allows reading nested JSON objects as strings. @@ -1554,6 +1596,28 @@ DESC format(JSONEachRow, $$ └──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` +#### input_format_try_infer_exponent_floats + +If enabled, ClickHouse will try to infer floats in exponential form for text formats (except JSON where numbers in exponential form are always inferred). + +Disabled by default. + +**Example** + +```sql +SET input_format_try_infer_exponent_floats = 1; +DESC format(CSV, +$$1.1E10 +2.3e-12 +42E00 +$$) +``` +```response +┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠+│ c1 │ Nullable(Float64) │ │ │ │ │ │ +└──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` + ## Self describing formats {#self-describing-formats} Self-describing formats contain information about the structure of the data in the data itself, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ae6ea165cc9..3f71223c910 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1018,6 +1018,7 @@ class IColumn; M(Bool, input_format_json_read_objects_as_strings, true, "Allow to parse JSON objects as strings in JSON input formats", 0) \ M(Bool, input_format_json_read_arrays_as_strings, true, "Allow to parse JSON arrays as strings in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_named_tuples_from_objects, true, "Try to infer named tuples from JSON objects in JSON input formats", 0) \ + M(Bool, input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects, false, "Use String type instead of an exception in case of ambiguous paths in JSON objects during named tuples inference", 0) \ M(Bool, input_format_json_infer_incomplete_types_as_strings, true, "Use type String for keys that contains only Nulls or empty objects/arrays during schema inference in JSON input formats", 0) \ M(Bool, input_format_json_named_tuples_as_objects, true, "Deserialize named tuple columns as JSON objects", 0) \ M(Bool, input_format_json_ignore_unknown_keys_in_named_tuple, true, "Ignore unknown keys in json object for named tuples", 0) \ @@ -1025,7 +1026,7 @@ class IColumn; M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ - M(Bool, input_format_try_infer_exponent_floats, false, "Try to infer floats in exponential notation while schema inference in text formats", 0) \ + M(Bool, input_format_try_infer_exponent_floats, false, "Try to infer floats in exponential notation while schema inference in text formats (except JSON, where exponent numbers are always inferred)", 0) \ M(Bool, output_format_markdown_escape_special_characters, false, "Escape special characters in Markdown", 0) \ M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \ M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index face1def4b4..f473d677ecd 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,9 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.3", { + {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, + }}, {"24.2", { {"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 16f8a341e03..577988871f3 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -450,8 +450,10 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo break; case FormatSettings::EscapingRule::JSON: result += fmt::format( - ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_bools_as_strings={}, read_objects_as_strings={}, read_numbers_as_strings={}, " - "read_arrays_as_strings={}, try_infer_objects_as_tuples={}, infer_incomplete_types_as_strings={}, try_infer_objects={}", + ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_bools_as_strings={}, read_objects_as_strings={}, " + "read_numbers_as_strings={}, " + "read_arrays_as_strings={}, try_infer_objects_as_tuples={}, infer_incomplete_types_as_strings={}, try_infer_objects={}, " + "use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers, settings.json.read_bools_as_strings, @@ -460,7 +462,8 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo settings.json.read_arrays_as_strings, settings.json.try_infer_objects_as_tuples, settings.json.infer_incomplete_types_as_strings, - settings.json.allow_object_type); + settings.json.allow_object_type, + settings.json.use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects); break; default: break; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a4a08d762b9..ccead6688a7 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -105,6 +105,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.write_named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects; format_settings.json.skip_null_value_in_named_tuples = settings.output_format_json_skip_null_value_in_named_tuples; format_settings.json.read_named_tuples_as_objects = settings.input_format_json_named_tuples_as_objects; + format_settings.json.use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects = settings.input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects; format_settings.json.defaults_for_missing_elements_in_named_tuple = settings.input_format_json_defaults_for_missing_elements_in_named_tuple; format_settings.json.ignore_unknown_keys_in_named_tuple = settings.input_format_json_ignore_unknown_keys_in_named_tuple; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 01c3632c730..42b21c77cef 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -202,6 +202,7 @@ struct FormatSettings bool quote_decimals = false; bool escape_forward_slashes = true; bool read_named_tuples_as_objects = false; + bool use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects = false; bool write_named_tuples_as_objects = false; bool skip_null_value_in_named_tuples = false; bool defaults_for_missing_elements_in_named_tuple = false; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 06b52e7a7a2..998f97fae0d 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -136,7 +136,7 @@ namespace bool empty() const { return paths.empty(); } - DataTypePtr finalize() const + DataTypePtr finalize(bool use_string_type_for_ambiguous_paths = false) const { if (paths.empty()) throw Exception(ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA, "Cannot infer named Tuple from JSON object because object is empty"); @@ -167,7 +167,7 @@ namespace current_node->leaf_type = type; } - return root_node.getType(); + return root_node.getType(use_string_type_for_ambiguous_paths); } private: @@ -180,7 +180,7 @@ namespace /// Store path to this node for better exception message in case of ambiguous paths. String path; - DataTypePtr getType() const + DataTypePtr getType(bool use_string_type_for_ambiguous_paths) const { /// Check if we have ambiguous paths. /// For example: @@ -191,7 +191,16 @@ namespace /// And after merge we will have ambiguous paths 'a.b.c' : Int32 and 'a.b' : Nullable(Nothing), /// but it's a valid case and we should ignore path 'a.b'. if (leaf_type && !isNothing(removeNullable(leaf_type)) && !nodes.empty()) - throw Exception(ErrorCodes::INCORRECT_DATA, "JSON objects have ambiguous paths: '{}' with type {} and '{}'", path, leaf_type->getName(), nodes.begin()->second.path); + { + if (use_string_type_for_ambiguous_paths) + return std::make_shared(); + throw Exception( + ErrorCodes::INCORRECT_DATA, + "JSON objects have ambiguous paths: '{}' (with type {}) and '{}'. You can enable setting " + "input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects to use String type " + "for path '{}'", + path, leaf_type->getName(), nodes.begin()->second.path, path); + } if (nodes.empty()) return leaf_type; @@ -203,7 +212,7 @@ namespace for (const auto & [name, node] : nodes) { node_names.push_back(name); - node_types.push_back(node.getType()); + node_types.push_back(node.getType(use_string_type_for_ambiguous_paths)); } return std::make_shared(std::move(node_types), std::move(node_names)); @@ -866,13 +875,15 @@ namespace return std::make_shared(nested_types); } + template bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings) { - if (settings.try_infer_exponent_floats) + if (is_json || settings.try_infer_exponent_floats) return tryReadFloatText(value, buf); return tryReadFloatTextNoExponent(value, buf); } + template DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings) { if (buf.eof()) @@ -911,7 +922,7 @@ namespace buf.position() = number_start; } - if (tryReadFloat(tmp_float, buf, settings)) + if (tryReadFloat(tmp_float, buf, settings)) { if (read_int && buf.position() == int_end) return std::make_shared(); @@ -945,7 +956,7 @@ namespace peekable_buf.rollbackToCheckpoint(true); } - if (tryReadFloat(tmp_float, peekable_buf, settings)) + if (tryReadFloat(tmp_float, peekable_buf, settings)) { /// Float parsing reads no fewer bytes than integer parsing, /// so position of the buffer is either the same, or further. @@ -957,7 +968,7 @@ namespace return std::make_shared(); } } - else if (tryReadFloat(tmp_float, buf, settings)) + else if (tryReadFloat(tmp_float, buf, settings)) { return std::make_shared(); } @@ -966,6 +977,36 @@ namespace return nullptr; } + template + DataTypePtr tryInferNumberFromStringImpl(std::string_view field, const FormatSettings & settings) + { + ReadBufferFromString buf(field); + + if (settings.try_infer_integers) + { + Int64 tmp_int; + if (tryReadIntText(tmp_int, buf) && buf.eof()) + return std::make_shared(); + + /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. + buf.position() = buf.buffer().begin(); + + /// In case of Int64 overflow, try to infer UInt64 + UInt64 tmp_uint; + if (tryReadIntText(tmp_uint, buf) && buf.eof()) + return std::make_shared(); + } + + /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. + buf.position() = buf.buffer().begin(); + + Float64 tmp; + if (tryReadFloat(tmp, buf, settings) && buf.eof()) + return std::make_shared(); + + return nullptr; + } + template DataTypePtr tryInferString(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info) { @@ -995,7 +1036,7 @@ namespace { if (settings.json.try_infer_numbers_from_strings) { - if (auto number_type = tryInferNumberFromString(field, settings)) + if (auto number_type = tryInferNumberFromStringImpl(field, settings)) { json_info->numbers_parsed_from_json_strings.insert(number_type.get()); return number_type; @@ -1238,7 +1279,7 @@ namespace } /// Number - return tryInferNumber(buf, settings); + return tryInferNumber(buf, settings); } } @@ -1294,7 +1335,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F return; } - data_type = json_paths->finalize(); + data_type = json_paths->finalize(settings.json.use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects); transformFinalInferredJSONTypeIfNeededImpl(data_type, settings, json_info, remain_nothing_types); return; } @@ -1377,31 +1418,7 @@ void transformFinalInferredJSONTypeIfNeeded(DataTypePtr & data_type, const Forma DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSettings & settings) { - ReadBufferFromString buf(field); - - if (settings.try_infer_integers) - { - Int64 tmp_int; - if (tryReadIntText(tmp_int, buf) && buf.eof()) - return std::make_shared(); - - /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. - buf.position() = buf.buffer().begin(); - - /// In case of Int64 overflow, try to infer UInt64 - UInt64 tmp_uint; - if (tryReadIntText(tmp_uint, buf) && buf.eof()) - return std::make_shared(); - } - - /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. - buf.position() = buf.buffer().begin(); - - Float64 tmp; - if (tryReadFloat(tmp, buf, settings) && buf.eof()) - return std::make_shared(); - - return nullptr; + return tryInferNumberFromStringImpl(field, settings); } DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const FormatSettings & settings) diff --git a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference index b6d1ff865e5..47e9b86237a 100644 --- a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference +++ b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.reference @@ -1,2 +1,3 @@ c1 Nullable(String) c1 Nullable(Float64) +x Nullable(Float64) diff --git a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql index 2a281e898f1..4f78855f5ce 100644 --- a/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql +++ b/tests/queries/0_stateless/02982_dont_infer_exponent_floats.sql @@ -1,2 +1,5 @@ DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 0; DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 1; +-- This setting should not take affect on JSON formats +DESC format(JSONEachRow, '{"x" : 1.1e20}') settings input_format_try_infer_exponent_floats = 0; + diff --git a/tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.reference b/tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.reference new file mode 100644 index 00000000000..0318b136ade --- /dev/null +++ b/tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.reference @@ -0,0 +1,3 @@ +obj Tuple(\n a Nullable(String)) +('42') +('{"b" : 42}') diff --git a/tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.sql b/tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.sql new file mode 100644 index 00000000000..4b986c94868 --- /dev/null +++ b/tests/queries/0_stateless/03004_json_named_tuples_inference_ambiguous_paths_as_string.sql @@ -0,0 +1,4 @@ +set input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=1; +desc format(JSONEachRow, '{"obj" : {"a" : 42}}, {"obj" : {"a" : {"b" : 42}}}'); +select * from format(JSONEachRow, '{"obj" : {"a" : 42}}, {"obj" : {"a" : {"b" : 42}}}'); + From d3c42b5a24e7472cd6d613adfba94e99aa4a7401 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 4 Mar 2024 17:36:06 +0000 Subject: [PATCH 107/197] Fix test flakiness + make index_granularity stable, it's randomized in tests --- tests/queries/0_stateless/03000_minmax_index_first.sql | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03000_minmax_index_first.sql b/tests/queries/0_stateless/03000_minmax_index_first.sql index 5dae245a0a0..5d2896c792a 100644 --- a/tests/queries/0_stateless/03000_minmax_index_first.sql +++ b/tests/queries/0_stateless/03000_minmax_index_first.sql @@ -8,8 +8,9 @@ CREATE TABLE skip_table INDEX v_mm v TYPE minmax GRANULARITY 2 ) ENGINE = MergeTree -PRIMARY KEY k; +PRIMARY KEY k +SETTINGS index_granularity = 8192; -INSERT INTO skip_table SELECT number, intDiv(number, 4096) FROM numbers(1000000); +INSERT INTO skip_table SELECT number, intDiv(number, 4096) FROM numbers(100000); SELECT trim(explain) FROM ( EXPLAIN indexes = 1 SELECT * FROM skip_table WHERE v = 125) WHERE explain like '%Name%'; From 2d8ca99860eb01baa5694b60b701f5f9d2874778 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Mar 2024 17:48:47 +0000 Subject: [PATCH 108/197] Refactor more system storages. --- src/Storages/StorageMergeTreeIndex.cpp | 79 ++++++++++++++--- src/Storages/StorageMergeTreeIndex.h | 8 +- src/Storages/System/StorageSystemColumns.cpp | 77 ++++++++++++++-- src/Storages/System/StorageSystemColumns.h | 3 +- .../StorageSystemDataSkippingIndices.cpp | 66 +++++++++++++- .../System/StorageSystemDataSkippingIndices.h | 3 +- .../System/StorageSystemDetachedParts.cpp | 48 +++++++++- .../System/StorageSystemDetachedParts.h | 17 ++-- .../StorageSystemDroppedTablesParts.cpp | 4 +- .../System/StorageSystemDroppedTablesParts.h | 6 +- .../System/StorageSystemPartsBase.cpp | 60 +++++++++++-- src/Storages/System/StorageSystemPartsBase.h | 38 +++++++- src/Storages/System/StorageSystemReplicas.cpp | 88 ++++++++++++++----- src/Storages/System/StorageSystemReplicas.h | 7 +- 14 files changed, 421 insertions(+), 83 deletions(-) diff --git a/src/Storages/StorageMergeTreeIndex.cpp b/src/Storages/StorageMergeTreeIndex.cpp index d875611bb50..5f89849e920 100644 --- a/src/Storages/StorageMergeTreeIndex.cpp +++ b/src/Storages/StorageMergeTreeIndex.cpp @@ -15,6 +15,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -216,7 +219,6 @@ StorageMergeTreeIndex::StorageMergeTreeIndex( : IStorage(table_id_) , source_table(source_table_) , with_marks(with_marks_) - , log(&Poco::Logger::get("StorageMergeTreeIndex")) { const auto * merge_tree = dynamic_cast(source_table.get()); if (!merge_tree) @@ -230,7 +232,47 @@ StorageMergeTreeIndex::StorageMergeTreeIndex( setInMemoryMetadata(storage_metadata); } -Pipe StorageMergeTreeIndex::read( +class ReadFromMergeTreeIndex : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromMergeTreeIndex"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromMergeTreeIndex( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , log(&Poco::Logger::get("StorageMergeTreeIndex")) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::shared_ptr storage; + Poco::Logger * log; + const ActionsDAG::Node * predicate = nullptr; +}; + +void ReadFromMergeTreeIndex::applyFilters(ActionDAGNodes added_filter_nodes) +{ + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); +} + +void StorageMergeTreeIndex::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -265,21 +307,32 @@ Pipe StorageMergeTreeIndex::read( context->checkAccess(AccessType::SELECT, source_table->getStorageID(), columns_from_storage); - auto header = storage_snapshot->getSampleBlockForColumns(column_names); - auto filtered_parts = getFilteredDataParts(query_info, context); + auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); - LOG_DEBUG(log, "Reading index{}from {} parts of table {}", - with_marks ? " with marks " : " ", - filtered_parts.size(), - source_table->getStorageID().getNameForLogs()); + auto this_ptr = std::static_pointer_cast(shared_from_this()); - return Pipe(std::make_shared(std::move(header), key_sample_block, std::move(filtered_parts), context, with_marks)); + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(sample_block), std::move(this_ptr)); + + query_plan.addStep(std::move(reading)); } -MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(SelectQueryInfo & query_info, const ContextPtr & context) const +void ReadFromMergeTreeIndex::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - const auto * select_query = query_info.query->as(); - if (!select_query || !select_query->where()) + auto filtered_parts = storage->getFilteredDataParts(predicate, context); + + LOG_DEBUG(log, "Reading index{}from {} parts of table {}", + storage->with_marks ? " with marks " : " ", + filtered_parts.size(), + storage->source_table->getStorageID().getNameForLogs()); + + pipeline.init(Pipe(std::make_shared(getOutputStream().header, storage->key_sample_block, std::move(filtered_parts), context, storage->with_marks))); +} + +MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(const ActionsDAG::Node * predicate, const ContextPtr & context) const +{ + if (!predicate) return data_parts; auto all_part_names = ColumnString::create(); @@ -287,7 +340,7 @@ MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(Selec all_part_names->insert(part->name); Block filtered_block{{std::move(all_part_names), std::make_shared(), part_name_column.name}}; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); if (!filtered_block.rows()) return {}; diff --git a/src/Storages/StorageMergeTreeIndex.h b/src/Storages/StorageMergeTreeIndex.h index b610d391655..a1fb61d5a56 100644 --- a/src/Storages/StorageMergeTreeIndex.h +++ b/src/Storages/StorageMergeTreeIndex.h @@ -21,7 +21,8 @@ public: const ColumnsDescription & columns, bool with_marks_); - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -33,14 +34,15 @@ public: String getName() const override { return "MergeTreeIndex"; } private: - MergeTreeData::DataPartsVector getFilteredDataParts(SelectQueryInfo & query_info, const ContextPtr & context) const; + friend class ReadFromMergeTreeIndex; + + MergeTreeData::DataPartsVector getFilteredDataParts(const ActionsDAG::Node * predicate, const ContextPtr & context) const; StoragePtr source_table; bool with_marks; MergeTreeData::DataPartsVector data_parts; Block key_sample_block; - Poco::Logger * log; }; } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index ad65f6f5476..6bc1208a6a9 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -16,6 +16,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -291,8 +294,51 @@ private: std::chrono::milliseconds lock_acquire_timeout; }; +class ReadFromSystemColumns : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemColumns"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; -Pipe StorageSystemColumns::read( + ReadFromSystemColumns( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::shared_ptr storage; + std::vector columns_mask; + const size_t max_block_size; + const ActionsDAG::Node * predicate = nullptr; +}; + +void ReadFromSystemColumns::applyFilters(ActionDAGNodes added_filter_nodes) +{ + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); +} + +void StorageSystemColumns::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -306,9 +352,22 @@ Pipe StorageSystemColumns::read( auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(header), std::move(this_ptr), std::move(columns_mask), max_block_size); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ Block block_to_filter; Storages storages; Pipes pipes; + auto header = getOutputStream().header; { /// Add `database` column. @@ -338,12 +397,13 @@ Pipe StorageSystemColumns::read( block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared(), "database")); /// Filter block with `database` column. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context); if (!block_to_filter.rows()) { - pipes.emplace_back(std::make_shared(header)); - return Pipe::unitePipes(std::move(pipes)); + pipes.emplace_back(std::make_shared(std::move(header))); + pipeline.init(Pipe::unitePipes(std::move(pipes))); + return; } ColumnPtr & database_column = block_to_filter.getByName("database").column; @@ -384,12 +444,13 @@ Pipe StorageSystemColumns::read( } /// Filter block with `database` and `table` columns. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context); if (!block_to_filter.rows()) { - pipes.emplace_back(std::make_shared(header)); - return Pipe::unitePipes(std::move(pipes)); + pipes.emplace_back(std::make_shared(std::move(header))); + pipeline.init(Pipe::unitePipes(std::move(pipes))); + return; } ColumnPtr filtered_database_column = block_to_filter.getByName("database").column; @@ -400,7 +461,7 @@ Pipe StorageSystemColumns::read( std::move(filtered_database_column), std::move(filtered_table_column), std::move(storages), context)); - return Pipe::unitePipes(std::move(pipes)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); } } diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 7b4b5dd8fb3..22b2541d93f 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -17,7 +17,8 @@ public: std::string getName() const override { return "SystemColumns"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 0c4eb197efd..2fa74ef23e6 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -12,7 +12,10 @@ #include #include #include +#include +#include #include +#include namespace DB @@ -176,7 +179,51 @@ private: DatabaseTablesIteratorPtr tables_it; }; -Pipe StorageSystemDataSkippingIndices::read( +class ReadFromSystemDataSkippingIndices : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemDataSkippingIndices"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemDataSkippingIndices( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::shared_ptr storage; + std::vector columns_mask; + const size_t max_block_size; + const ActionsDAG::Node * predicate = nullptr; +}; + +void ReadFromSystemDataSkippingIndices::applyFilters(ActionDAGNodes added_filter_nodes) +{ + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); +} + +void StorageSystemDataSkippingIndices::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -190,6 +237,17 @@ Pipe StorageSystemDataSkippingIndices::read( auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(header), std::move(this_ptr), std::move(columns_mask), max_block_size); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemDataSkippingIndices::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ MutableColumnPtr column = ColumnString::create(); const auto databases = DatabaseCatalog::instance().getDatabases(); @@ -207,11 +265,11 @@ Pipe StorageSystemDataSkippingIndices::read( /// Condition on "database" in a query acts like an index. Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); ColumnPtr & filtered_databases = block.getByPosition(0).column; - return Pipe(std::make_shared( - std::move(columns_mask), std::move(header), max_block_size, std::move(filtered_databases), context)); + pipeline.init(Pipe(std::make_shared( + std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases), context))); } } diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.h b/src/Storages/System/StorageSystemDataSkippingIndices.h index 8a1e8c159b4..8bf1da98368 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.h +++ b/src/Storages/System/StorageSystemDataSkippingIndices.h @@ -14,7 +14,8 @@ public: std::string getName() const override { return "SystemDataSkippingIndices"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 3dae43976f7..913983952c3 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include @@ -285,7 +287,34 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i setInMemoryMetadata(storage_metadata); } -Pipe StorageSystemDetachedParts::read( +class ReadFromSystemDetachedParts : public ReadFromSystemPartsBase +{ +public: + ReadFromSystemDetachedParts( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + size_t max_block_size_, + size_t num_streams_) + : ReadFromSystemPartsBase(column_names_, query_info_, storage_snapshot_, context_, sample_block, std::move(storage_), std::move(columns_mask_), false) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + {} + + std::string getName() const override { return "ReadFromSystemDetachedParts"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + +private: + const size_t max_block_size; + const size_t num_streams; +}; + +void StorageSystemDetachedParts::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -299,17 +328,28 @@ Pipe StorageSystemDetachedParts::read( auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); - auto state = std::make_shared(StoragesInfoStream(query_info, context)); + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(header), std::move(this_ptr), std::move(columns_mask), max_block_size, num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemDetachedParts::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + auto state = std::make_shared(StoragesInfoStream(predicate, context)); Pipe pipe; for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(header.cloneEmpty(), state, columns_mask, max_block_size); + auto source = std::make_shared(getOutputStream().header, state, columns_mask, max_block_size); pipe.addSource(std::move(source)); } - return pipe; + pipeline.init(std::move(pipe)); } } diff --git a/src/Storages/System/StorageSystemDetachedParts.h b/src/Storages/System/StorageSystemDetachedParts.h index 20ac69f0eea..52b964e3b3c 100644 --- a/src/Storages/System/StorageSystemDetachedParts.h +++ b/src/Storages/System/StorageSystemDetachedParts.h @@ -20,14 +20,15 @@ public: bool isSystemStorage() const override { return true; } protected: - Pipe read( - const Names & /* column_names */, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_block_size*/, - size_t /*num_streams*/) override; + void read( + QueryPlan & query_plan, + const Names & /* column_names */, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + size_t /*num_streams*/) override; }; } diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp index bbe99c688c9..20baeee1d3b 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.cpp +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -10,7 +10,7 @@ namespace DB { -StoragesDroppedInfoStream::StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context) +StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAG::Node * predicate, ContextPtr context) : StoragesInfoStreamBase(context) { /// Will apply WHERE to subset of columns and then add more columns. @@ -73,7 +73,7 @@ StoragesDroppedInfoStream::StoragesDroppedInfoStream(const SelectQueryInfo & que if (block_to_filter.rows()) { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.h b/src/Storages/System/StorageSystemDroppedTablesParts.h index f548697a6a9..a44abea7285 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.h +++ b/src/Storages/System/StorageSystemDroppedTablesParts.h @@ -9,7 +9,7 @@ namespace DB class StoragesDroppedInfoStream : public StoragesInfoStreamBase { public: - StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context); + StoragesDroppedInfoStream(const ActionsDAG::Node * predicate, ContextPtr context); protected: bool tryLockTable(StoragesInfo &) override { @@ -30,9 +30,9 @@ public: std::string getName() const override { return "SystemDroppedTablesParts"; } protected: - std::unique_ptr getStoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) override + std::unique_ptr getStoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context) override { - return std::make_unique(query_info, context); + return std::make_unique(predicate, context); } }; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 48dab8c4777..16e2fd080fb 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -2,6 +2,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -79,7 +82,7 @@ StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, boo return data->getProjectionPartsVectorForInternalUsage({State::Active}, &state); } -StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) +StoragesInfoStream::StoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context) : StoragesInfoStreamBase(context) { /// Will apply WHERE to subset of columns and then add more columns. @@ -111,7 +114,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte std::move(database_column_mut), std::make_shared(), "database")); /// Filter block_to_filter with column 'database'. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context); rows = block_to_filter.rows(); /// Block contains new columns, update database_column. @@ -190,7 +193,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte if (rows) { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context); rows = block_to_filter.rows(); } @@ -201,7 +204,36 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte } -Pipe StorageSystemPartsBase::read( +ReadFromSystemPartsBase::ReadFromSystemPartsBase( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + bool has_state_column_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , columns_mask(std::move(columns_mask_)) + , has_state_column(has_state_column_) +{ +} + +void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes) +{ + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); +} + +void StorageSystemPartsBase::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -212,20 +244,32 @@ Pipe StorageSystemPartsBase::read( { bool has_state_column = hasStateColumn(column_names, storage_snapshot); - auto stream = getStoragesInfoStream(query_info, context); - /// Create the result. Block sample = storage_snapshot->metadata->getSampleBlock(); auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample, column_names); + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(header), std::move(this_ptr), std::move(columns_mask), has_state_column); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemPartsBase::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + auto stream = storage->getStoragesInfoStream(predicate, context); + auto header = getOutputStream().header; + MutableColumns res_columns = header.cloneEmptyColumns(); if (has_state_column) res_columns.push_back(ColumnString::create()); while (StoragesInfo info = stream->next()) { - processNextStorage(context, res_columns, columns_mask, info, has_state_column); + storage->processNextStorage(context, res_columns, columns_mask, info, has_state_column); } if (has_state_column) @@ -234,7 +278,7 @@ Pipe StorageSystemPartsBase::read( UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); - return Pipe(std::make_shared(std::move(header), std::move(chunk))); + pipeline.init(Pipe(std::make_shared(std::move(header), std::move(chunk)))); } diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 1127a8906e5..2b4055373d1 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -114,7 +115,7 @@ protected: class StoragesInfoStream : public StoragesInfoStreamBase { public: - StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context); + StoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context); }; /** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family. @@ -122,7 +123,8 @@ public: class StorageSystemPartsBase : public IStorage { public: - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -139,17 +141,45 @@ private: static bool hasStateColumn(const Names & column_names, const StorageSnapshotPtr & storage_snapshot); protected: + friend class ReadFromSystemPartsBase; + const FormatSettings format_settings = {}; StorageSystemPartsBase(const StorageID & table_id_, ColumnsDescription && columns); - virtual std::unique_ptr getStoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) + virtual std::unique_ptr getStoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context) { - return std::make_unique(query_info, context); + return std::make_unique(predicate, context); } virtual void processNextStorage(ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) = 0; }; +class ReadFromSystemPartsBase : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemPartsBase"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemPartsBase( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + bool has_state_column_); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +protected: + std::shared_ptr storage; + std::vector columns_mask; + const bool has_state_column; + const ActionsDAG::Node * predicate = nullptr; +}; + + } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index eeb3db342b4..26dead8cb01 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -14,6 +14,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -182,13 +185,6 @@ public: , requests_with_zk_fields(max_threads) {} - Pipe read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context); - -private: StatusRequestsPool requests_without_zk_fields; StatusRequestsPool requests_with_zk_fields; }; @@ -241,8 +237,51 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) StorageSystemReplicas::~StorageSystemReplicas() = default; +class ReadFromSystemReplicas : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemReplicas"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; -Pipe StorageSystemReplicas::read( + ReadFromSystemReplicas( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::map> replicated_tables_, + bool with_zk_fields_, + std::shared_ptr impl_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , replicated_tables(std::move(replicated_tables_)) + , with_zk_fields(with_zk_fields_) + , impl(std::move(impl_)) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::map> replicated_tables; + const bool with_zk_fields; + std::shared_ptr impl; + const ActionsDAG::Node * predicate = nullptr; +}; + +void ReadFromSystemReplicas::applyFilters(ActionDAGNodes added_filter_nodes) +{ + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); +} + +void StorageSystemReplicas::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -250,15 +289,6 @@ Pipe StorageSystemReplicas::read( QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, const size_t /*num_streams*/) -{ - return impl->read(column_names, storage_snapshot, query_info, context); -} - -Pipe StorageSystemReplicasImpl::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context) { storage_snapshot->check(column_names); @@ -305,6 +335,18 @@ Pipe StorageSystemReplicasImpl::read( } } + auto header = storage_snapshot->metadata->getSampleBlock(); + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(header), std::move(replicated_tables), with_zk_fields, impl); // /*std::move(this_ptr),*/ std::move(columns_mask), max_block_size); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemReplicas::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + auto header = getOutputStream().header; + MutableColumnPtr col_database_mut = ColumnString::create(); MutableColumnPtr col_table_mut = ColumnString::create(); MutableColumnPtr col_engine_mut = ColumnString::create(); @@ -332,10 +374,14 @@ Pipe StorageSystemReplicasImpl::read( { col_engine, std::make_shared(), "engine" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); if (!filtered_block.rows()) - return {}; + { + auto source = std::make_shared(std::move(header)); + pipeline.init(Pipe(std::move(source))); + return; + } col_database = filtered_block.getByName("database").column; col_table = filtered_block.getByName("table").column; @@ -347,7 +393,7 @@ Pipe StorageSystemReplicasImpl::read( size_t tables_size = col_database->size(); /// Use separate queues for requests with and without ZooKeeper fields. - StatusRequestsPool & get_status_requests = with_zk_fields ? requests_with_zk_fields : requests_without_zk_fields; + StatusRequestsPool & get_status_requests = with_zk_fields ? impl->requests_with_zk_fields : impl->requests_without_zk_fields; QueryStatusPtr query_status = context ? context->getProcessListElement() : nullptr; @@ -435,7 +481,7 @@ Pipe StorageSystemReplicasImpl::read( UInt64 num_rows = fin_columns.at(0)->size(); Chunk chunk(std::move(fin_columns), num_rows); - return Pipe(std::make_shared(storage_snapshot->metadata->getSampleBlock(), std::move(chunk))); + pipeline.init(Pipe(std::make_shared(header, std::move(chunk)))); } diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index f6bdfc937ea..11c5371310f 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -20,7 +20,8 @@ public: std::string getName() const override { return "SystemReplicas"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -32,7 +33,7 @@ public: bool isSystemStorage() const override { return true; } private: - std::unique_ptr impl; + std::shared_ptr impl; }; } From 9a0546168094d38692725f89677077e32bd144b5 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Mar 2024 17:49:33 +0000 Subject: [PATCH 109/197] Better exception message --- docs/en/interfaces/schema-inference.md | 2 +- src/Formats/SchemaInferenceUtils.cpp | 45 ++++++++++++++------------ 2 files changed, 25 insertions(+), 22 deletions(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index f2e9136d1db..05fae994cbe 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -568,7 +568,7 @@ Result: ```text Code: 636. DB::Exception: The table structure cannot be extracted from a JSONEachRow format file. Error: -Code: 117. DB::Exception: JSON objects have ambiguous paths: 'a' (with type Int64) and 'a.b'. You can enable setting input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects to use String type for path 'a'. (INCORRECT_DATA) (version 24.3.1.1). +Code: 117. DB::Exception: JSON objects have ambiguous data: in some objects path 'a' has type 'Int64' and in some - 'Tuple(b String)'. You can enable setting input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects to use String type for path 'a'. (INCORRECT_DATA) (version 24.3.1.1). You can specify the structure manually. (CANNOT_EXTRACT_TABLE_STRUCTURE) ``` diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 998f97fae0d..cb574551d26 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -182,26 +182,6 @@ namespace DataTypePtr getType(bool use_string_type_for_ambiguous_paths) const { - /// Check if we have ambiguous paths. - /// For example: - /// 'a.b.c' : Int32 and 'a.b' : String - /// Also check if leaf type is Nothing, because the next situation is possible: - /// {"a" : {"b" : null}} -> 'a.b' : Nullable(Nothing) - /// {"a" : {"b" : {"c" : 42}}} -> 'a.b.c' : Int32 - /// And after merge we will have ambiguous paths 'a.b.c' : Int32 and 'a.b' : Nullable(Nothing), - /// but it's a valid case and we should ignore path 'a.b'. - if (leaf_type && !isNothing(removeNullable(leaf_type)) && !nodes.empty()) - { - if (use_string_type_for_ambiguous_paths) - return std::make_shared(); - throw Exception( - ErrorCodes::INCORRECT_DATA, - "JSON objects have ambiguous paths: '{}' (with type {}) and '{}'. You can enable setting " - "input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects to use String type " - "for path '{}'", - path, leaf_type->getName(), nodes.begin()->second.path, path); - } - if (nodes.empty()) return leaf_type; @@ -215,7 +195,30 @@ namespace node_types.push_back(node.getType(use_string_type_for_ambiguous_paths)); } - return std::make_shared(std::move(node_types), std::move(node_names)); + auto tuple_type = std::make_shared(std::move(node_types), std::move(node_names)); + + /// Check if we have ambiguous paths. + /// For example: + /// 'a.b.c' : Int32 and 'a.b' : String + /// Also check if leaf type is Nothing, because the next situation is possible: + /// {"a" : {"b" : null}} -> 'a.b' : Nullable(Nothing) + /// {"a" : {"b" : {"c" : 42}}} -> 'a.b.c' : Int32 + /// And after merge we will have ambiguous paths 'a.b.c' : Int32 and 'a.b' : Nullable(Nothing), + /// but it's a valid case and we should ignore path 'a.b'. + if (leaf_type && !isNothing(removeNullable(leaf_type)) && !nodes.empty()) + { + if (use_string_type_for_ambiguous_paths) + return std::make_shared(); + + throw Exception( + ErrorCodes::INCORRECT_DATA, + "JSON objects have ambiguous data: in some objects path '{}' has type '{}' and in some - '{}'. You can enable setting " + "input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects to use String type " + "for path '{}'", + path, leaf_type->getName(), tuple_type->getName(), path); + } + + return tuple_type; } }; From 47d8f039e56de7299f741613d57fc972e71157d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Mar 2024 18:01:12 +0000 Subject: [PATCH 110/197] Cleanup VirtualColumnUtils --- src/Storages/StorageMerge.cpp | 25 ++- src/Storages/VirtualColumnUtils.cpp | 229 ---------------------------- src/Storages/VirtualColumnUtils.h | 17 --- 3 files changed, 20 insertions(+), 251 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index b827670bd4c..eb45a8fac66 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -63,10 +63,12 @@ #include #include +namespace DB +{ + namespace { -using namespace DB; bool columnIsPhysical(ColumnDefaultKind kind) { return kind == ColumnDefaultKind::Default || kind == ColumnDefaultKind::Materialized; @@ -82,10 +84,23 @@ bool columnDefaultKindHasSameType(ColumnDefaultKind lhs, ColumnDefaultKind rhs) return false; } +/// Adds to the select query section `WITH value AS column_name` +/// +/// For example: +/// - `WITH 9000 as _port`. +void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value) +{ + auto & select = ast->as(); + if (!select.with()) + select.setExpression(ASTSelectQuery::Expression::WITH, std::make_shared()); + + auto literal = std::make_shared(value); + literal->alias = column_name; + literal->prefer_alias_to_column_name = true; + select.with()->children.push_back(literal); } -namespace DB -{ +} namespace ErrorCodes { @@ -928,8 +943,8 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextPtr & modified_ if (!is_storage_merge_engine) { - VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", current_storage_id.table_name); - VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_database", current_storage_id.database_name); + rewriteEntityInAst(modified_query_info.query, "_table", current_storage_id.table_name); + rewriteEntityInAst(modified_query_info.query, "_database", current_storage_id.database_name); } } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 07ac61c110d..8457a018625 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -55,189 +55,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ - -/// Verifying that the function depends only on the specified columns -bool isValidFunction(const ASTPtr & expression, const std::function & is_constant) -{ - const auto * function = expression->as(); - if (function && functionIsInOrGlobalInOperator(function->name)) - { - // Second argument of IN can be a scalar subquery - return isValidFunction(function->arguments->children[0], is_constant); - } - else - return is_constant(expression); -} - -/// Extract all subfunctions of the main conjunction, but depending only on the specified columns -bool extractFunctions(const ASTPtr & expression, const std::function & is_constant, ASTs & result) -{ - const auto * function = expression->as(); - - if (function) - { - if (function->name == "and" || function->name == "indexHint") - { - bool ret = true; - for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, is_constant, result); - return ret; - } - else if (function->name == "or") - { - bool ret = false; - ASTs or_args; - for (const auto & child : function->arguments->children) - ret |= extractFunctions(child, is_constant, or_args); - - if (!or_args.empty()) - { - /// In case of there are less number of arguments for which - /// is_constant() == true, we need to add always-true - /// implicitly to avoid breaking AND invariant. - /// - /// Consider the following: - /// - /// ((value = 10) OR (_table = 'v2')) AND ((_table = 'v1') OR (value = 20)) - /// - /// Without implicit always-true: - /// - /// (_table = 'v2') AND (_table = 'v1') - /// - /// With: - /// - /// (_table = 'v2' OR 1) AND (_table = 'v1' OR 1) -> (_table = 'v2') OR (_table = 'v1') - /// - if (or_args.size() != function->arguments->children.size()) - or_args.push_back(std::make_shared(Field(1))); - result.push_back(makeASTForLogicalOr(std::move(or_args))); - } - return ret; - } - } - - if (isValidFunction(expression, is_constant)) - { - result.push_back(expression->clone()); - return true; - } - else - return false; -} - -/// Construct a conjunction from given functions -ASTPtr buildWhereExpression(ASTs && functions) -{ - if (functions.empty()) - return nullptr; - if (functions.size() == 1) - return functions[0]; - return makeASTForLogicalAnd(std::move(functions)); -} - -} - namespace VirtualColumnUtils { -void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value, const String & func) -{ - auto & select = ast->as(); - if (!select.with()) - select.setExpression(ASTSelectQuery::Expression::WITH, std::make_shared()); - - if (func.empty()) - { - auto literal = std::make_shared(value); - literal->alias = column_name; - literal->prefer_alias_to_column_name = true; - select.with()->children.push_back(literal); - } - else - { - auto literal = std::make_shared(value); - literal->prefer_alias_to_column_name = true; - - auto function = makeASTFunction(func, literal); - function->alias = column_name; - function->prefer_alias_to_column_name = true; - select.with()->children.push_back(function); - } -} - -bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block block, ASTPtr & expression_ast) -{ - if (block.rows() == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot prepare filter with empty block"); - - /// Take the first row of the input block to build a constant block - auto columns = block.getColumns(); - Columns const_columns(columns.size()); - for (size_t i = 0; i < columns.size(); ++i) - { - if (isColumnConst(*columns[i])) - const_columns[i] = columns[i]->cloneResized(1); - else - const_columns[i] = ColumnConst::create(columns[i]->cloneResized(1), 1); - } - - block.setColumns(const_columns); - - bool unmodified = true; - const auto & select = query->as(); - if (!select.where() && !select.prewhere()) - return unmodified; - - // Provide input columns as constant columns to check if an expression is - // constant and depends on the columns from provided block (the last is - // required to allow skipping some conditions for handling OR). - std::function is_constant = [&block, &context](const ASTPtr & expr) - { - auto actions = std::make_shared(block.getColumnsWithTypeAndName()); - PreparedSetsPtr prepared_sets = std::make_shared(); - const NamesAndTypesList source_columns; - const NamesAndTypesList aggregation_keys; - const ColumnNumbersList grouping_set_keys; - - ActionsVisitor::Data visitor_data( - context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true, - { aggregation_keys, grouping_set_keys, GroupByKind::NONE }); - - ActionsVisitor(visitor_data).visit(expr); - actions = visitor_data.getActions(); - auto expr_column_name = expr->getColumnName(); - - const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name); - if (!expr_const_node) - return false; - auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}); - const auto & nodes = filter_actions->getNodes(); - bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node) - { - return block.has(node.result_name); - }); - if (!has_dependent_columns) - return false; - - auto expression_actions = std::make_shared(actions); - auto block_with_constants = block; - expression_actions->execute(block_with_constants, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); - return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column); - }; - - /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. - ASTs functions; - if (select.where()) - unmodified &= extractFunctions(select.where(), is_constant, functions); - if (select.prewhere()) - unmodified &= extractFunctions(select.prewhere(), is_constant, functions); - - expression_ast = buildWhereExpression(std::move(functions)); - return unmodified; -} - static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & context) { for (const auto & node : actions->getNodes()) @@ -294,55 +114,6 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) } } -void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast) -{ - if (block.rows() == 0) - return; - - if (!expression_ast) - prepareFilterBlockWithQuery(query, context, block, expression_ast); - - if (!expression_ast) - return; - - /// Let's analyze and calculate the prepared expression. - auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); - ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); - - makeSets(actions, context); - - Block block_with_filter = block; - actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); - - /// Filter the block. - String filter_column_name = expression_ast->getColumnName(); - ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullIfNeeded(); - if (filter_column->getDataType() != TypeIndex::UInt8) - return; - - ConstantFilterDescription constant_filter(*filter_column); - - if (constant_filter.always_true) - { - return; - } - - if (constant_filter.always_false) - { - block = block.cloneEmpty(); - return; - } - - FilterDescription filter(*filter_column); - - for (size_t i = 0; i < block.columns(); ++i) - { - ColumnPtr & column = block.safeGetByPosition(i).column; - column = column->filter(*filter.data, -1); - } -} - NamesAndTypesList getPathFileAndSizeVirtualsForStorage(NamesAndTypesList storage_columns) { auto default_virtuals = NamesAndTypesList{ diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 7a9b2605339..3e7299b4a63 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -17,23 +17,6 @@ class NamesAndTypesList; namespace VirtualColumnUtils { -/// Adds to the select query section `WITH value AS column_name`, and uses func -/// to wrap the value (if any) -/// -/// For example: -/// - `WITH 9000 as _port`. -/// - `WITH toUInt16(9000) as _port`. -void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value, const String & func = ""); - -/// Prepare `expression_ast` to filter block. Returns true if `expression_ast` is not trimmed, that is, -/// `block` provides all needed columns for `expression_ast`, else return false. -bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block block, ASTPtr & expression_ast); - -/// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query. -/// Only elements of the outer conjunction are considered, depending only on the columns present in the block. -/// If `expression_ast` is passed, use it to filter block. -void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast = {}); - /// Similar to filterBlockWithQuery, but uses ActionsDAG as a predicate. /// Basically it is filterBlockWithDAG(splitFilterDagForAllowedInputs). void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); From d209508753a522782b24b61f4a399e0042d1a0ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Mar 2024 18:26:42 +0000 Subject: [PATCH 111/197] Fixing style. --- src/Storages/VirtualColumnUtils.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 8457a018625..386dcd17733 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -50,11 +50,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace VirtualColumnUtils { From b86fcf0e7ddde02a1aeb07d0fa99504fbf677430 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 4 Mar 2024 19:53:18 +0100 Subject: [PATCH 112/197] Add missing compression clone() in ASTQueryWithOutput --- src/Parsers/ASTQueryWithOutput.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 4bf1e6cb231..b9b07eb7070 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -23,6 +23,16 @@ void ASTQueryWithOutput::cloneOutputOptions(ASTQueryWithOutput & cloned) const cloned.settings_ast = settings_ast->clone(); cloned.children.push_back(cloned.settings_ast); } + if (compression) + { + cloned.compression = compression->clone(); + cloned.children.push_back(cloned.compression); + } + if (compression_level) + { + cloned.compression_level = compression_level->clone(); + cloned.children.push_back(cloned.compression_level); + } } void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const From 37891660be107c41f9883d0d9d3fa8428d49e316 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 4 Mar 2024 20:01:03 +0100 Subject: [PATCH 113/197] More missing resets --- src/Parsers/ASTQueryWithOutput.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index b9b07eb7070..3890ed2347a 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -74,9 +74,12 @@ bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) /// FIXME: try to prettify this cast using `as<>()` if (auto * ast_with_output = dynamic_cast(&ast)) { - ast_with_output->format.reset(); ast_with_output->out_file.reset(); + ast_with_output->format.reset(); ast_with_output->settings_ast.reset(); + ast_with_output->compression.reset(); + ast_with_output->compression_level.reset(); + ast_with_output->children.clear(); return true; } From 680bb3392e080f167ce01e0ebf8f380a43b1de24 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 4 Mar 2024 20:01:27 +0100 Subject: [PATCH 114/197] More missing trash --- src/Parsers/ASTInsertQuery.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTInsertQuery.h b/src/Parsers/ASTInsertQuery.h index 6a4ce078f79..b0f444ed755 100644 --- a/src/Parsers/ASTInsertQuery.h +++ b/src/Parsers/ASTInsertQuery.h @@ -59,11 +59,13 @@ public: if (database) { res->database = database->clone(); res->children.push_back(res->database); } if (table) { res->table = table->clone(); res->children.push_back(res->table); } if (columns) { res->columns = columns->clone(); res->children.push_back(res->columns); } - if (select) { res->select = select->clone(); res->children.push_back(res->select); } - if (watch) { res->watch = watch->clone(); res->children.push_back(res->watch); } if (table_function) { res->table_function = table_function->clone(); res->children.push_back(res->table_function); } if (partition_by) { res->partition_by = partition_by->clone(); res->children.push_back(res->partition_by); } if (settings_ast) { res->settings_ast = settings_ast->clone(); res->children.push_back(res->settings_ast); } + if (select) { res->select = select->clone(); res->children.push_back(res->select); } + if (watch) { res->watch = watch->clone(); res->children.push_back(res->watch); } + if (infile) { res->infile = infile->clone(); res->children.push_back(res->infile); } + if (compression) { res->compression = compression->clone(); res->children.push_back(res->compression); } return res; } From d5825ec80d19c853b7f2ee03cdb296e17abf0e59 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 4 Mar 2024 18:34:41 +0100 Subject: [PATCH 115/197] Faster (almost 2x) mutexes (was slower due to ThreadFuzzer) Recently I noticed that DB::injection() pops up in perf top, so let's optimize it slightly: - Add -fomit-frame-pointer -momit-leaf-frame-pointer explicitly -- almost 0 effect - Add ALWAYS_INLINE for ThreadFuzzer::isStarted() (just in case) - Disable ThreadFuzzer if non of env variables had been set, this is needed to avoid extra checks in DB::injection() - Add ALWAYS_INLINE for ThreadFuzzer::injection() And here are some results for ThreadFuzzer test: - before: elapsed 6.27368 / test time 654 ms - after: elapsed 3.14167 / test time 325 ms - disabled: elapsed 2.46666 / test time 263 ms *But note, it is still slower then with ThreadFuzzer disabled.* Note, that this numbers for AMD 5975WX, for server with 2x Xeon Silver 4216 2.10: - before: elapsed 12.109 / test time 1325 ms - after: elapsed 10.506 / test time 1101 ms - disabled: elapsed 8.41043 / test time 917 ms P.S. I've also tried with the same glibc version as server had - zero changes. Refs: https://gist.github.com/azat/51a5fcc3a40af9f678906a3a6e14e079 Signed-off-by: Azat Khuzhin --- src/CMakeLists.txt | 2 + src/Common/ThreadFuzzer.cpp | 49 +++++++++++++++++++----- src/Common/ThreadFuzzer.h | 3 +- src/Common/tests/gtest_thread_fuzzer.cpp | 36 +++++++++++++++++ 4 files changed, 79 insertions(+), 11 deletions(-) create mode 100644 src/Common/tests/gtest_thread_fuzzer.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index dff70e06ce4..73aa409e995 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -174,6 +174,8 @@ endif () add_library(clickhouse_common_io ${clickhouse_common_io_headers} ${clickhouse_common_io_sources}) +set_source_files_properties(Common/ThreadFuzzer.cpp PROPERTIES COMPILE_FLAGS "-fomit-frame-pointer -momit-leaf-frame-pointer") + add_library (clickhouse_malloc OBJECT Common/malloc.cpp) set_source_files_properties(Common/malloc.cpp PROPERTIES COMPILE_FLAGS "-fno-builtin") diff --git a/src/Common/ThreadFuzzer.cpp b/src/Common/ThreadFuzzer.cpp index 0868613d880..1d944f4a458 100644 --- a/src/Common/ThreadFuzzer.cpp +++ b/src/Common/ThreadFuzzer.cpp @@ -51,7 +51,11 @@ ThreadFuzzer::ThreadFuzzer() { initConfiguration(); if (!isEffective()) + { + /// It has no effect - disable it + stop(); return; + } setup(); } @@ -172,6 +176,8 @@ void ThreadFuzzer::stop() void ThreadFuzzer::start() { + if (!instance().isEffective()) + return; started.store(true, std::memory_order_relaxed); } @@ -180,11 +186,11 @@ bool ThreadFuzzer::isStarted() return started.load(std::memory_order_relaxed); } -static void injection( +static void injectionImpl( double yield_probability, double migrate_probability, double sleep_probability, - double sleep_time_us [[maybe_unused]]) + double sleep_time_us) { DENY_ALLOCATIONS_IN_SCOPE; if (!ThreadFuzzer::isStarted()) @@ -222,6 +228,19 @@ static void injection( } } +static ALWAYS_INLINE void injection( + double yield_probability, + double migrate_probability, + double sleep_probability, + double sleep_time_us) +{ + DENY_ALLOCATIONS_IN_SCOPE; + if (!ThreadFuzzer::isStarted()) + return; + + injectionImpl(yield_probability, migrate_probability, sleep_probability, sleep_time_us); +} + void ThreadFuzzer::maybeInjectSleep() { auto & fuzzer = ThreadFuzzer::instance(); @@ -286,13 +305,13 @@ void ThreadFuzzer::setup() const #if THREAD_FUZZER_WRAP_PTHREAD #define INJECTION_BEFORE(NAME) \ - injection( \ + injectionImpl( \ NAME##_before_yield_probability.load(std::memory_order_relaxed), \ NAME##_before_migrate_probability.load(std::memory_order_relaxed), \ NAME##_before_sleep_probability.load(std::memory_order_relaxed), \ NAME##_before_sleep_time_us.load(std::memory_order_relaxed)); #define INJECTION_AFTER(NAME) \ - injection( \ + injectionImpl( \ NAME##_after_yield_probability.load(std::memory_order_relaxed), \ NAME##_after_migrate_probability.load(std::memory_order_relaxed), \ NAME##_after_sleep_probability.load(std::memory_order_relaxed), \ @@ -383,13 +402,16 @@ static void * getFunctionAddress(const char * name) static constinit RET(*real_##NAME)(__VA_ARGS__) = nullptr; \ extern "C" RET NAME(__VA_ARGS__) \ { \ - INJECTION_BEFORE(NAME); \ + bool thread_fuzzer_enabled = ThreadFuzzer::isStarted(); \ + if (thread_fuzzer_enabled) \ + INJECTION_BEFORE(NAME); \ if (unlikely(!real_##NAME)) { \ real_##NAME = \ reinterpret_cast(getFunctionAddress(#NAME)); \ } \ auto && ret{real_##NAME(arg)}; \ - INJECTION_AFTER(NAME); \ + if (thread_fuzzer_enabled) \ + INJECTION_AFTER(NAME); \ return ret; \ } FOR_EACH_WRAPPED_FUNCTION(MAKE_WRAPPER_USING_DLSYM) @@ -399,10 +421,17 @@ FOR_EACH_WRAPPED_FUNCTION(MAKE_WRAPPER_USING_DLSYM) extern "C" RET __##NAME(__VA_ARGS__); \ extern "C" RET NAME(__VA_ARGS__) \ { \ - INJECTION_BEFORE(NAME); \ - auto && ret{__##NAME(arg)}; \ - INJECTION_AFTER(NAME); \ - return ret; \ + if (!ThreadFuzzer::isStarted()) \ + { \ + return __##NAME(arg); \ + } \ + else \ + { \ + INJECTION_BEFORE(NAME); \ + auto && ret{__##NAME(arg)}; \ + INJECTION_AFTER(NAME); \ + return ret; \ + } \ } FOR_EACH_WRAPPED_FUNCTION(MAKE_WRAPPER_USING_INTERNAL_SYMBOLS) #undef MAKE_WRAPPER_USING_INTERNAL_SYMBOLS diff --git a/src/Common/ThreadFuzzer.h b/src/Common/ThreadFuzzer.h index 9dd55fe7995..1cff27a7588 100644 --- a/src/Common/ThreadFuzzer.h +++ b/src/Common/ThreadFuzzer.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -56,7 +57,7 @@ public: static void stop(); static void start(); - static bool isStarted(); + static bool ALWAYS_INLINE isStarted(); static void maybeInjectSleep(); static void maybeInjectMemoryLimitException(); diff --git a/src/Common/tests/gtest_thread_fuzzer.cpp b/src/Common/tests/gtest_thread_fuzzer.cpp new file mode 100644 index 00000000000..ac2ccebebc6 --- /dev/null +++ b/src/Common/tests/gtest_thread_fuzzer.cpp @@ -0,0 +1,36 @@ +#include +#include +#include +#include +#include +#include + +TEST(ThreadFuzzer, mutex) +{ + /// Initialize ThreadFuzzer::started + DB::ThreadFuzzer::instance().isEffective(); + + std::mutex mutex; + std::atomic elapsed_ns = 0; + + auto func = [&]() + { + Stopwatch watch; + for (size_t i = 0; i < 1e6; ++i) + { + mutex.lock(); + mutex.unlock(); + } + elapsed_ns += watch.elapsedNanoseconds(); + }; + + std::vector> threads(10); + + for (auto & thread : threads) + thread.emplace(func); + + for (auto & thread : threads) + thread->join(); + + std::cout << "elapsed: " << elapsed_ns/1e9 << "\n"; +} From e715ff622d50b3c1dda8c4e90bcb97c431b6a81b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 5 Mar 2024 00:52:13 +0100 Subject: [PATCH 116/197] Better --- docker/test/fuzzer/run-fuzzer.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 1a1888861af..0234ead69d0 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -247,8 +247,11 @@ quit fuzzer_pid=$! echo "Fuzzer pid is $fuzzer_pid" + # The fuzzer_pid belongs to the timeout process. + actual_fuzzer_pid=$(ps -o pid= --ppid "$fuzzer_pid") + echo "Attaching gdb to the fuzzer itself" - gdb -batch -command script.gdb -p $fuzzer_pid & + gdb -batch -command script.gdb -p $actual_fuzzer_pid & # Wait for the fuzzer to complete. # Note that the 'wait || ...' thing is required so that the script doesn't From c735caf44f7dfcc245d6c77ca111dee686e914a8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 5 Mar 2024 11:54:13 +0800 Subject: [PATCH 117/197] Fix typo --- docker/test/upgrade/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 0354c14bef5..580ddb65597 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -81,7 +81,7 @@ remove_keeper_config "create_if_not_exists" "[01]" sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \ | sed "s|azure|azure_blob_storage|" \ > /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp -sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/azure_storage_conf.xml #todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ From b5fb12c6ec580f3bf7a165eb32456c0d196a374c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 5 Mar 2024 11:54:58 +0800 Subject: [PATCH 118/197] Fix typo --- docker/test/upgrade/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 580ddb65597..ec30cbf128e 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -128,7 +128,7 @@ sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-serv sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \ | sed "s|azure|azure_blob_storage|" \ > /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp -sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/azure_storage_conf.xml #todo: remove these after 24.3 released. sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ From 9a3312f12d9a7c25e6eab6b32fb6198973c0e68b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 5 Mar 2024 11:55:32 +0800 Subject: [PATCH 119/197] Remove debug --- docker/test/upgrade/run.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index ec30cbf128e..12151aa6a3c 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -105,8 +105,6 @@ rm /etc/clickhouse-server/users.d/nonconst_timezone.xml rm /etc/clickhouse-server/users.d/s3_cache_new.xml rm /etc/clickhouse-server/users.d/replicated_ddl_entry.xml -cat /etc/clickhouse-server/config.d/storage_conf.xml - start stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log From 4251cc3a117682329a437db0a5e92f876d59821f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 5 Mar 2024 17:05:40 +0800 Subject: [PATCH 120/197] fix building --- src/Functions/multiIf.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 5bf9f2af420..f5cb2375c53 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; } namespace @@ -419,8 +420,13 @@ private: calculateInserts(instructions, rows, inserts); res_data.resize_exact(rows); - if (res_null_map) + if constexpr (nullable_result) + { + if (res_null_map) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid result null_map while result type is nullable"); + res_null_map->resize_exact(rows); + } std::vector data_cols(instructions.size(), nullptr); std::vector null_map_cols(instructions.size(), nullptr); From 7426baa065cc053bb64b2e31d4c240e3e0d8a02d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 5 Mar 2024 10:20:14 +0100 Subject: [PATCH 121/197] Update test --- .../00849_multiple_comma_join_2.sql | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql index 51bf5a2ede1..6530f691087 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql @@ -62,49 +62,49 @@ SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explai --- EXPLAIN QUERY TREE SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4 SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 CROSS JOIN t3 SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 CROSS JOIN t3) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3 SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3) SETTINGS allow_experimental_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3 SETTINGS allow_experimental_analyzer = 1); + EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3) SETTINGS allow_experimental_analyzer = 1; INSERT INTO t1 values (1,1), (2,2), (3,3), (4,4); INSERT INTO t2 values (1,1), (1, Null); From a6f5323c38d8420765332e3e778399de3fc69acf Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 5 Mar 2024 10:09:17 +0000 Subject: [PATCH 122/197] CI: Fix pending status for build report in backports #do_not_test --- tests/ci/build_report_check.py | 2 +- tests/ci/ci_config.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 94e429ad77b..48640f15ac0 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -51,7 +51,7 @@ def main(): builds_for_check = CI_CONFIG.get_builds_for_report( build_check_name, release=pr_info.is_release(), - backport=pr_info.head_ref.startswith("backport"), + backport=pr_info.head_ref.startswith("backport/"), ) required_builds = len(builds_for_check) missing_builds = 0 diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b9c332e8e18..b9ee5670066 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -586,7 +586,7 @@ class CIConfig: Build.PACKAGE_TSAN, Build.PACKAGE_DEBUG, ] - if release and report_name == JobNames.BUILD_CHECK_SPECIAL: + if (release or backport) and report_name == JobNames.BUILD_CHECK_SPECIAL: return [ Build.BINARY_DARWIN, Build.BINARY_DARWIN_AARCH64, From edf2dc91687786742687624af40d1ac93c92f16b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 5 Mar 2024 10:20:57 +0000 Subject: [PATCH 123/197] Update version_date.tsv and changelogs after v23.8.10.43-lts --- docs/changelogs/v23.8.10.43-lts.md | 39 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 40 insertions(+) create mode 100644 docs/changelogs/v23.8.10.43-lts.md diff --git a/docs/changelogs/v23.8.10.43-lts.md b/docs/changelogs/v23.8.10.43-lts.md new file mode 100644 index 00000000000..0093467d129 --- /dev/null +++ b/docs/changelogs/v23.8.10.43-lts.md @@ -0,0 +1,39 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.10.43-lts (a278225bba9) FIXME as compared to v23.8.9.54-lts (192a1d231fa) + +#### Improvement +* Backported in [#58819](https://github.com/ClickHouse/ClickHouse/issues/58819): Add `SYSTEM JEMALLOC PURGE` for purging unused jemalloc pages, `SYSTEM JEMALLOC [ ENABLE | DISABLE | FLUSH ] PROFILE` for controlling jemalloc profile if the profiler is enabled. Add jemalloc-related 4LW command in Keeper: `jmst` for dumping jemalloc stats, `jmfp`, `jmep`, `jmdp` for controlling jemalloc profile if the profiler is enabled. [#58665](https://github.com/ClickHouse/ClickHouse/pull/58665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#60286](https://github.com/ClickHouse/ClickHouse/issues/60286): Copy S3 file GCP fallback to buffer copy in case GCP returned `Internal Error` with `GATEWAY_TIMEOUT` HTTP error code. [#60164](https://github.com/ClickHouse/ClickHouse/pull/60164) ([Maksim Kita](https://github.com/kitaisreal)). + +#### Build/Testing/Packaging Improvement +* Backported in [#59879](https://github.com/ClickHouse/ClickHouse/issues/59879): If you want to run initdb scripts every time when ClickHouse container is starting you shoud initialize environment varible CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS. [#59808](https://github.com/ClickHouse/ClickHouse/pull/59808) ([Alexander Nikolaev](https://github.com/AlexNik)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Background merges correctly use temporary data storage in the cache [#57275](https://github.com/ClickHouse/ClickHouse/pull/57275) ([vdimir](https://github.com/vdimir)). +* MergeTree mutations reuse source part index granularity [#57352](https://github.com/ClickHouse/ClickHouse/pull/57352) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix double destroy call on exception throw in addBatchLookupTable8 [#58745](https://github.com/ClickHouse/ClickHouse/pull/58745) ([Raúl Marín](https://github.com/Algunenano)). +* Fix JSONExtract function for LowCardinality(Nullable) columns [#58808](https://github.com/ClickHouse/ClickHouse/pull/58808) ([vdimir](https://github.com/vdimir)). +* Fix: LIMIT BY and LIMIT in distributed query [#59153](https://github.com/ClickHouse/ClickHouse/pull/59153) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix translate() with FixedString input [#59356](https://github.com/ClickHouse/ClickHouse/pull/59356) ([Raúl Marín](https://github.com/Algunenano)). +* Fix error "Read beyond last offset" for AsynchronousBoundedReadBuffer [#59630](https://github.com/ClickHouse/ClickHouse/pull/59630) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix query start time on non initial queries [#59662](https://github.com/ClickHouse/ClickHouse/pull/59662) ([Raúl Marín](https://github.com/Algunenano)). +* Fix leftPad / rightPad function with FixedString input [#59739](https://github.com/ClickHouse/ClickHouse/pull/59739) ([Raúl Marín](https://github.com/Algunenano)). +* rabbitmq: fix having neither acked nor nacked messages [#59775](https://github.com/ClickHouse/ClickHouse/pull/59775) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix cosineDistance crash with Nullable [#60150](https://github.com/ClickHouse/ClickHouse/pull/60150) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix rare race in external sort/aggregation with temporary data in cache [#58013](https://github.com/ClickHouse/ClickHouse/pull/58013) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible race in ManyAggregatedData dtor. [#58624](https://github.com/ClickHouse/ClickHouse/pull/58624) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix 02720_row_policy_column_with_dots [#59453](https://github.com/ClickHouse/ClickHouse/pull/59453) ([Duc Canh Le](https://github.com/canhld94)). +* Pin python dependencies in stateless tests [#59663](https://github.com/ClickHouse/ClickHouse/pull/59663) ([Raúl Marín](https://github.com/Algunenano)). +* Make ZooKeeper actually sequentialy consistent [#59735](https://github.com/ClickHouse/ClickHouse/pull/59735) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove broken test while we fix it [#60547](https://github.com/ClickHouse/ClickHouse/pull/60547) ([Raúl Marín](https://github.com/Algunenano)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 572ceddf590..53bf705637d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -25,6 +25,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.10.43-lts 2024-03-05 v23.8.9.54-lts 2024-01-05 v23.8.8.20-lts 2023-11-25 v23.8.7.24-lts 2023-11-17 From 738c481cce72716fb8e6cd2295ee22040a2a14a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 5 Mar 2024 10:36:42 +0000 Subject: [PATCH 124/197] Fix fuzzer report --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 9358e88e1e8..ccf450c94f2 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -387,7 +387,7 @@ if [ -f core.zst ]; then fi # Keep all the lines in the paragraphs containing that either contain or don't start with 20... (year) -sed -n '//,/^$/p' s.log | awk '// || !/^20/' server.log > fatal.log ||: +sed -n '//,/^$/p' server.log | awk '// || !/^20/' > fatal.log ||: FATAL_LINK='' if [ -s fatal.log ]; then FATAL_LINK='fatal.log' From ee3edf25894bc902fba96c95a0ad151d7b1a717c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Mar 2024 10:40:45 +0000 Subject: [PATCH 125/197] Fix system parts _state --- src/Storages/System/StorageSystemPartsBase.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 631b316167c..5921fbc8f0f 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -249,6 +249,9 @@ void StorageSystemPartsBase::read( auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample, column_names); + if (has_state_column) + header.insert(ColumnWithTypeAndName(std::make_shared(), "_state")); + auto this_ptr = std::static_pointer_cast(shared_from_this()); auto reading = std::make_unique( @@ -264,17 +267,12 @@ void ReadFromSystemPartsBase::initializePipeline(QueryPipelineBuilder & pipeline auto header = getOutputStream().header; MutableColumns res_columns = header.cloneEmptyColumns(); - if (has_state_column) - res_columns.push_back(ColumnString::create()); while (StoragesInfo info = stream->next()) { storage->processNextStorage(context, res_columns, columns_mask, info, has_state_column); } - if (has_state_column) - header.insert(ColumnWithTypeAndName(std::make_shared(), "_state")); - UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); From 0bd4c6afa0cb470b003216a8d75a85131831873b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 5 Mar 2024 10:49:01 +0100 Subject: [PATCH 126/197] Use while-loop --- src/Storages/StorageS3.cpp | 69 +++++++++++++++++++------------------- 1 file changed, 35 insertions(+), 34 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 6892e75f506..b73965b8774 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -258,45 +258,46 @@ private: KeyWithInfoPtr nextAssumeLocked() { - if (buffer_iter != buffer.end()) + do { - auto answer = *buffer_iter; - ++buffer_iter; - - /// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key. - /// So we get object info lazily here on 'next()' request. - if (!answer->info) + if (buffer_iter != buffer.end()) { - answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); - if (file_progress_callback) - file_progress_callback(FileProgress(0, answer->info->size)); + auto answer = *buffer_iter; + ++buffer_iter; + + /// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key. + /// So we get object info lazily here on 'next()' request. + if (!answer->info) + { + answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); + if (file_progress_callback) + file_progress_callback(FileProgress(0, answer->info->size)); + } + + return answer; } - return answer; - } + if (is_finished) + return {}; - if (is_finished) - return {}; - - try - { - fillInternalBufferAssumeLocked(); - } - catch (...) - { - /// In case of exception thrown while listing new batch of files - /// iterator may be partially initialized and its further using may lead to UB. - /// Iterator is used by several processors from several threads and - /// it may take some time for threads to stop processors and they - /// may still use this iterator after exception is thrown. - /// To avoid this UB, reset the buffer and return defaults for further calls. - is_finished = true; - buffer.clear(); - buffer_iter = buffer.begin(); - throw; - } - - return nextAssumeLocked(); + try + { + fillInternalBufferAssumeLocked(); + } + catch (...) + { + /// In case of exception thrown while listing new batch of files + /// iterator may be partially initialized and its further using may lead to UB. + /// Iterator is used by several processors from several threads and + /// it may take some time for threads to stop processors and they + /// may still use this iterator after exception is thrown. + /// To avoid this UB, reset the buffer and return defaults for further calls. + is_finished = true; + buffer.clear(); + buffer_iter = buffer.begin(); + throw; + } + } while (true); } void fillInternalBufferAssumeLocked() From a80b3ad2f4208a32d7105da202bc6f2ba736510c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 5 Mar 2024 12:15:31 +0100 Subject: [PATCH 127/197] Fix test --- tests/queries/0_stateless/02722_database_filesystem.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index f3af7abcbb3..0c75c15fc69 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -81,8 +81,6 @@ CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); # FILE_DOESNT_EXIST: unknown file ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||: -# BAD_ARGUMENTS: Cannot determine the file format by it's extension -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # Clean ${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;" rm -rd $tmp_dir From d155f266fd25a7a35dccc0d4e3cffce1fbc07304 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 5 Mar 2024 12:15:48 +0100 Subject: [PATCH 128/197] Fix test reference --- tests/queries/0_stateless/02722_database_filesystem.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02722_database_filesystem.reference b/tests/queries/0_stateless/02722_database_filesystem.reference index 17f84cfc49c..2aa7e74befb 100644 --- a/tests/queries/0_stateless/02722_database_filesystem.reference +++ b/tests/queries/0_stateless/02722_database_filesystem.reference @@ -23,4 +23,3 @@ OK OK OK OK -OK From 23ec4d4c6e386b67953cee01be5c740f47e39c78 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 5 Mar 2024 11:53:19 +0000 Subject: [PATCH 129/197] Update version_date.tsv and changelogs after v23.3.20.27-lts --- docs/changelogs/v23.3.20.27-lts.md | 29 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 30 insertions(+) create mode 100644 docs/changelogs/v23.3.20.27-lts.md diff --git a/docs/changelogs/v23.3.20.27-lts.md b/docs/changelogs/v23.3.20.27-lts.md new file mode 100644 index 00000000000..9f49e47f0bc --- /dev/null +++ b/docs/changelogs/v23.3.20.27-lts.md @@ -0,0 +1,29 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.3.20.27-lts (cc974ba4f81) FIXME as compared to v23.3.19.32-lts (c4d4ca8ec02) + +#### Improvement +* Backported in [#58818](https://github.com/ClickHouse/ClickHouse/issues/58818): Add `SYSTEM JEMALLOC PURGE` for purging unused jemalloc pages, `SYSTEM JEMALLOC [ ENABLE | DISABLE | FLUSH ] PROFILE` for controlling jemalloc profile if the profiler is enabled. Add jemalloc-related 4LW command in Keeper: `jmst` for dumping jemalloc stats, `jmfp`, `jmep`, `jmdp` for controlling jemalloc profile if the profiler is enabled. [#58665](https://github.com/ClickHouse/ClickHouse/pull/58665) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Build/Testing/Packaging Improvement +* Backported in [#59877](https://github.com/ClickHouse/ClickHouse/issues/59877): If you want to run initdb scripts every time when ClickHouse container is starting you shoud initialize environment varible CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS. [#59808](https://github.com/ClickHouse/ClickHouse/pull/59808) ([Alexander Nikolaev](https://github.com/AlexNik)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix double destroy call on exception throw in addBatchLookupTable8 [#58745](https://github.com/ClickHouse/ClickHouse/pull/58745) ([Raúl Marín](https://github.com/Algunenano)). +* Fix: LIMIT BY and LIMIT in distributed query [#59153](https://github.com/ClickHouse/ClickHouse/pull/59153) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix translate() with FixedString input [#59356](https://github.com/ClickHouse/ClickHouse/pull/59356) ([Raúl Marín](https://github.com/Algunenano)). +* Fix leftPad / rightPad function with FixedString input [#59739](https://github.com/ClickHouse/ClickHouse/pull/59739) ([Raúl Marín](https://github.com/Algunenano)). +* Fix cosineDistance crash with Nullable [#60150](https://github.com/ClickHouse/ClickHouse/pull/60150) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix possible race in ManyAggregatedData dtor. [#58624](https://github.com/ClickHouse/ClickHouse/pull/58624) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Make ZooKeeper actually sequentialy consistent [#59735](https://github.com/ClickHouse/ClickHouse/pull/59735) ([Alexander Tokmakov](https://github.com/tavplubix)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 53bf705637d..e372e407ce1 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -55,6 +55,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.20.27-lts 2024-03-05 v23.3.19.32-lts 2024-01-05 v23.3.18.15-lts 2023-11-25 v23.3.17.13-lts 2023-11-17 From 24437cdd637dd56401990c839191306b8533a41f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Mar 2024 11:55:04 +0000 Subject: [PATCH 130/197] Polish test --- tests/queries/0_stateless/03000_minmax_index_first.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03000_minmax_index_first.sql b/tests/queries/0_stateless/03000_minmax_index_first.sql index 5d2896c792a..d978e6ea8fa 100644 --- a/tests/queries/0_stateless/03000_minmax_index_first.sql +++ b/tests/queries/0_stateless/03000_minmax_index_first.sql @@ -14,3 +14,5 @@ SETTINGS index_granularity = 8192; INSERT INTO skip_table SELECT number, intDiv(number, 4096) FROM numbers(100000); SELECT trim(explain) FROM ( EXPLAIN indexes = 1 SELECT * FROM skip_table WHERE v = 125) WHERE explain like '%Name%'; + +DROP TABLE skip_table; From 3058e5995046d986635d17b515e1a84da91e53a9 Mon Sep 17 00:00:00 2001 From: Brokenice0415 Date: Tue, 5 Mar 2024 20:06:39 +0800 Subject: [PATCH 131/197] update notices --- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/KeeperServer.cpp | 15 ++++++--------- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 3c8f1d271d0..a32552616ee 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -26,7 +26,7 @@ struct Settings; M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \ M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \ M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Upper bound of election timer (avoid too often leader elections)", 0) \ - M(Milliseconds, leadership_expiry_ms, 0, "How long will a leader expire after not getting enough peer responses. Set it lower or equal to election_timeout_lower_bound_ms to avoid multiple leaders.", 0) \ + M(Milliseconds, leadership_expiry_ms, 0, "Duration after which a leader will expire if it fails to receive responses from peers. Set it lower or equal to election_timeout_lower_bound_ms to avoid multiple leaders.", 0) \ M(UInt64, reserved_log_items, 100000, "How many log items to store (don't remove during compaction)", 0) \ M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \ M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 80d61518e19..57dc9596038 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -319,16 +319,13 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co params.leadership_expiry_ = getValueOrMaxInt32AndLogWarning( coordination_settings->leadership_expiry_ms.totalMilliseconds(), "leadership_expiry_ms", log); - if (params.leadership_expiry_ > 0) + if (params.leadership_expiry_ > 0 && params.leadership_expiry_ <= params.election_timeout_lower_bound_) { - if (params.leadership_expiry_ < params.election_timeout_lower_bound_) - { - LOG_WARNING( - log, - "leadership_expiry_ is smaller than election_timeout_lower_bound_ms. " - "Notice that too small leadership_expiry_ may make Raft group " - "sensitive to network status."); - } + LOG_INFO( + log, + "leadership_expiry_ is smaller than or equal to election_timeout_lower_bound_ms, which can avoid multiple leaders. " + "Notice that too small leadership_expiry_ may make Raft group sensitive to network status. " + ); } params.reserved_log_items_ = getValueOrMaxInt32AndLogWarning(coordination_settings->reserved_log_items, "reserved_log_items", log); From bbe8f5b3c45d96aaf963d972b36115d2a659bf43 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Dec 2023 17:29:47 +0100 Subject: [PATCH 132/197] adjust a test --- ...922_deduplication_with_zero_copy.reference | 2 -- .../02922_deduplication_with_zero_copy.sh | 21 +++++++++++-------- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.reference b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.reference index e394fe46b11..29376f82128 100644 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.reference +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.reference @@ -1,8 +1,6 @@ zero_copy: -all_0_0_0 tables: table -alter_partition_version async_blocks block_numbers blocks columns leader_election log lost_part_count metadata mutations nonincrement_block_numbers part_moves_shard pinned_part_uuids quorum replicas table_shared_id temp zero_copy_hdfs zero_copy_s3 zero_copy: tables: diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh index 77950d923f0..381311c5033 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -3,7 +3,7 @@ set -e -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CURDIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -27,6 +27,7 @@ create table r2 (n int) allow_remote_fs_zero_copy_replication=1; " + function get_shared_locks() { table_shared_id="$1" @@ -42,11 +43,12 @@ function get_shared_locks() done } + function filter_temporary_locks() { while read -r lock do - owner=$($CLICKHOUSE_KEEPER_CLIENT -q "get_stat ${lock}" | grep 'ephemeralOwner' | sed 's/.*= //') + owner="$($CLICKHOUSE_KEEPER_CLIENT -q "get_stat ${lock}" | grep 'ephemeralOwner' | sed 's/.*= //')" if [[ "${owner}" -eq "0" ]] then echo "${lock}" @@ -63,10 +65,11 @@ function insert_duplicates() { wait $CLICKHOUSE_CLIENT -nm -q " +system sync replica r1; system sync replica r2; " - count=$($CLICKHOUSE_CLIENT -q "select count() from r2;") + count="$($CLICKHOUSE_CLIENT -q "select count() from r2;")" [[ "${count}" -eq "1" ]] } @@ -89,8 +92,8 @@ system sync replica r2; " done - persistent_locks="$(get_shared_locks ${table_shared_id} | filter_temporary_locks)" - num=$(echo "${persistent_locks}" | wc -w) + persistent_locks="$(get_shared_locks "${table_shared_id}" | filter_temporary_locks)" + num="$(echo "${persistent_locks}" | wc -w)" if [[ "${num}" -ne "2" ]] then @@ -101,21 +104,23 @@ system sync replica r2; } + export -f query_with_retry export -f filter_temporary_locks export -f insert_duplicates export -f get_shared_locks export -f loop -table_shared_id=$($CLICKHOUSE_KEEPER_CLIENT -q "get /test/02922/${CLICKHOUSE_DATABASE}/table/table_shared_id") +table_shared_id="$($CLICKHOUSE_KEEPER_CLIENT -q "get /test/02922/${CLICKHOUSE_DATABASE}/table/table_shared_id")" exit_code=0 -timeout 40 bash -c loop "${table_shared_id}" || exit_code="${?}" +timeout 40 bash -c "loop '${table_shared_id}'" || exit_code="${?}" if [[ "${exit_code}" -ne "124" ]] then echo "timeout expected, but loop exited with code: ${exit_code}." echo "the error is found if loop ends with 0." + echo "table_shared_id=${table_shared_id}" exit 1 fi @@ -125,11 +130,9 @@ function list_keeper_nodes() { echo "zero_copy:" $CLICKHOUSE_KEEPER_CLIENT -q "ls /clickhouse/zero_copy/zero_copy_s3" | grep -o "${table_shared_id}" | \ sed "s/${table_shared_id}//g" || : - $CLICKHOUSE_KEEPER_CLIENT -q "ls /clickhouse/zero_copy/zero_copy_s3/${table_shared_id}" 2>/dev/null || : echo "tables:" $CLICKHOUSE_KEEPER_CLIENT -q "ls /test/02922/${CLICKHOUSE_DATABASE}" | grep -o "table" || : - $CLICKHOUSE_KEEPER_CLIENT -q "ls /test/02922/${CLICKHOUSE_DATABASE}/table" 2>/dev/null || : } list_keeper_nodes "${table_shared_id}" From 82cc3f4339e856b229c7d9b988f3d4fcad0397be Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 26 Feb 2024 14:52:05 +0000 Subject: [PATCH 133/197] CI: make workflow yml abstract #no_merge_commit #no_ci_cache #job_style_check #job_package_debug #job_ClickHouse_build_check #job_Stateless_tests_debug #job_binary_release --- .github/workflows/pull_request.yml | 855 ++------------------- .github/workflows/reusable_build_stage.yml | 32 + .github/workflows/reusable_test_stage.yml | 25 + tests/ci/ci.py | 28 +- tests/ci/ci_config.py | 227 ++++-- tests/ci/test_ci_config.py | 15 +- 6 files changed, 318 insertions(+), 864 deletions(-) create mode 100644 .github/workflows/reusable_build_stage.yml create mode 100644 .github/workflows/reusable_test_stage.yml diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 8783f959ec6..c065219f980 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -13,9 +13,7 @@ on: # yamllint disable-line rule:truthy - opened branches: - master -########################################################################################## -##################################### SMALL CHECKS ####################################### -########################################################################################## + jobs: RunConfig: runs-on: [self-hosted, style-checker-aarch64] @@ -70,13 +68,13 @@ jobs: python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ runner.temp }}/ci_run_data.json --post --job-name 'Style check' BuildDockers: needs: [RunConfig] - if: ${{ !failure() && !cancelled() }} + if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }} uses: ./.github/workflows/reusable_docker.yml with: data: ${{ needs.RunConfig.outputs.data }} StyleCheck: needs: [RunConfig, BuildDockers] - if: ${{ !failure() && !cancelled() }} + if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Style check')}} uses: ./.github/workflows/reusable_test.yml with: test_name: Style check @@ -89,19 +87,9 @@ jobs: ROBOT_CLICKHOUSE_SSH_KEY< Dict[str, Dict[str, Any]]: + """ + populates GH Actions' workflow with real jobs + "Builds_1": [{"job_name": NAME, "runner_type": RUNER_TYPE}] + "Tests_1": [{"job_name": NAME, "runner_type": RUNER_TYPE}] + ... + """ + result = {} # type: Dict[str, Any] + stages_to_do = [] + for job in jobs_data["jobs_to_do"]: + stage_type = CI_CONFIG.get_job_ci_stage(job) + if stage_type == CIStages.NA: + continue + if stage_type not in result: + result[stage_type] = [] + stages_to_do.append(stage_type) + result[stage_type].append( + {"job_name": job, "runner_type": CI_CONFIG.get_runner_type(job)} + ) + result["stages_to_do"] = stages_to_do + return result + + def _create_gh_status( commit: Any, job: str, batch: int, num_batches: int, job_status: CommitStatusData ) -> None: @@ -1733,6 +1756,7 @@ def main() -> int: result["build"] = build_digest result["docs"] = docs_digest result["ci_flags"] = ci_flags + result["stages_data"] = _generate_ci_stage_config(jobs_data) result["jobs_data"] = jobs_data result["docker_data"] = docker_data ### CONFIGURE action: end diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b9ee5670066..308a9098c29 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 +from copy import deepcopy import logging from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser from dataclasses import dataclass, field @@ -10,6 +11,24 @@ from ci_utils import WithIter from integration_test_images import IMAGES +class CIStages: + NA = "UNKNOWN" + BUILDS_1 = "Builds_1" + BUILDS_2 = "Builds_2" + TESTS_1 = "Tests_1" + TESTS_2 = "Tests_2" + + +class Runners(metaclass=WithIter): + BUILDER = "builder" + STYLE_CHECKER = "style-checker" + STYLE_CHECKER_ARM = "style-checker-aarch64" + FUNC_TESTER = "func-tester" + FUNC_TESTER_ARM = "func-tester-aarch64" + STRESS_TESTER = "stress-tester" + FUZZER_UNIT_TESTER = "fuzzer-unit-tester" + + class Labels(metaclass=WithIter): """ Label names or commit tokens in normalized form @@ -207,6 +226,45 @@ class JobConfig: random_bucket: str = "" +builds_job_config = JobConfig( + required_on_release_branch=True, + digest=DigestConfig( + include_paths=[ + "./src", + "./contrib/*-cmake", + "./contrib/consistent-hashing", + "./contrib/murmurhash", + "./contrib/libfarmhash", + "./contrib/pdqsort", + "./contrib/cityhash102", + "./contrib/sparse-checkout", + "./contrib/libmetrohash", + "./contrib/update-submodules.sh", + "./contrib/CMakeLists.txt", + "./CMakeLists.txt", + "./PreLoad.cmake", + "./cmake", + "./base", + "./programs", + "./packages", + "./docker/packager/packager", + "./rust", + # FIXME: This is a WA to rebuild the CH and recreate the Performance.tar.zst artifact + # when there are changes in performance test scripts. + # Due to the current design of the perf test we need to rebuild CH when the performance test changes, + # otherwise the changes will not be visible in the PerformanceTest job in CI + "./tests/performance", + ], + exclude_files=[".md"], + docker=["clickhouse/binary-builder"], + git_submodules=True, + ), + run_command="build_check.py $BUILD_NAME", +) +fuzzer_build_job_config = deepcopy(builds_job_config) +fuzzer_build_job_config.run_by_label = Labels.libFuzzer + + @dataclass class BuildConfig: name: str @@ -222,43 +280,7 @@ class BuildConfig: sparse_checkout: bool = False comment: str = "" static_binary_name: str = "" - job_config: JobConfig = field( - default_factory=lambda: JobConfig( - required_on_release_branch=True, - digest=DigestConfig( - include_paths=[ - "./src", - "./contrib/*-cmake", - "./contrib/consistent-hashing", - "./contrib/murmurhash", - "./contrib/libfarmhash", - "./contrib/pdqsort", - "./contrib/cityhash102", - "./contrib/sparse-checkout", - "./contrib/libmetrohash", - "./contrib/update-submodules.sh", - "./contrib/CMakeLists.txt", - "./CMakeLists.txt", - "./PreLoad.cmake", - "./cmake", - "./base", - "./programs", - "./packages", - "./docker/packager/packager", - "./rust", - # FIXME: This is a WA to rebuild the CH and recreate the Performance.tar.zst artifact - # when there are changes in performance test scripts. - # Due to the current design of the perf test we need to rebuild CH when the performance test changes, - # otherwise the changes will not be visible in the PerformanceTest job in CI - "./tests/performance", - ], - exclude_files=[".md"], - docker=["clickhouse/binary-builder"], - git_submodules=True, - ), - run_command="build_check.py $BUILD_NAME", - ) - ) + job_config: JobConfig = field(default_factory=lambda: deepcopy(builds_job_config)) def export_env(self, export: bool = False) -> str: def process(field_name: str, field: Union[bool, str]) -> str: @@ -466,6 +488,19 @@ sql_test_params = { "run_command": "sqltest.py", "timeout": 10800, } +clickbench_test_params = { + "digest": DigestConfig( + include_paths=[ + "tests/ci/clickbench.py", + ], + docker=["clickhouse/clickbench"], + ), + "run_command": 'clickbench.py "$CHECK_NAME"', +} +install_test_params = { + "digest": install_check_digest, + "run_command": 'install_check.py "$CHECK_NAME"', +} @dataclass @@ -487,6 +522,37 @@ class CIConfig: return config return None + def get_job_ci_stage(self, job_name: str) -> str: + if job_name in [ + JobNames.STYLE_CHECK, + JobNames.FAST_TEST, + JobNames.JEPSEN_KEEPER, + JobNames.BUILD_CHECK, + JobNames.BUILD_CHECK_SPECIAL, + ]: + # FIXME: we can't currently handle Jepsen in the Stage as it's job has concurrency directive + # BUILD_CHECK and BUILD_CHECK_SPECIAL runs not in stage because we need them even if Builds stage failed + return CIStages.NA + stage_type = None + if self.is_build_job(job_name): + stage_type = CIStages.BUILDS_1 + if job_name in CI_CONFIG.get_builds_for_report( + JobNames.BUILD_CHECK_SPECIAL + ): + # special builds go to Build_2 stage to not delay Builds_1/Test_1 + stage_type = CIStages.BUILDS_2 + elif self.is_docs_job(job_name): + stage_type = CIStages.TESTS_1 + elif job_name == JobNames.BUILD_CHECK_SPECIAL: + stage_type = CIStages.TESTS_2 + elif self.is_test_job(job_name): + stage_type = CIStages.TESTS_1 + if job_name == JobNames.LIBFUZZER_TEST: + # since fuzzers build in Builds_2, test must be in Tests_2 + stage_type = CIStages.TESTS_2 + assert stage_type, f"BUG [{job_name}]" + return stage_type + def get_job_config(self, check_name: str) -> JobConfig: res = None for config in ( @@ -500,6 +566,63 @@ class CIConfig: break return res # type: ignore + def get_runner_type(self, check_name: str) -> str: + result = None + if self.is_build_job(check_name) or check_name == JobNames.FAST_TEST: + result = Runners.BUILDER + elif any( + words in check_name.lower() + for words in [ + "install packages", + "compatibility check", + "docker", + "build check", + "jepsen", + "style check", + ] + ): + result = Runners.STYLE_CHECKER + elif check_name == JobNames.DOCS_CHECK: + # docs job is demanding + result = Runners.FUNC_TESTER_ARM + elif any( + words in check_name.lower() + for words in [ + "stateless", + "stateful", + "clickbench", + "sqllogic test", + "libfuzzer", + "bugfix validation", + ] + ): + result = Runners.FUNC_TESTER + elif any( + words in check_name.lower() + for words in ["stress", "upgrade", "integration", "performance comparison"] + ): + result = Runners.STRESS_TESTER + elif any( + words in check_name.lower() + for words in ["ast fuzzer", "unit tests", "sqlancer", "sqltest"] + ): + result = Runners.FUZZER_UNIT_TESTER + + assert result, f"BUG, no runner for [{check_name}]" + + if ("aarch" in check_name or "arm" in check_name) and "aarch" not in result: + if result == Runners.STRESS_TESTER: + # FIXME: no arm stress tester group atm + result = Runners.FUNC_TESTER_ARM + elif result == Runners.BUILDER: + # crosscompile - no arm required + pass + else: + # switch to aarch64 runnner + result += "-aarch64" + + return result + @staticmethod def normalize_string(input_string: str) -> str: lowercase_string = input_string.lower() @@ -600,11 +723,7 @@ class CIConfig: @classmethod def is_test_job(cls, job: str) -> bool: - return ( - not cls.is_build_job(job) - and not cls.is_build_job(job) - and job != JobNames.STYLE_CHECK - ) + return not cls.is_build_job(job) and job != JobNames.STYLE_CHECK @classmethod def is_docs_job(cls, job: str) -> bool: @@ -845,7 +964,7 @@ CI_CONFIG = CIConfig( name=Build.FUZZERS, compiler="clang-17", package_type="fuzzers", - job_config=JobConfig(run_by_label=Labels.libFuzzer), + job_config=fuzzer_build_job_config, ), }, builds_report_config={ @@ -889,6 +1008,7 @@ CI_CONFIG = CIConfig( include_paths=["**/*.md", "./docs", "tests/ci/docs_check.py"], docker=["clickhouse/docs-builder"], ), + run_command="docs_check.py", ), ), JobNames.FAST_TEST: TestConfig( @@ -918,10 +1038,10 @@ CI_CONFIG = CIConfig( }, test_configs={ JobNames.INSTALL_TEST_AMD: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(digest=install_check_digest) + Build.PACKAGE_RELEASE, job_config=JobConfig(**install_test_params) # type: ignore ), JobNames.INSTALL_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(digest=install_check_digest) + Build.PACKAGE_AARCH64, job_config=JobConfig(**install_test_params) # type: ignore ), JobNames.STATEFUL_TEST_ASAN: TestConfig( Build.PACKAGE_ASAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore @@ -1139,9 +1259,20 @@ CI_CONFIG = CIConfig( JobNames.SQLTEST: TestConfig( Build.PACKAGE_RELEASE, job_config=JobConfig(**sql_test_params) # type: ignore ), - JobNames.CLCIKBENCH_TEST: TestConfig(Build.PACKAGE_RELEASE), - JobNames.CLCIKBENCH_TEST_ARM: TestConfig(Build.PACKAGE_AARCH64), - JobNames.LIBFUZZER_TEST: TestConfig(Build.FUZZERS, job_config=JobConfig(run_by_label=Labels.libFuzzer)), # type: ignore + JobNames.CLCIKBENCH_TEST: TestConfig( + Build.PACKAGE_RELEASE, job_config=JobConfig(**clickbench_test_params) # type: ignore + ), + JobNames.CLCIKBENCH_TEST_ARM: TestConfig( + Build.PACKAGE_AARCH64, job_config=JobConfig(**clickbench_test_params) # type: ignore + ), + JobNames.LIBFUZZER_TEST: TestConfig( + Build.FUZZERS, + job_config=JobConfig( + run_by_label=Labels.libFuzzer, + timeout=10800, + run_command='libfuzzer_test_check.py "$CHECK_NAME" 10800', + ), + ), # type: ignore }, ) CI_CONFIG.validate() diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 49d49d9c328..04c90105276 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -1,15 +1,12 @@ #!/usr/bin/env python3 import unittest +from ci_config import JobNames, CI_CONFIG, Runners class TestCIConfig(unittest.TestCase): - def test_no_errors_in_ci_config(self): - raised = None - try: - from ci_config import ( # pylint: disable=import-outside-toplevel - CI_CONFIG as _, - ) - except Exception as exc: - raised = exc - self.assertIsNone(raised, f"CI_CONFIG import raised error {raised}") + def test_runner_config(self): + """check runner is provided w/o exception""" + for job in JobNames: + runner = CI_CONFIG.get_runner_type(job) + self.assertIn(runner, Runners) From 6e085c713f03d722c0881241bbd3bf81802d3298 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Feb 2024 19:01:04 +0100 Subject: [PATCH 134/197] generate key for clickhouse_remove_objects_capability --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 8 ++++---- src/Disks/ObjectStorages/S3/DiskS3Utils.cpp | 13 +++++++------ src/Disks/ObjectStorages/S3/DiskS3Utils.h | 2 +- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index f4df579de73..898c7eb98fa 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -96,10 +96,10 @@ S3::URI getS3URI(const Poco::Util::AbstractConfiguration & config, const std::st } void checkS3Capabilities( - S3ObjectStorage & storage, const S3Capabilities s3_capabilities, const String & name, const String & key_with_trailing_slash) + S3ObjectStorage & storage, const S3Capabilities s3_capabilities, const String & name) { /// If `support_batch_delete` is turned on (default), check and possibly switch it off. - if (s3_capabilities.support_batch_delete && !checkBatchRemove(storage, key_with_trailing_slash)) + if (s3_capabilities.support_batch_delete && !checkBatchRemove(storage)) { LOG_WARNING( getLogger("S3ObjectStorage"), @@ -134,7 +134,7 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) - checkS3Capabilities(*object_storage, s3_capabilities, name, uri.key); + checkS3Capabilities(*object_storage, s3_capabilities, name); return object_storage; }); @@ -170,7 +170,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) - checkS3Capabilities(*object_storage, s3_capabilities, name, uri.key); + checkS3Capabilities(*object_storage, s3_capabilities, name); return object_storage; }); diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp index bb7b53b2d22..529df59ca17 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp @@ -79,13 +79,14 @@ static String getServerUUID() return toString(server_uuid); } -bool checkBatchRemove(S3ObjectStorage & storage, const String & key_with_trailing_slash) +bool checkBatchRemove(S3ObjectStorage & storage) { - /// NOTE: key_with_trailing_slash is the disk prefix, it is required - /// because access is done via S3ObjectStorage not via IDisk interface - /// (since we don't have disk yet). - const String path = fmt::format("{}clickhouse_remove_objects_capability_{}", key_with_trailing_slash, getServerUUID()); - StoredObject object(path); + /// NOTE: Here we are going to write and later drop some key. + /// We are using generateObjectKeyForPath() which returns random object key. + /// That generated key is placed in a right directory where we should have write access. + const String path = fmt::format("clickhouse_remove_objects_capability_{}", getServerUUID()); + const auto key = storage.generateObjectKeyForPath(path); + StoredObject object(key.serialize(), path); try { auto file = storage.writeObject(object, WriteMode::Rewrite); diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.h b/src/Disks/ObjectStorages/S3/DiskS3Utils.h index 29e39d4bc1b..ddd204d449d 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.h +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.h @@ -18,7 +18,7 @@ ObjectStorageKeysGeneratorPtr getKeyGenerator( const String & config_prefix); class S3ObjectStorage; -bool checkBatchRemove(S3ObjectStorage & storage, const std::string & key_with_trailing_slash); +bool checkBatchRemove(S3ObjectStorage & storage); } From 248c93ed2dd46ae73cc086a2d14ce62be1f61d94 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 5 Mar 2024 14:37:16 +0100 Subject: [PATCH 135/197] Change error code --- src/Interpreters/executeQuery.cpp | 3 ++- tests/queries/0_stateless/03003_analyzer_setting.sql | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 18f0cd6601f..f318f363eda 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -104,6 +104,7 @@ namespace ErrorCodes extern const int QUERY_WAS_CANCELLED; extern const int INCORRECT_DATA; extern const int SYNTAX_ERROR; + extern const int INCORRECT_QUERY; } namespace FailPoints @@ -669,7 +670,7 @@ void validateAnalyzerSettings(ASTPtr ast, bool context_value) if (auto * value = set_query->changes.tryGet("allow_experimental_analyzer")) { if (top_level != value->safeGet()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting 'allow_experimental_analyzer' is changed in the subquery. Top level value: {}", top_level); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'allow_experimental_analyzer' is changed in the subquery. Top level value: {}", top_level); } } diff --git a/tests/queries/0_stateless/03003_analyzer_setting.sql b/tests/queries/0_stateless/03003_analyzer_setting.sql index 43e1bd0d955..2e5cab71277 100644 --- a/tests/queries/0_stateless/03003_analyzer_setting.sql +++ b/tests/queries/0_stateless/03003_analyzer_setting.sql @@ -2,8 +2,8 @@ CREATE TABLE test (dummy Int8) ENGINE = Distributed(test_cluster_two_shards, 'sy SET allow_experimental_analyzer = 0; -SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 1); -- { serverError LOGICAL_ERROR} +SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 1); -- { serverError INCORRECT_QUERY } SET allow_experimental_analyzer = 1; -SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 0); -- { serverError LOGICAL_ERROR} +SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 0); -- { serverError INCORRECT_QUERY } From b4e90e512115071bc0edd9d25a9b9dcac00e9214 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Tue, 5 Mar 2024 08:49:33 +0100 Subject: [PATCH 136/197] Address PR comments --- src/Interpreters/InterpreterInsertQuery.cpp | 8 ++++-- ...001_insert_threads_deduplication.reference | 2 ++ .../03001_insert_threads_deduplication.sql | 27 +++++++++++++++++++ 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index df833803970..5680857ed3d 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -515,9 +515,13 @@ BlockIO InterpreterInsertQuery::execute() : std::min(settings.max_insert_threads, pipeline.getNumStreams()); /// Deduplication when passing insert_deduplication_token breaks if using more than one thread - const String & deduplication_token = settings.insert_deduplication_token; - if (!deduplication_token.empty()) + if (!settings.insert_deduplication_token.toString().empty()) + { + LOG_DEBUG( + getLogger("InsertQuery"), + "Insert-select query using insert_deduplication_token, setting streams to 1 to avoid deduplication issues"); pre_streams_size = 1; + } if (table->supportsParallelInsert()) sink_streams_size = pre_streams_size; diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.reference b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference index 2559e5c49e7..b6d6006f84c 100644 --- a/tests/queries/0_stateless/03001_insert_threads_deduplication.reference +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.reference @@ -1,2 +1,4 @@ 3 6 +12 +18 diff --git a/tests/queries/0_stateless/03001_insert_threads_deduplication.sql b/tests/queries/0_stateless/03001_insert_threads_deduplication.sql index 5b5cb1d6845..093d2b3185d 100644 --- a/tests/queries/0_stateless/03001_insert_threads_deduplication.sql +++ b/tests/queries/0_stateless/03001_insert_threads_deduplication.sql @@ -1,4 +1,7 @@ +-- Tags: distributed + DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS landing_dist SYNC; DROP TABLE IF EXISTS ds SYNC; CREATE TABLE landing @@ -10,6 +13,14 @@ CREATE TABLE landing ENGINE = MergeTree() ORDER BY timestamp; +CREATE TABLE landing_dist +( + timestamp DateTime64(3), + status String, + id String +) +ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), 'landing', rand()); + SYSTEM STOP MERGES landing; -- Stopping merges to force 3 parts INSERT INTO landing (status, id, timestamp) SELECT * FROM generateRandom() LIMIT 1; @@ -38,5 +49,21 @@ SETTINGS insert_deduplicate=1, insert_deduplication_token='token2', SELECT count() FROM ds; +-- When reading from distributed table, 6 rows are going to be retrieved +-- due to the being using the two shards cluster + +INSERT INTO ds SELECT * FROM landing_dist +SETTINGS insert_deduplicate=1, insert_deduplication_token='token3', + max_insert_threads=5; + +SELECT count() FROM ds; + +INSERT INTO ds SELECT * FROM landing_dist +SETTINGS insert_deduplicate=1, insert_deduplication_token='token4', + max_insert_threads=1; + +SELECT count() FROM ds; + DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS landing_dist SYNC; DROP TABLE IF EXISTS ds SYNC; From 0bedd98bb7eb4e4c560ee92de1d1d958ec95897e Mon Sep 17 00:00:00 2001 From: Joris Clement <7713214+joclement@users.noreply.github.com> Date: Tue, 5 Mar 2024 14:49:12 +0100 Subject: [PATCH 137/197] style: trailing whitespace --- docs/en/operations/settings/settings-formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index f7d9586dd5b..0cac71df983 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -467,7 +467,7 @@ Enabled by default. Allow to use String type for JSON keys that contain only `Null`/`{}`/`[]` in data sample during schema inference. In JSON formats any value can be read as String, and we can avoid errors like `Cannot determine type for column 'column_name' by first 25000 rows of data, most likely this column contains only Nulls or empty Arrays/Maps` during schema inference -by using String type for keys with unknown types. +by using String type for keys with unknown types. Example: @@ -1605,7 +1605,7 @@ possible values: - `1` — Enabled. Pretty formats will use ANSI escape sequences except for `NoEscapes` formats. - `auto` - Enabled if `stdout` is a terminal except for `NoEscapes` formats. -Default value is `auto`. +Default value is `auto`. ### output_format_pretty_grid_charset {#output_format_pretty_grid_charset} From b791ced86f3c7a739e63a26a8ecbf52ed376db1d Mon Sep 17 00:00:00 2001 From: Joris Clement <7713214+joclement@users.noreply.github.com> Date: Tue, 5 Mar 2024 14:49:25 +0100 Subject: [PATCH 138/197] docs: typo With https://github.com/ClickHouse/ClickHouse/pull/39423 allowing the the usage of single quotes was disabled by default. Adapt the documentation accordingly. --- docs/en/operations/settings/settings-formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0cac71df983..477fbf94625 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -891,7 +891,7 @@ Default value: `,`. If it is set to true, allow strings in single quotes. -Enabled by default. +Disabled by default. ### format_csv_allow_double_quotes {#format_csv_allow_double_quotes} From 97f955fcde7b1e8c0fe935d5233cd791780b78a0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Mar 2024 14:10:46 +0000 Subject: [PATCH 139/197] Fix sanitizers --- .../System/StorageSystemDetachedParts.cpp | 29 +++++++++++++++---- .../System/StorageSystemPartsBase.cpp | 24 +++++++++++++++ src/Storages/System/StorageSystemPartsBase.h | 26 ----------------- 3 files changed, 48 insertions(+), 31 deletions(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 913983952c3..fa74093a5a5 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -287,7 +287,7 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i setInMemoryMetadata(storage_metadata); } -class ReadFromSystemDetachedParts : public ReadFromSystemPartsBase +class ReadFromSystemDetachedParts : public SourceStepWithFilter { public: ReadFromSystemDetachedParts( @@ -296,23 +296,42 @@ public: const StorageSnapshotPtr & storage_snapshot_, const ContextPtr & context_, Block sample_block, - std::shared_ptr storage_, + std::shared_ptr storage_, std::vector columns_mask_, size_t max_block_size_, size_t num_streams_) - : ReadFromSystemPartsBase(column_names_, query_info_, storage_snapshot_, context_, sample_block, std::move(storage_), std::move(columns_mask_), false) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , columns_mask(std::move(columns_mask_)) , max_block_size(max_block_size_) , num_streams(num_streams_) {} std::string getName() const override { return "ReadFromSystemDetachedParts"; } void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters(ActionDAGNodes added_filter_nodes) override; -private: +protected: + std::shared_ptr storage; + std::vector columns_mask; + + const ActionsDAG::Node * predicate = nullptr; const size_t max_block_size; const size_t num_streams; }; +void ReadFromSystemDetachedParts::applyFilters(ActionDAGNodes added_filter_nodes) +{ + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); +} + void StorageSystemDetachedParts::read( QueryPlan & query_plan, const Names & column_names, @@ -328,7 +347,7 @@ void StorageSystemDetachedParts::read( auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); - auto this_ptr = std::static_pointer_cast(shared_from_this()); + auto this_ptr = std::static_pointer_cast(shared_from_this()); auto reading = std::make_unique( column_names, query_info, storage_snapshot, diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 5921fbc8f0f..f31dd5a955e 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -203,6 +203,30 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAG::Node * predicate, Conte storage_uuid_column = block_to_filter.getByName("uuid").column; } +class ReadFromSystemPartsBase : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemPartsBase"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemPartsBase( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + bool has_state_column_); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +protected: + std::shared_ptr storage; + std::vector columns_mask; + const bool has_state_column; + const ActionsDAG::Node * predicate = nullptr; +}; ReadFromSystemPartsBase::ReadFromSystemPartsBase( const Names & column_names_, diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 2b4055373d1..b5d4a74e58b 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -156,30 +156,4 @@ protected: processNextStorage(ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) = 0; }; -class ReadFromSystemPartsBase : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromSystemPartsBase"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - ReadFromSystemPartsBase( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::shared_ptr storage_, - std::vector columns_mask_, - bool has_state_column_); - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - -protected: - std::shared_ptr storage; - std::vector columns_mask; - const bool has_state_column; - const ActionsDAG::Node * predicate = nullptr; -}; - - } From 56e0dd0ab0c6adc3e1e2e7a15ba74a96594c0762 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 5 Mar 2024 15:30:42 +0100 Subject: [PATCH 140/197] More fixes --- src/Parsers/ASTQueryWithOutput.cpp | 23 +++++++++++++++++------ src/Parsers/ParserQueryWithOutput.cpp | 2 ++ 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 3890ed2347a..e47f4dcf29d 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -74,12 +74,23 @@ bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) /// FIXME: try to prettify this cast using `as<>()` if (auto * ast_with_output = dynamic_cast(&ast)) { - ast_with_output->out_file.reset(); - ast_with_output->format.reset(); - ast_with_output->settings_ast.reset(); - ast_with_output->compression.reset(); - ast_with_output->compression_level.reset(); - ast_with_output->children.clear(); + auto remove_if_exists = [&](ASTPtr & p) + { + if (p) + { + if (auto it = std::find(ast_with_output->children.begin(), ast_with_output->children.end(), p); + it != ast_with_output->children.end()) + ast_with_output->children.erase(it); + p.reset(); + } + }; + + remove_if_exists(ast_with_output->out_file); + remove_if_exists(ast_with_output->format); + remove_if_exists(ast_with_output->settings_ast); + remove_if_exists(ast_with_output->compression); + remove_if_exists(ast_with_output->compression_level); + return true; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 7a627ae5f6a..4a0b928608b 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -136,6 +136,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserStringLiteral compression; if (!compression.parse(pos, query_with_output.compression, expected)) return false; + query_with_output.children.push_back(query_with_output.compression); ParserKeyword s_compression_level("LEVEL"); if (s_compression_level.ignore(pos, expected)) @@ -143,6 +144,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserNumber compression_level; if (!compression_level.parse(pos, query_with_output.compression_level, expected)) return false; + query_with_output.children.push_back(query_with_output.compression_level); } } From 67be42fa678df061ebe83510d42c859853cd4d62 Mon Sep 17 00:00:00 2001 From: Pradeep Chhetri Date: Wed, 24 May 2023 15:02:22 +0800 Subject: [PATCH 141/197] Enable pylint for all tests --- utils/check-style/check-style | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 3a966daea41..620aec2fda2 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -152,8 +152,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.xml' | grep -vP $EXCLUDE_DIRS | xargs xmllint --noout --nonet -# FIXME: for now only clickhouse-test -pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test $ROOT_PATH/tests/ci/*.py +pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test $ROOT_PATH/tests/**/*.py find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*.yml' \) -type f | grep -vP $EXCLUDE_DIRS | From 17f1e442413eed9c4f0e7484bcaa1295b660e2b0 Mon Sep 17 00:00:00 2001 From: Pradeep Chhetri Date: Mon, 29 May 2023 22:34:31 +0800 Subject: [PATCH 142/197] Fix linter issues in ci-runner.py --- tests/integration/ci-runner.py | 49 +++++++++++++++++++++------------- 1 file changed, 30 insertions(+), 19 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 08dd9ba276b..acafa16b837 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -11,6 +11,7 @@ import shlex import shutil import string import subprocess +import sys import time import zlib # for crc32 from collections import defaultdict @@ -196,7 +197,7 @@ def clear_ip_tables_and_restart_daemons(): shell=True, ) except subprocess.CalledProcessError as err: - logging.info("docker kill excepted: " + str(err)) + logging.info("docker kill excepted: %s", str(err)) try: logging.info("Removing all docker containers") @@ -205,7 +206,7 @@ def clear_ip_tables_and_restart_daemons(): shell=True, ) except subprocess.CalledProcessError as err: - logging.info("docker rm excepted: " + str(err)) + logging.info("docker rm excepted: %s", str(err)) # don't restart docker if it's disabled if os.environ.get("CLICKHOUSE_TESTS_RUNNER_RESTART_DOCKER", "1") == "1": @@ -213,7 +214,7 @@ def clear_ip_tables_and_restart_daemons(): logging.info("Stopping docker daemon") subprocess.check_output("service docker stop", shell=True) except subprocess.CalledProcessError as err: - logging.info("docker stop excepted: " + str(err)) + logging.info("docker stop excepted: %s", str(err)) try: for i in range(200): @@ -228,7 +229,7 @@ def clear_ip_tables_and_restart_daemons(): else: raise Exception("Docker daemon doesn't responding") except subprocess.CalledProcessError as err: - logging.info("Can't reload docker: " + str(err)) + logging.info("Can't reload docker: %s", str(err)) iptables_iter = 0 try: @@ -276,13 +277,14 @@ class ClickhouseIntegrationTestsRunner: def base_path(self): return os.path.join(str(self.result_path), "../") - def should_skip_tests(self): + @staticmethod + def should_skip_tests(): return [] def get_image_with_version(self, name): if name in self.image_versions: return name + ":" + self.image_versions[name] - logging.warn( + logging.warning( "Cannot find image %s in params list %s", name, self.image_versions ) if ":" not in name: @@ -292,7 +294,7 @@ class ClickhouseIntegrationTestsRunner: def get_image_version(self, name: str): if name in self.image_versions: return self.image_versions[name] - logging.warn( + logging.warning( "Cannot find image %s in params list %s", name, self.image_versions ) return "latest" @@ -322,13 +324,14 @@ class ClickhouseIntegrationTestsRunner: ) return except subprocess.CalledProcessError as err: - logging.info("docker-compose pull failed: " + str(err)) + logging.info("docker-compose pull failed: %s", str(err)) continue logging.error("Pulling images failed for 5 attempts. Will fail the worker.") # We pass specific retcode to to ci/integration_test_check.py to skip status reporting and restart job - exit(13) + sys.exit(13) - def _can_run_with(self, path, opt): + @staticmethod + def _can_run_with(path, opt): with open(path, "r") as script: for line in script: if opt in line: @@ -358,7 +361,9 @@ class ClickhouseIntegrationTestsRunner: if retcode == 0: logging.info("Installation of %s successfull", full_path) else: - raise Exception("Installation of %s failed", full_path) + raise Exception( + "Installation of {} failed".format(full_path) + ) break else: raise Exception("Package with {} not found".format(package)) @@ -387,10 +392,11 @@ class ClickhouseIntegrationTestsRunner: os.getenv("CLICKHOUSE_TESTS_LIBRARY_BRIDGE_BIN_PATH"), ) - def _compress_logs(self, dir, relpaths, result_path): + @staticmethod + def _compress_logs(directory, relpaths, result_path): retcode = subprocess.call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL "tar --use-compress-program='zstd --threads=0' -cf {} -C {} {}".format( - result_path, dir, " ".join(relpaths) + result_path, directory, " ".join(relpaths) ), shell=True, ) @@ -443,7 +449,8 @@ class ClickhouseIntegrationTestsRunner: return list(sorted(all_tests)) - def _get_parallel_tests_skip_list(self, repo_path): + @staticmethod + def _get_parallel_tests_skip_list(repo_path): skip_list_file_path = "{}/tests/integration/parallel_skip.json".format( repo_path ) @@ -462,7 +469,8 @@ class ClickhouseIntegrationTestsRunner: skip_list_tests = json.load(skip_list_file) return list(sorted(skip_list_tests)) - def group_test_by_file(self, tests): + @staticmethod + def group_test_by_file(tests): result = {} for test in tests: test_file = test.split("::")[0] @@ -471,7 +479,8 @@ class ClickhouseIntegrationTestsRunner: result[test_file].append(test) return result - def _update_counters(self, main_counters, current_counters, broken_tests): + @staticmethod + def _update_counters(main_counters, current_counters, broken_tests): for test in current_counters["PASSED"]: if test not in main_counters["PASSED"]: if test in main_counters["FAILED"]: @@ -525,7 +534,8 @@ class ClickhouseIntegrationTestsRunner: logging.info("Cannot run with custom docker image version :(") return image_cmd - def _find_test_data_dirs(self, repo_path, test_names): + @staticmethod + def _find_test_data_dirs(repo_path, test_names): relpaths = {} for test_name in test_names: if "/" in test_name: @@ -543,7 +553,8 @@ class ClickhouseIntegrationTestsRunner: relpaths[relpath] = mtime return relpaths - def _get_test_data_dirs_difference(self, new_snapshot, old_snapshot): + @staticmethod + def _get_test_data_dirs_difference(new_snapshot, old_snapshot): res = set() for path in new_snapshot: if (path not in old_snapshot) or (old_snapshot[path] != new_snapshot[path]): @@ -569,7 +580,7 @@ class ClickhouseIntegrationTestsRunner: broken_tests, ) except Exception as e: - logging.info("Failed to run {}:\n{}".format(str(test_group), str(e))) + logging.info("Failed to run %s:\n%s", str(test_group), str(e)) counters = { "ERROR": [], "PASSED": [], From 774fcdfff65ee404ff9af374365f847e73f9bde9 Mon Sep 17 00:00:00 2001 From: Pradeep Chhetri Date: Tue, 30 May 2023 16:16:38 +0800 Subject: [PATCH 143/197] wip --- tests/integration/conftest.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 6e76270c607..3e03a49aaec 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -1,7 +1,9 @@ -from helpers.cluster import run_and_check +#!/usr/bin/env python3 + import pytest import logging import os +from helpers.cluster import run_and_check from helpers.test_tools import TSV from helpers.network import _NetworkManager From 76cbd783319cc7d924c4afbf4839f8b05585f087 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Feb 2024 13:54:08 +0100 Subject: [PATCH 144/197] Update pylint, mypy, and black in style check image --- docker/test/style/Dockerfile | 2 +- pyproject.toml | 39 ++++++++++++++++++++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 pyproject.toml diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index abc2dba0e9d..6153aec2144 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -18,7 +18,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ yamllint \ locales \ - && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==23.12.0 boto3 codespell==2.2.1 mypy==1.8.0 PyGithub unidiff pylint==3.1.0 \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ && rm -rf /root/.cache/pip diff --git a/pyproject.toml b/pyproject.toml new file mode 100644 index 00000000000..277b24d4fd7 --- /dev/null +++ b/pyproject.toml @@ -0,0 +1,39 @@ +[tool.pylint.BASIC] +max-module-lines=2000 +# due to SQL +max-line-length=200 +# Drop/decrease them one day: +max-branches=50 +max-nested-blocks=10 +max-statements=200 + +[tool.pylint.FORMAT] +#ignore-long-lines = (# )??$ + +[tool.pylint.'MESSAGES CONTROL'] + # pytest.mark.parametrize is not callable (not-callable) +disable = '''missing-docstring, + too-few-public-methods, + invalid-name, + too-many-arguments, + keyword-arg-before-vararg, + too-many-locals, + too-many-instance-attributes, + cell-var-from-loop, + fixme, + too-many-public-methods, + wildcard-import, + unused-wildcard-import, + singleton-comparison, + not-callable, + redefined-outer-name, + broad-except, + bare-except, + no-else-return, + global-statement + ''' + +[tool.pylint.SIMILARITIES] +# due to SQL +min-similarity-lines=1000 + From e31078e204d135a6594c6bb356ec4b447a4353c4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Feb 2024 18:46:15 +0100 Subject: [PATCH 145/197] Fix some issues detected by new pylint --- tests/ci/ast_fuzzer_check.py | 4 +- tests/ci/bugfix_validate_check.py | 38 +++++----- tests/ci/build_download_helper.py | 9 +-- tests/ci/cherry_pick.py | 12 ++-- tests/ci/ci.py | 17 +++-- tests/ci/ci_config.py | 23 +++--- tests/ci/clickhouse_helper.py | 49 ++++++------- tests/ci/functional_test_check.py | 8 +-- tests/ci/libfuzzer_test_check.py | 15 ++-- tests/ci/stress_check.py | 2 +- tests/ci/style_check.py | 2 +- tests/ci/tee_popen.py | 13 ++-- tests/clickhouse-test | 112 ++++++++++++++++-------------- 13 files changed, 153 insertions(+), 151 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 6e3da7fa816..1ecf805cadc 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -67,14 +67,14 @@ def main(): build_name = get_build_name_for_check(check_name) urls = read_build_urls(build_name, reports_path) if not urls: - raise Exception("No build URLs found") + raise ValueError("No build URLs found") for url in urls: if url.endswith("/clickhouse"): build_url = url break else: - raise Exception("Cannot find the clickhouse binary among build results") + raise ValueError("Cannot find the clickhouse binary among build results") logging.info("Got build url %s", build_url) diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index ae7fce1f102..7aaf18e7765 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -1,37 +1,37 @@ #!/usr/bin/env python3 -from pathlib import Path -import subprocess -import sys -from typing import List, Sequence, Tuple import csv import logging +import subprocess +import sys +from pathlib import Path +from typing import List, Sequence, Tuple -from report import ( - ERROR, - FAILURE, - SKIPPED, - SUCCESS, - FAIL, - OK, - TestResult, - TestResults, - JobReport, -) -from env_helper import TEMP_PATH -from stopwatch import Stopwatch from ci_config import JobNames from ci_utils import normalize_string +from env_helper import TEMP_PATH from functional_test_check import NO_CHANGES_MSG +from report import ( + ERROR, + FAIL, + FAILURE, + OK, + SKIPPED, + SUCCESS, + JobReport, + TestResult, + TestResults, +) +from stopwatch import Stopwatch def post_commit_status_from_file(file_path: Path) -> List[str]: with open(file_path, "r", encoding="utf-8") as f: res = list(csv.reader(f, delimiter="\t")) if len(res) < 1: - raise Exception(f'Can\'t read from "{file_path}"') + raise IndexError(f'Can\'t read from "{file_path}"') if len(res[0]) != 3: - raise Exception(f'Can\'t read from "{file_path}"') + raise IndexError(f'Can\'t read from "{file_path}"') return res[0] diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 0d24cb80021..7ad39a5fff1 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -8,9 +8,8 @@ import time from pathlib import Path from typing import Any, Callable, List, Union -import requests # type: ignore - import get_robot_token as grt # we need an updated ROBOT_TOKEN +import requests # type: ignore from ci_config import CI_CONFIG DOWNLOAD_RETRIES_COUNT = 5 @@ -30,9 +29,10 @@ def get_with_retries( "Getting URL with %i tries and sleep %i in between: %s", retries, sleep, url ) exc = Exception("A placeholder to satisfy typing and avoid nesting") + timeout = kwargs.pop("timeout", 30) for i in range(retries): try: - response = requests.get(url, **kwargs) + response = requests.get(url, timeout=timeout, **kwargs) response.raise_for_status() return response except Exception as e: @@ -74,10 +74,11 @@ def get_gh_api( token_is_set = "Authorization" in kwargs.get("headers", {}) exc = Exception("A placeholder to satisfy typing and avoid nesting") try_cnt = 0 + timeout = kwargs.pop("timeout", 30) while try_cnt < retries: try_cnt += 1 try: - response = requests.get(url, **kwargs) + response = requests.get(url, timeout=timeout, **kwargs) response.raise_for_status() return response except requests.HTTPError as e: diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 2c40b2a4099..d92504e30bd 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -456,11 +456,13 @@ class Backport: tomorrow = date.today() + timedelta(days=1) logging.info("Receive PRs suppose to be backported") - query_args = dict( - query=f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", - label=",".join(self.labels_to_backport + [self.must_create_backport_label]), - merged=[since_date, tomorrow], - ) + query_args = { + "query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", + "label": ",".join( + self.labels_to_backport + [self.must_create_backport_label] + ), + "merged": [since_date, tomorrow], + } logging.info("Query to find the backport PRs:\n %s", query_args) self.prs_for_backport = self.gh.get_pulls_from_search(**query_args) logging.info( diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 234eec48463..d6e07d4ff6e 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -397,7 +397,7 @@ class CiCache: status.dump_to_file(record_file) elif record_type == self.RecordType.PENDING: assert isinstance(status, PendingState) - with open(record_file, "w") as json_file: + with open(record_file, "w", encoding="utf-8") as json_file: json.dump(asdict(status), json_file) else: assert False @@ -1005,7 +1005,7 @@ def _mark_success_action( def _print_results(result: Any, outfile: Optional[str], pretty: bool = False) -> None: if outfile: - with open(outfile, "w") as f: + with open(outfile, "w", encoding="utf-8") as f: if isinstance(result, str): print(result, file=f) elif isinstance(result, dict): @@ -1125,8 +1125,7 @@ def _configure_jobs( jobs_to_wait: Dict[str, Dict[str, Any]] = {} randomization_buckets = {} # type: Dict[str, Set[str]] - for job in digests: - digest = digests[job] + for job, digest in digests.items(): job_config = CI_CONFIG.get_job_config(job) num_batches: int = job_config.num_batches batches_to_do: List[int] = [] @@ -1612,11 +1611,11 @@ def main() -> int: indata: Optional[Dict[str, Any]] = None if args.infile: - indata = ( - json.loads(args.infile) - if not os.path.isfile(args.infile) - else json.load(open(args.infile)) - ) + if os.path.isfile(args.infile): + with open(args.infile, encoding="utf-8") as jfd: + indata = json.load(jfd) + else: + indata = json.loads(args.infile) assert indata and isinstance(indata, dict), "Invalid --infile json" result: Dict[str, Any] = {} diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b9ee5670066..f7e056ad066 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -570,8 +570,7 @@ class CIConfig: self.builds_report_config, self.test_configs, ): - for check_name in config: # type: ignore - yield check_name + yield from config # type: ignore def get_builds_for_report( self, report_name: str, release: bool = False, backport: bool = False @@ -698,17 +697,15 @@ CI_CONFIG = CIConfig( job for job in JobNames if not any( - [ - nogo in job - for nogo in ( - "asan", - "tsan", - "msan", - "ubsan", - # skip build report jobs as not all builds will be done - "build check", - ) - ] + nogo in job + for nogo in ( + "asan", + "tsan", + "msan", + "ubsan", + # skip build report jobs as not all builds will be done + "build check", + ) ) ] ), diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index f338a1e14c3..3ab41673dec 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -1,13 +1,12 @@ #!/usr/bin/env python3 -from pathlib import Path -from typing import Dict, List, Optional import fileinput import json import logging import time +from pathlib import Path +from typing import Dict, List, Optional import requests # type: ignore - from get_robot_token import get_parameter_from_ssm from pr_info import PRInfo from report import TestResults @@ -72,11 +71,11 @@ class ClickHouseHelper: if args: url = args[0] url = kwargs.get("url", url) - kwargs["timeout"] = kwargs.get("timeout", 100) + timeout = kwargs.pop("timeout", 100) for i in range(5): try: - response = requests.post(*args, **kwargs) + response = requests.post(*args, timeout=timeout, **kwargs) except Exception as e: error = f"Received exception while sending data to {url} on {i} attempt: {e}" logging.warning(error) @@ -148,7 +147,9 @@ class ClickHouseHelper: for i in range(5): response = None try: - response = requests.get(self.url, params=params, headers=self.auth) + response = requests.get( + self.url, params=params, headers=self.auth, timeout=100 + ) response.raise_for_status() return response.text except Exception as ex: @@ -215,24 +216,24 @@ def prepare_tests_results_for_clickhouse( head_ref = pr_info.head_ref head_repo = pr_info.head_name - common_properties = dict( - pull_request_number=pr_info.number, - commit_sha=pr_info.sha, - commit_url=pr_info.commit_html_url, - check_name=check_name, - check_status=check_status, - check_duration_ms=int(float(check_duration) * 1000), - check_start_time=check_start_time, - report_url=report_url, - pull_request_url=pull_request_url, - base_ref=base_ref, - base_repo=base_repo, - head_ref=head_ref, - head_repo=head_repo, - task_url=pr_info.task_url, - instance_type=get_instance_type(), - instance_id=get_instance_id(), - ) + common_properties = { + "pull_request_number": pr_info.number, + "commit_sha": pr_info.sha, + "commit_url": pr_info.commit_html_url, + "check_name": check_name, + "check_status": check_status, + "check_duration_ms": int(float(check_duration) * 1000), + "check_start_time": check_start_time, + "report_url": report_url, + "pull_request_url": pull_request_url, + "base_ref": base_ref, + "base_repo": base_repo, + "head_ref": head_ref, + "head_repo": head_repo, + "task_url": pr_info.task_url, + "instance_type": get_instance_type(), + "instance_id": get_instance_id(), + } # Always publish a total record for all checks. For checks with individual # tests, also publish a record per test. diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 1e5807a96c0..4416b7863a6 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -12,10 +12,9 @@ from typing import List, Tuple from build_download_helper import download_all_deb_packages from clickhouse_helper import CiLogsCredentials - -from docker_images_helper import DockerImage, pull_image, get_docker_image +from docker_images_helper import DockerImage, get_docker_image, pull_image from download_release_packages import download_last_release -from env_helper import REPORT_PATH, TEMP_PATH, REPO_COPY +from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH from pr_info import PRInfo from report import ERROR, SUCCESS, JobReport, StatusType, TestResults, read_test_results from stopwatch import Stopwatch @@ -54,8 +53,7 @@ def get_image_name(check_name: str) -> str: return "clickhouse/stateless-test" if "stateful" in check_name.lower(): return "clickhouse/stateful-test" - else: - raise Exception(f"Cannot deduce image name based on check name {check_name}") + raise ValueError(f"Cannot deduce image name based on check name {check_name}") def get_run_command( diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 465b9a6b3a7..62dadc6fea9 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -9,19 +9,13 @@ from pathlib import Path from typing import List from build_download_helper import download_fuzzers -from clickhouse_helper import ( - CiLogsCredentials, -) -from docker_images_helper import DockerImage, pull_image, get_docker_image - -from env_helper import REPORT_PATH, TEMP_PATH, REPO_COPY +from clickhouse_helper import CiLogsCredentials +from docker_images_helper import DockerImage, get_docker_image, pull_image +from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH from pr_info import PRInfo - from stopwatch import Stopwatch - from tee_popen import TeePopen - NO_CHANGES_MSG = "Nothing to run" @@ -130,7 +124,8 @@ def main(): os.chmod(fuzzers_path / file, 0o777) elif file.endswith("_seed_corpus.zip"): corpus_path = fuzzers_path / (file.removesuffix("_seed_corpus.zip") + ".in") - zipfile.ZipFile(fuzzers_path / file, "r").extractall(corpus_path) + with zipfile.ZipFile(fuzzers_path / file, "r") as zfd: + zfd.extractall(corpus_path) result_path = temp_path / "result_path" result_path.mkdir(parents=True, exist_ok=True) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 49c1515c69f..0f2c0cdd222 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -95,7 +95,7 @@ def process_results( results_path = result_directory / "test_results.tsv" test_results = read_test_results(results_path, True) if len(test_results) == 0: - raise Exception("Empty results") + raise ValueError("Empty results") except Exception as e: return ( ERROR, diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 0c7160aeea4..d004f3ed215 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -43,7 +43,7 @@ def process_result( results_path = result_directory / "test_results.tsv" test_results = read_test_results(results_path) if len(test_results) == 0: - raise Exception("Empty results") + raise ValueError("Empty results") return state, description, test_results, additional_files except Exception: diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index a50532aea20..13db50df53f 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -1,14 +1,14 @@ #!/usr/bin/env python3 -from io import TextIOWrapper -from pathlib import Path -from subprocess import Popen, PIPE, STDOUT -from threading import Thread -from time import sleep -from typing import Optional, Union import logging import os import sys +from io import TextIOWrapper +from pathlib import Path +from subprocess import PIPE, STDOUT, Popen +from threading import Thread +from time import sleep +from typing import Optional, Union # Very simple tee logic implementation. You can specify a shell command, output @@ -98,5 +98,6 @@ class TeePopen: @property def log_file(self) -> TextIOWrapper: if self._log_file is None: + # pylint:disable-next=consider-using-with self._log_file = open(self._log_file_name, "w", encoding="utf-8") return self._log_file diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5afda75aa5c..ce0feadf050 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -5,48 +5,46 @@ # pylint: disable=too-many-lines # pylint: disable=anomalous-backslash-in-string -import enum -from queue import Full -import shutil -import itertools -import sys -import os -import os.path -import glob -import platform -import signal -import re import copy -import traceback -import math +import enum +import glob # Not requests, to avoid requiring extra dependency. import http.client -import urllib.parse +import itertools import json +import math +import multiprocessing +import os +import os.path +import platform +import random +import re +import shutil +import signal +import socket +import string +import subprocess +import sys +import traceback +import urllib.parse # for crc32 import zlib - from argparse import ArgumentParser -from typing import Tuple, Union, Optional, Dict, Set, List -import subprocess -from subprocess import Popen -from subprocess import PIPE +from contextlib import closing from datetime import datetime, timedelta -from time import time, sleep from errno import ESRCH +from queue import Full +from subprocess import PIPE, Popen +from time import sleep, time +from typing import Dict, List, Optional, Set, Tuple, Union try: import termcolor # type: ignore except ImportError: termcolor = None -import random -import string -import multiprocessing -import socket -from contextlib import closing USE_JINJA = True try: @@ -70,7 +68,7 @@ TEST_FILE_EXTENSIONS = [".sql", ".sql.j2", ".sh", ".py", ".expect"] VERSION_PATTERN = r"^((\d+\.)?(\d+\.)?(\d+\.)?\d+)$" -def stringhash(s): +def stringhash(s: str) -> int: # default hash() function consistent # only during process invocation https://stackoverflow.com/a/42089311 return zlib.crc32(s.encode("utf-8")) @@ -94,6 +92,10 @@ def trim_for_log(s): return "\n".join(lines) +class TestException(Exception): + pass + + class HTTPError(Exception): def __init__(self, message=None, code=None): self.message = message @@ -250,7 +252,7 @@ def get_db_engine(args, database_name): def get_create_database_settings(args, testcase_args): - create_database_settings = dict() + create_database_settings = {} if testcase_args: create_database_settings["log_comment"] = testcase_args.testcase_basename if args.db_engine == "Ordinary": @@ -1186,7 +1188,7 @@ class TestCase: ) if result_is_different: - diff_proc = Popen( + with Popen( [ "diff", "-U", @@ -1197,23 +1199,23 @@ class TestCase: encoding="latin-1", stdout=PIPE, universal_newlines=True, - ) - if self.show_whitespaces_in_diff: - sed_proc = Popen( - ["sed", "-e", "s/[ \t]\\+$/&$/g"], - stdin=diff_proc.stdout, - stdout=PIPE, - ) - diff_proc.stdout.close() # Allow diff to receive a SIGPIPE if cat exits. - diff = sed_proc.communicate()[0].decode("utf-8", errors="ignore") - else: - diff = diff_proc.communicate()[0] + ) as diff_proc: + if self.show_whitespaces_in_diff: + with Popen( + ["sed", "-e", "s/[ \t]\\+$/&$/g"], + stdin=diff_proc.stdout, + stdout=PIPE, + ) as sed_proc: + diff = sed_proc.communicate()[0].decode( + "utf-8", errors="ignore" + ) + else: + diff = diff_proc.communicate()[0] if diff.startswith("Binary files "): diff += "Content of stdout:\n===================\n" - file = open(self.stdout_file, "rb") - diff += str(file.read()) - file.close() + with open(self.stdout_file, "rb") as file: + diff += str(file.read()) diff += "===================" description += f"\n{diff}\n" if debug_log: @@ -1376,6 +1378,7 @@ class TestCase: command = pattern.format(**params) + # pylint:disable-next=consider-using-with; TODO: fix proc = Popen(command, shell=True, env=os.environ, start_new_session=True) while ( @@ -1542,7 +1545,7 @@ class TestCase: ) if len(leftover_tables) != 0: - raise Exception( + raise TestException( f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally." ) @@ -1625,7 +1628,7 @@ class TestSuite: ): return "#" else: - raise Exception(f"Unknown file_extension: {filename}") + raise TestException(f"Unknown file_extension: {filename}") def parse_tags_from_line(line, comment_sign) -> Set[str]: if not line.startswith(comment_sign): @@ -1686,17 +1689,22 @@ class TestSuite: self.suite_tmp_path: str = suite_tmp_path self.suite: str = suite - filter_func = lambda x: True # noqa: ignore E731 - if args.run_by_hash_num is not None and args.run_by_hash_total is not None: if args.run_by_hash_num > args.run_by_hash_total: - raise Exception( + raise TestException( f"Incorrect run by hash, value {args.run_by_hash_num} bigger than total {args.run_by_hash_total}" ) - filter_func = ( - lambda x: stringhash(x) % args.run_by_hash_total == args.run_by_hash_num - ) + def filter_func(x: str) -> bool: + return bool( + stringhash(x) % args.run_by_hash_total == args.run_by_hash_num + ) + + else: + + def filter_func(x: str) -> bool: + _ = x + return True self.all_tests: List[str] = self.get_tests_list( self.tests_in_suite_key_func, filter_func @@ -2418,7 +2426,7 @@ def main(args): pid = get_server_pid() print("Got server pid", pid) print_stacktraces() - raise Exception(msg) + raise TestException(msg) args.build_flags = collect_build_flags(args) args.changed_merge_tree_settings = collect_changed_merge_tree_settings(args) @@ -2622,7 +2630,7 @@ def find_binary(name): if os.access(bin_path, os.X_OK): return bin_path - raise Exception(f"{name} was not found in PATH") + raise TestException(f"{name} was not found in PATH") def find_clickhouse_command(binary, command): From 91cff01ce0bc483334d58d8b40b9fe83bdda319a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Feb 2024 19:25:02 +0100 Subject: [PATCH 146/197] More fixes in tests/ci --- tests/ci/build_download_helper.py | 6 +++++- tests/ci/commit_status_helper.py | 6 +++--- tests/ci/compatibility_check.py | 6 +++--- tests/ci/docker_images_check.py | 27 ++++++++++++++----------- tests/ci/docker_manifests_merge.py | 26 +++++++++++++----------- tests/ci/docker_server.py | 4 ++-- tests/ci/get_previous_release_tag.py | 9 +++++---- tests/ci/jepsen_check.py | 12 +++++------ tests/ci/release.py | 18 ++++++++--------- tests/ci/report.py | 8 ++++---- tests/ci/sqlancer_check.py | 4 ++-- tests/ci/sqltest.py | 4 ++-- tests/ci/ssh.py | 16 +++++++-------- tests/ci/stress.py | 30 +++++++++++++++++----------- tests/ci/unit_tests_check.py | 2 +- 15 files changed, 97 insertions(+), 81 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 7ad39a5fff1..c7a461e003b 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -8,8 +8,12 @@ import time from pathlib import Path from typing import Any, Callable, List, Union -import get_robot_token as grt # we need an updated ROBOT_TOKEN +# isort: off import requests # type: ignore + +# isort: on + +import get_robot_token as grt # we need an updated ROBOT_TOKEN from ci_config import CI_CONFIG DOWNLOAD_RETRIES_COUNT = 5 diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 9d2742ba5ba..1c2d8b2ade8 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -303,7 +303,7 @@ def post_commit_status_to_file( file_path: Path, description: str, state: str, report_url: str ) -> None: if file_path.exists(): - raise Exception(f'File "{file_path}" already exists!') + raise FileExistsError(f'File "{file_path}" already exists!') with open(file_path, "w", encoding="utf-8") as f: out = csv.writer(f, delimiter="\t") out.writerow([state, report_url, description]) @@ -329,7 +329,7 @@ class CommitStatusData: @classmethod def load_from_file(cls, file_path: Union[Path, str]): # type: ignore res = {} - with open(file_path, "r") as json_file: + with open(file_path, "r", encoding="utf-8") as json_file: res = json.load(json_file) return CommitStatusData(**cls._filter_dict(res)) @@ -347,7 +347,7 @@ class CommitStatusData: def dump_to_file(self, file_path: Union[Path, str]) -> None: file_path = Path(file_path) or STATUS_FILE_PATH - with open(file_path, "w") as json_file: + with open(file_path, "w", encoding="utf-8") as json_file: json.dump(asdict(self), json_file) def is_ok(self): diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 8009ef24760..5e980660749 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -26,7 +26,7 @@ DOWNLOAD_RETRIES_COUNT = 5 def process_os_check(log_path: Path) -> TestResult: name = log_path.name - with open(log_path, "r") as log: + with open(log_path, "r", encoding="utf-8") as log: line = log.read().split("\n")[0].strip() if line != "OK": return TestResult(name, "FAIL") @@ -35,7 +35,7 @@ def process_os_check(log_path: Path) -> TestResult: def process_glibc_check(log_path: Path, max_glibc_version: str) -> TestResults: test_results = [] # type: TestResults - with open(log_path, "r") as log: + with open(log_path, "r", encoding="utf-8") as log: for line in log: if line.strip(): columns = line.strip().split(" ") @@ -204,7 +204,7 @@ def main(): elif "aarch64" in check_name: max_glibc_version = "2.18" # because of build with newer sysroot? else: - raise Exception("Can't determine max glibc version") + raise RuntimeError("Can't determine max glibc version") state, description, test_results, additional_logs = process_result( result_path, diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index ad497a00eba..b04a3975545 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -195,18 +195,21 @@ def main(): ok_cnt = 0 status = SUCCESS # type: StatusType - image_tags = ( - json.loads(args.image_tags) - if not os.path.isfile(args.image_tags) - else json.load(open(args.image_tags)) - ) - missing_images = ( - image_tags - if args.missing_images == "all" - else json.loads(args.missing_images) - if not os.path.isfile(args.missing_images) - else json.load(open(args.missing_images)) - ) + + if os.path.isfile(args.image_tags): + with open(args.image_tags, "r", encoding="utf-8") as jfd: + image_tags = json.load(jfd) + else: + image_tags = json.loads(args.image_tags) + + if args.missing_images == "all": + missing_images = image_tags + elif os.path.isfile(args.missing_images): + with open(args.missing_images, "r", encoding="utf-8") as jfd: + missing_images = json.load(jfd) + else: + missing_images = json.loads(args.missing_images) + images_build_list = get_images_oredered_list() for image in images_build_list: diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index fc00969d5d6..3c122545735 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -135,18 +135,20 @@ def main(): archs = args.suffixes assert len(archs) > 1, "arch suffix input param is invalid" - image_tags = ( - json.loads(args.image_tags) - if not os.path.isfile(args.image_tags) - else json.load(open(args.image_tags)) - ) - missing_images = ( - list(image_tags) - if args.missing_images == "all" - else json.loads(args.missing_images) - if not os.path.isfile(args.missing_images) - else json.load(open(args.missing_images)) - ) + if os.path.isfile(args.image_tags): + with open(args.image_tags, "r", encoding="utf-8") as jfd: + image_tags = json.load(jfd) + else: + image_tags = json.loads(args.image_tags) + + if args.missing_images == "all": + missing_images = image_tags + elif os.path.isfile(args.missing_images): + with open(args.missing_images, "r", encoding="utf-8") as jfd: + missing_images = json.load(jfd) + else: + missing_images = json.loads(args.missing_images) + test_results = [] status = SUCCESS # type: StatusType diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 38d0ea6d86b..fda0dd16d0b 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -363,8 +363,8 @@ def main(): image = DockerImageData(image_path, image_repo, False) args.release_type = auto_release_type(args.version, args.release_type) tags = gen_tags(args.version, args.release_type) - repo_urls = dict() - direct_urls: Dict[str, List[str]] = dict() + repo_urls = {} + direct_urls: Dict[str, List[str]] = {} release_or_pr, _ = get_release_or_pr(pr_info, args.version) for arch, build_name in zip(ARCH, ("package_release", "package_aarch64")): diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c2d279f7fec..b74432425d3 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 -import re import logging +import re from typing import List, Optional, Tuple import requests # type: ignore @@ -82,13 +82,14 @@ def get_previous_release(server_version: Optional[Version]) -> Optional[ReleaseI CLICKHOUSE_TAGS_URL, {"page": page, "per_page": 100}, timeout=10 ) if not response.ok: - raise Exception( - "Cannot load the list of tags from github: " + response.reason + logger.error( + "Cannot load the list of tags from github: %s", response.reason ) + response.raise_for_status() releases_str = set(re.findall(VERSION_PATTERN, response.text)) if len(releases_str) == 0: - raise Exception( + raise ValueError( "Cannot find previous release for " + str(server_version) + " server version" diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index fb7540abda3..01263c3085d 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -46,7 +46,7 @@ FAILED_TESTS_ANCHOR = "# Failed tests" def _parse_jepsen_output(path: Path) -> TestResults: test_results = [] # type: TestResults current_type = "" - with open(path, "r") as f: + with open(path, "r", encoding="utf-8") as f: for line in f: if SUCCESSFUL_TESTS_ANCHOR in line: current_type = "OK" @@ -101,7 +101,7 @@ def prepare_autoscaling_group_and_get_hostnames(count): instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) counter += 1 if counter > 30: - raise Exception("Cannot wait autoscaling group") + raise RuntimeError("Cannot wait autoscaling group") ec2_client = boto3.client("ec2", region_name="us-east-1") return get_instances_addresses(ec2_client, instances) @@ -119,12 +119,12 @@ def clear_autoscaling_group(): instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) counter += 1 if counter > 30: - raise Exception("Cannot wait autoscaling group") + raise RuntimeError("Cannot wait autoscaling group") def save_nodes_to_file(instances: List[Any], temp_path: Path) -> Path: nodes_path = temp_path / "nodes.txt" - with open(nodes_path, "w") as f: + with open(nodes_path, "w", encoding="utf-8") as f: f.write("\n".join(instances)) f.flush() return nodes_path @@ -159,7 +159,7 @@ def main(): ) args = parser.parse_args() - if args.program != "server" and args.program != "keeper": + if args.program not in ("server", "keeper"): logging.warning("Invalid argument '%s'", args.program) sys.exit(0) @@ -220,7 +220,7 @@ def main(): f"{S3_URL}/{S3_BUILDS_BUCKET}/{version}/{sha}/binary_release/clickhouse" ) print(f"Clickhouse version: [{version_full}], sha: [{sha}], url: [{build_url}]") - head = requests.head(build_url) + head = requests.head(build_url, timeout=60) assert head.status_code == 200, f"Clickhouse binary not found: {build_url}" else: build_name = get_build_name_for_check(check_name) diff --git a/tests/ci/release.py b/tests/ci/release.py index 2b3331938e7..679e65560f5 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -54,7 +54,7 @@ class Repo: elif protocol == "origin": self._url = protocol else: - raise Exception(f"protocol must be in {self.VALID}") + raise ValueError(f"protocol must be in {self.VALID}") def __str__(self): return self._repo @@ -144,7 +144,7 @@ class Release: for status in statuses: if status["context"] == RELEASE_READY_STATUS: if not status["state"] == SUCCESS: - raise Exception( + raise ValueError( f"the status {RELEASE_READY_STATUS} is {status['state']}" ", not success" ) @@ -153,7 +153,7 @@ class Release: page += 1 - raise Exception( + raise KeyError( f"the status {RELEASE_READY_STATUS} " f"is not found for commit {self.release_commit}" ) @@ -188,7 +188,7 @@ class Release: raise if check_run_from_master and self._git.branch != "master": - raise Exception("the script must be launched only from master") + raise RuntimeError("the script must be launched only from master") self.set_release_info() @@ -229,7 +229,7 @@ class Release: def check_no_tags_after(self): tags_after_commit = self.run(f"git tag --contains={self.release_commit}") if tags_after_commit: - raise Exception( + raise RuntimeError( f"Commit {self.release_commit} belongs to following tags:\n" f"{tags_after_commit}\nChoose another commit" ) @@ -253,7 +253,7 @@ class Release: ) output = self.run(f"git branch --contains={self.release_commit} {branch}") if branch not in output: - raise Exception( + raise RuntimeError( f"commit {self.release_commit} must belong to {branch} " f"for {self.release_type} release" ) @@ -464,9 +464,9 @@ class Release: logging.warning("Rolling back checked out %s for %s", ref, orig_ref) self.run(f"git reset --hard; git checkout -f {orig_ref}") raise - else: - if with_checkout_back and need_rollback: - self.run(rollback_cmd) + # Normal flow when we need to checkout back + if with_checkout_back and need_rollback: + self.run(rollback_cmd) @contextmanager def _create_branch(self, name: str, start_point: str = "") -> Iterator[None]: diff --git a/tests/ci/report.py b/tests/ci/report.py index 282c343eec3..669409d1729 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -22,8 +22,8 @@ from typing import ( from build_download_helper import get_gh_api from ci_config import CI_CONFIG, BuildConfig -from env_helper import REPORT_PATH, TEMP_PATH from ci_utils import normalize_string +from env_helper import REPORT_PATH, TEMP_PATH logger = logging.getLogger(__name__) @@ -296,7 +296,7 @@ class JobReport: def load(cls, from_file=None): # type: ignore res = {} from_file = from_file or JOB_REPORT_FILE - with open(from_file, "r") as json_file: + with open(from_file, "r", encoding="utf-8") as json_file: res = json.load(json_file) # Deserialize the nested lists of TestResult test_results_data = res.get("test_results", []) @@ -316,7 +316,7 @@ class JobReport: raise TypeError("Type not serializable") to_file = to_file or JOB_REPORT_FILE - with open(to_file, "w") as json_file: + with open(to_file, "w", encoding="utf-8") as json_file: json.dump(asdict(self), json_file, default=path_converter, indent=2) @@ -418,7 +418,7 @@ class BuildResult: def load_from_file(cls, file: Union[Path, str]): # type: ignore if not Path(file).exists(): return None - with open(file, "r") as json_file: + with open(file, "r", encoding="utf-8") as json_file: res = json.load(json_file) return BuildResult(**res) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 59d2a3d6275..9d33c480598 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -46,14 +46,14 @@ def main(): build_name = get_build_name_for_check(check_name) urls = read_build_urls(build_name, reports_path) if not urls: - raise Exception("No build URLs found") + raise ValueError("No build URLs found") for url in urls: if url.endswith("/clickhouse"): build_url = url break else: - raise Exception("Cannot find binary clickhouse among build results") + raise ValueError("Cannot find the clickhouse binary among build results") logging.info("Got build url %s", build_url) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 2fe6aabd69c..c8c2adbbd56 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -53,14 +53,14 @@ def main(): print(build_name) urls = read_build_urls(build_name, reports_path) if not urls: - raise Exception("No build URLs found") + raise ValueError("No build URLs found") for url in urls: if url.endswith("/clickhouse"): build_url = url break else: - raise Exception("Cannot find the clickhouse binary among build results") + raise ValueError("Cannot find the clickhouse binary among build results") logging.info("Got build url %s", build_url) diff --git a/tests/ci/ssh.py b/tests/ci/ssh.py index 275f26fd65f..321826fcf44 100644 --- a/tests/ci/ssh.py +++ b/tests/ci/ssh.py @@ -1,11 +1,11 @@ #!/usr/bin/env python3 -import shutil +import logging import os +import shutil +import signal import subprocess import tempfile -import logging -import signal class SSHAgent: @@ -21,7 +21,7 @@ class SSHAgent: def start(self): if shutil.which("ssh-agent") is None: - raise Exception("ssh-agent binary is not available") + raise RuntimeError("ssh-agent binary is not available") self._env_backup["SSH_AUTH_SOCK"] = os.environ.get("SSH_AUTH_SOCK") self._env_backup["SSH_OPTIONS"] = os.environ.get("SSH_OPTIONS") @@ -54,7 +54,7 @@ class SSHAgent: def remove(self, key_pub): if key_pub not in self._keys: - raise Exception(f"Private key not found, public part: {key_pub}") + raise ValueError(f"Private key not found, public part: {key_pub}") if self._keys[key_pub] > 1: self._keys[key_pub] -= 1 @@ -107,7 +107,7 @@ class SSHAgent: if p.returncode: message = stderr.strip() + b"\n" + stdout.strip() - raise Exception(message.strip().decode()) + raise RuntimeError(message.strip().decode()) return stdout @@ -115,9 +115,9 @@ class SSHAgent: class SSHKey: def __init__(self, key_name=None, key_value=None): if key_name is None and key_value is None: - raise Exception("Either key_name or key_value must be specified") + raise ValueError("Either key_name or key_value must be specified") if key_name is not None and key_value is not None: - raise Exception("key_name or key_value must be specified") + raise ValueError("key_name or key_value must be specified") if key_name is not None: self.key = os.getenv(key_name) else: diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 7d582e683e0..7ccc058f79f 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -1,14 +1,14 @@ #!/usr/bin/env python3 # -*- coding: utf-8 -*- """This script is used in docker images for stress tests and upgrade tests""" -from multiprocessing import cpu_count -from pathlib import Path -from subprocess import Popen, call, check_output, STDOUT, PIPE -from typing import List import argparse import logging import random import time +from multiprocessing import cpu_count +from pathlib import Path +from subprocess import PIPE, STDOUT, Popen, call, check_output +from typing import List def get_options(i: int, upgrade_check: bool) -> str: @@ -90,12 +90,13 @@ def run_func_test( ] pipes = [] for i, path in enumerate(output_paths): - with open(path, "w") as op: + with open(path, "w", encoding="utf-8") as op: full_command = ( f"{cmd} {get_options(i, upgrade_check)} {global_time_limit_option} " f"{skip_tests_option} {upgrade_check_option}" ) logging.info("Run func tests '%s'", full_command) + # pylint:disable-next=consider-using-with pipes.append(Popen(full_command, shell=True, stdout=op, stderr=op)) time.sleep(0.5) return pipes @@ -204,6 +205,7 @@ def prepare_for_hung_check(drop_databases: bool) -> bool: continue command = make_query_command(f"DETACH DATABASE {db}") # we don't wait for drop + # pylint:disable-next=consider-using-with Popen(command, shell=True) break except Exception as ex: @@ -212,7 +214,7 @@ def prepare_for_hung_check(drop_databases: bool) -> bool: ) time.sleep(i) else: - raise Exception( + raise RuntimeError( "Cannot drop databases after stress tests. Probably server consumed " "too much memory and cannot execute simple queries" ) @@ -293,7 +295,9 @@ def main(): args = parse_args() if args.drop_databases and not args.hung_check: - raise Exception("--drop-databases only used in hung check (--hung-check)") + raise argparse.ArgumentTypeError( + "--drop-databases only used in hung check (--hung-check)" + ) # FIXME Hung check with ubsan is temporarily disabled due to # https://github.com/ClickHouse/ClickHouse/issues/45372 @@ -359,15 +363,17 @@ def main(): ] ) hung_check_log = args.output_folder / "hung_check.log" # type: Path - tee = Popen(["/usr/bin/tee", hung_check_log], stdin=PIPE) - res = call(cmd, shell=True, stdout=tee.stdin, stderr=STDOUT, timeout=600) - if tee.stdin is not None: - tee.stdin.close() + with Popen(["/usr/bin/tee", hung_check_log], stdin=PIPE) as tee: + res = call(cmd, shell=True, stdout=tee.stdin, stderr=STDOUT, timeout=600) + if tee.stdin is not None: + tee.stdin.close() if res != 0 and have_long_running_queries and not suppress_hung_check: logging.info("Hung check failed with exit code %d", res) else: hung_check_status = "No queries hung\tOK\t\\N\t\n" - with open(args.output_folder / "test_results.tsv", "w+") as results: + with open( + args.output_folder / "test_results.tsv", "w+", encoding="utf-8" + ) as results: results.write(hung_check_status) hung_check_log.unlink() diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index f64f114d3de..2c2862d926a 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -23,7 +23,7 @@ def get_test_name(line): for element in elements: if "(" not in element and ")" not in element: return element - raise Exception(f"No test name in line '{line}'") + raise ValueError(f"No test name in line '{line}'") def process_results( From 842ca051a9a096f64bc289a4a2364867df401a43 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Feb 2024 21:24:45 +0100 Subject: [PATCH 147/197] Fix more linter issues in ci-runner.py --- tests/integration/ci-runner.py | 134 ++++++++++++++++----------------- 1 file changed, 67 insertions(+), 67 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index acafa16b837..e7f691d2237 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -15,6 +15,7 @@ import sys import time import zlib # for crc32 from collections import defaultdict +from itertools import chain from integration_test_images import IMAGES @@ -103,7 +104,7 @@ def get_counters(fname): "SKIPPED": set([]), } - with open(fname, "r") as out: + with open(fname, "r", encoding="utf-8") as out: for line in out: line = line.strip() # Example of log: @@ -119,7 +120,7 @@ def get_counters(fname): # [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client # ^^^^^^^^^^^^^ if line.strip().startswith("["): - line = re.sub("^\[[^\[\]]*\] \[[^\[\]]*\] ", "", line) + line = re.sub(r"^\[[^\[\]]*\] \[[^\[\]]*\] ", "", line) line_arr = line.split(" ") if len(line_arr) < 2: @@ -161,7 +162,7 @@ def get_counters(fname): def parse_test_times(fname): read = False description_output = [] - with open(fname, "r") as out: + with open(fname, "r", encoding="utf-8") as out: for line in out: if read and "==" in line: break @@ -227,7 +228,7 @@ def clear_ip_tables_and_restart_daemons(): time.sleep(0.5) logging.info("Waiting docker to start, current %s", str(err)) else: - raise Exception("Docker daemon doesn't responding") + raise RuntimeError("Docker daemon doesn't responding") except subprocess.CalledProcessError as err: logging.info("Can't reload docker: %s", str(err)) @@ -306,13 +307,9 @@ class ClickhouseIntegrationTestsRunner: image_cmd = self._get_runner_image_cmd(repo_path) cmd = ( - "cd {repo_path}/tests/integration && " - "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} --pre-pull --command '{command}' ".format( - repo_path=repo_path, - runner_opts=self._get_runner_opts(), - image_cmd=image_cmd, - command=r""" echo Pre Pull finished """, - ) + f"cd {repo_path}/tests/integration && " + f"timeout --signal=KILL 1h ./runner {self._get_runner_opts()} {image_cmd} " + "--pre-pull --command ' echo Pre Pull finished ' " ) for i in range(5): @@ -332,7 +329,7 @@ class ClickhouseIntegrationTestsRunner: @staticmethod def _can_run_with(path, opt): - with open(path, "r") as script: + with open(path, "r", encoding="utf-8") as script: for line in script: if opt in line: return True @@ -352,21 +349,23 @@ class ClickhouseIntegrationTestsRunner: logging.info("Package found in %s", full_path) log_name = "install_" + f + ".log" log_path = os.path.join(str(self.path()), log_name) - with open(log_path, "w") as log: - cmd = "dpkg -x {} .".format(full_path) + with open(log_path, "w", encoding="utf-8") as log: + cmd = f"dpkg -x {full_path} ." logging.info("Executing installation cmd %s", cmd) - retcode = subprocess.Popen( + with subprocess.Popen( cmd, shell=True, stderr=log, stdout=log - ).wait() - if retcode == 0: - logging.info("Installation of %s successfull", full_path) - else: - raise Exception( - "Installation of {} failed".format(full_path) - ) + ) as proc: + if proc.wait() == 0: + logging.info( + "Installation of %s successfull", full_path + ) + else: + raise RuntimeError( + f"Installation of {full_path} failed" + ) break else: - raise Exception("Package with {} not found".format(package)) + raise FileNotFoundError(f"Package with {package} not found") # logging.info("Unstripping binary") # logging.info( # "Unstring %s", @@ -395,9 +394,8 @@ class ClickhouseIntegrationTestsRunner: @staticmethod def _compress_logs(directory, relpaths, result_path): retcode = subprocess.call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL - "tar --use-compress-program='zstd --threads=0' -cf {} -C {} {}".format( - result_path, directory, " ".join(relpaths) - ), + f"tar --use-compress-program='zstd --threads=0' -cf {result_path} -C " + f"{directory} {' '.join(relpaths)}", shell=True, ) # tar return 1 when the files are changed on compressing, we ignore it @@ -451,21 +449,18 @@ class ClickhouseIntegrationTestsRunner: @staticmethod def _get_parallel_tests_skip_list(repo_path): - skip_list_file_path = "{}/tests/integration/parallel_skip.json".format( - repo_path - ) + skip_list_file_path = f"{repo_path}/tests/integration/parallel_skip.json" if ( not os.path.isfile(skip_list_file_path) or os.path.getsize(skip_list_file_path) == 0 ): - raise Exception( - "There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format( - skip_list_file_path - ) + raise ValueError( + "There is something wrong with getting all tests list: " + f"file '{skip_list_file_path}' is empty or does not exist." ) skip_list_tests = [] - with open(skip_list_file_path, "r") as skip_list_file: + with open(skip_list_file_path, "r", encoding="utf-8") as skip_list_file: skip_list_tests = json.load(skip_list_file) return list(sorted(skip_list_tests)) @@ -520,14 +515,15 @@ class ClickhouseIntegrationTestsRunner: logging.info( "Can run with custom docker image version %s", runner_version ) - image_cmd += " --docker-image-version={} ".format(runner_version) + image_cmd += f" --docker-image-version={runner_version} " else: if self._can_run_with( os.path.join(repo_path, "tests/integration", "runner"), "--docker-compose-images-tags", ): - image_cmd += "--docker-compose-images-tags={} ".format( - self.get_image_with_version(img) + image_cmd += ( + "--docker-compose-images-tags=" + f"{self.get_image_with_version(img)} " ) else: image_cmd = "" @@ -580,7 +576,7 @@ class ClickhouseIntegrationTestsRunner: broken_tests, ) except Exception as e: - logging.info("Failed to run %s:\n%s", str(test_group), str(e)) + logging.info("Failed to run %s:\n%s", test_group, e) counters = { "ERROR": [], "PASSED": [], @@ -641,31 +637,27 @@ class ClickhouseIntegrationTestsRunner: info_path = os.path.join(repo_path, "tests/integration", info_basename) test_cmd = " ".join([shlex.quote(test) for test in sorted(test_names)]) - parallel_cmd = ( - " --parallel {} ".format(num_workers) if num_workers > 0 else "" - ) + parallel_cmd = f" --parallel {num_workers} " if num_workers > 0 else "" # -r -- show extra test summary: # -f -- (f)ailed # -E -- (E)rror # -p -- (p)assed # -s -- (s)kipped - cmd = "cd {}/tests/integration && timeout --signal=KILL 1h ./runner {} {} -t {} {} -- -rfEps --run-id={} --color=no --durations=0 {} | tee {}".format( - repo_path, - self._get_runner_opts(), - image_cmd, - test_cmd, - parallel_cmd, - i, - _get_deselect_option(self.should_skip_tests()), - info_path, + cmd = ( + f"cd {repo_path}/tests/integration && " + f"timeout --signal=KILL 1h ./runner {self._get_runner_opts()} " + f"{image_cmd} -t {test_cmd} {parallel_cmd} -- -rfEps --run-id={i} " + f"--color=no --durations=0 {_get_deselect_option(self.should_skip_tests())} " + f"| tee {info_path}" ) log_basename = test_group_str + "_" + str(i) + ".log" log_path = os.path.join(repo_path, "tests/integration", log_basename) - with open(log_path, "w") as log: + with open(log_path, "w", encoding="utf-8") as log: logging.info("Executing cmd: %s", cmd) # ignore retcode, since it meaningful due to pipe to tee - subprocess.Popen(cmd, shell=True, stderr=log, stdout=log).wait() + with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as proc: + proc.wait() extra_logs_names = [log_basename] log_result_path = os.path.join( @@ -756,11 +748,14 @@ class ClickhouseIntegrationTestsRunner: # want to mark them as error so we filter by '::'. for test in tests_in_group: if ( - test not in counters["PASSED"] - and test not in counters["ERROR"] - and test not in counters["SKIPPED"] - and test not in counters["FAILED"] - and test not in counters["BROKEN"] + test + not in chain( + counters["PASSED"], + counters["ERROR"], + counters["SKIPPED"], + counters["FAILED"], + counters["BROKEN"], + ) and "::" in test ): counters["ERROR"].append(test) @@ -825,7 +820,7 @@ class ClickhouseIntegrationTestsRunner: ( c + " (✕" + str(final_retry) + ")", text_state, - "{:.2f}".format(tests_times[c]), + f"{tests_times[c]:.2f}", ) for c in counters[state] ] @@ -847,7 +842,7 @@ class ClickhouseIntegrationTestsRunner: self._install_clickhouse(build_path) logging.info("Pulling images") - runner._pre_pull_images(repo_path) + self._pre_pull_images(repo_path) logging.info( "Dump iptables before run %s", @@ -920,11 +915,15 @@ class ClickhouseIntegrationTestsRunner: logging.info("Shuffling test groups") random.shuffle(items_to_run) - broken_tests = list() + broken_tests = [] if self.use_analyzer: - with open(f"{repo_path}/tests/analyzer_integration_broken_tests.txt") as f: + with open( + f"{repo_path}/tests/analyzer_integration_broken_tests.txt", + "r", + encoding="utf-8", + ) as f: broken_tests = f.read().splitlines() - logging.info(f"Broken tests in the list: {len(broken_tests)}") + logging.info("Broken tests in the list: %s", len(broken_tests)) for group, tests in items_to_run: logging.info("Running test group %s containing %s tests", group, len(tests)) @@ -976,12 +975,12 @@ class ClickhouseIntegrationTestsRunner: else: text_state = state test_result += [ - (c, text_state, "{:.2f}".format(tests_times[c]), tests_log_paths[c]) + (c, text_state, f"{tests_times[c]:.2f}", tests_log_paths[c]) for c in counters[state] ] failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) - status_text = "fail: {}, passed: {}".format(failed_sum, len(counters["PASSED"])) + status_text = f"fail: {failed_sum}, passed: {len(counters['PASSED'])}" if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text @@ -998,10 +997,10 @@ class ClickhouseIntegrationTestsRunner: def write_results(results_file, status_file, results, status): - with open(results_file, "w") as f: + with open(results_file, "w", encoding="utf-8") as f: out = csv.writer(f, delimiter="\t") out.writerows(results) - with open(status_file, "w") as f: + with open(status_file, "w", encoding="utf-8") as f: out = csv.writer(f, delimiter="\t") out.writerow(status) @@ -1014,7 +1013,8 @@ if __name__ == "__main__": result_path = os.environ.get("CLICKHOUSE_TESTS_RESULT_PATH") params_path = os.environ.get("CLICKHOUSE_TESTS_JSON_PARAMS_PATH") - params = json.loads(open(params_path, "r").read()) + with open(params_path, "r", encoding="utf-8") as jfd: + params = json.loads(jfd.read()) runner = ClickhouseIntegrationTestsRunner(result_path, params) logging.info("Running tests") From 711da9505e990094b970d008e4307d3dee6c2a10 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Feb 2024 16:02:30 +0100 Subject: [PATCH 148/197] Fix liter issues in conftest.py --- tests/integration/conftest.py | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 3e03a49aaec..a4b18ff523a 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -1,12 +1,11 @@ #!/usr/bin/env python3 -import pytest import logging import os -from helpers.cluster import run_and_check -from helpers.test_tools import TSV -from helpers.network import _NetworkManager +import pytest # pylint:disable=import-error; for style check +from helpers.cluster import run_and_check +from helpers.network import _NetworkManager # This is a workaround for a problem with logging in pytest [1]. # @@ -34,32 +33,35 @@ def tune_local_port_range(): def cleanup_environment(): try: if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS", 0)) == 1: - logging.debug(f"Cleaning all iptables rules") + logging.debug("Cleaning all iptables rules") _NetworkManager.clean_all_user_iptables_rules() result = run_and_check(["docker ps | wc -l"], shell=True) if int(result) > 1: if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS", 0)) != 1: logging.warning( - f"Docker containters({int(result)}) are running before tests run. They can be left from previous pytest run and cause test failures.\n" - "You can set env PYTEST_CLEANUP_CONTAINERS=1 or use runner with --cleanup-containers argument to enable automatic containers cleanup." + "Docker containters(%s) are running before tests run. " + "They can be left from previous pytest run and cause test failures.\n" + "You can set env PYTEST_CLEANUP_CONTAINERS=1 or use runner with " + "--cleanup-containers argument to enable automatic containers cleanup.", + int(result), ) else: logging.debug("Trying to kill unstopped containers...") run_and_check( - [f"docker kill $(docker container list --all --quiet)"], + ["docker kill $(docker container list --all --quiet)"], shell=True, nothrow=True, ) run_and_check( - [f"docker rm $docker container list --all --quiet)"], + ["docker rm $docker container list --all --quiet)"], shell=True, nothrow=True, ) logging.debug("Unstopped containers killed") r = run_and_check(["docker-compose", "ps", "--services", "--all"]) - logging.debug(f"Docker ps before start:{r.stdout}") + logging.debug("Docker ps before start:%s", r.stdout) else: - logging.debug(f"No running containers") + logging.debug("No running containers") logging.debug("Pruning Docker networks") run_and_check( @@ -68,8 +70,7 @@ def cleanup_environment(): nothrow=True, ) except Exception as e: - logging.exception(f"cleanup_environment:{str(e)}") - pass + logging.exception("cleanup_environment:%s", e) yield From 770d7104745e7755feed946883ca962184d938a7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Feb 2024 17:39:40 +0100 Subject: [PATCH 149/197] Fix linter issues in sqllogic module --- tests/sqllogic/connection.py | 24 ++++---- tests/sqllogic/exceptions.py | 26 ++------ tests/sqllogic/runner.py | 41 +++++++------ tests/sqllogic/test_parser.py | 75 +++++++++++------------ tests/sqllogic/test_runner.py | 108 ++++++++++++++++------------------ 5 files changed, 124 insertions(+), 150 deletions(-) diff --git a/tests/sqllogic/connection.py b/tests/sqllogic/connection.py index 2d5e1f8a9e9..8bbb76d83df 100644 --- a/tests/sqllogic/connection.py +++ b/tests/sqllogic/connection.py @@ -1,18 +1,15 @@ #!/usr/bin/env python3 # -*- coding: utf-8 -*- -import datetime -import logging -import pyodbc -import sqlite3 -import traceback import enum +import logging import random +import sqlite3 import string from contextlib import contextmanager +import pyodbc # pylint:disable=import-error; for style check from exceptions import ProgramError - logger = logging.getLogger("connection") logger.setLevel(logging.DEBUG) @@ -22,9 +19,7 @@ class OdbcConnectingArgs: self._kwargs = kwargs def __str__(self): - conn_str = ";".join( - ["{}={}".format(x, y) for x, y in self._kwargs.items() if y] - ) + conn_str = ";".join([f"{x}={y}" for x, y in self._kwargs.items() if y]) return conn_str def update_database(self, database): @@ -49,6 +44,7 @@ class OdbcConnectingArgs: for kv in conn_str.split(";"): if kv: k, v = kv.split("=", 1) + # pylint:disable-next=protected-access args._kwargs[k] = v return args @@ -82,7 +78,7 @@ class KnownDBMS(str, enum.Enum): clickhouse = "ClickHouse" -class ConnectionWrap(object): +class ConnectionWrap: def __init__(self, connection=None, factory=None, factory_kwargs=None): self._factory = factory self._factory_kwargs = factory_kwargs @@ -126,7 +122,7 @@ class ConnectionWrap(object): f"SELECT name FROM system.tables WHERE database='{self.DATABASE_NAME}'" ) elif self.DBMS_NAME == KnownDBMS.sqlite.value: - list_query = f"SELECT name FROM sqlite_master WHERE type='table'" + list_query = "SELECT name FROM sqlite_master WHERE type='table'" else: logger.warning( "unable to drop all tables for unknown database: %s", self.DBMS_NAME @@ -154,7 +150,7 @@ class ConnectionWrap(object): self._use_database(database) logger.info( "currentDatabase : %s", - execute_request(f"SELECT currentDatabase()", self).get_result(), + execute_request("SELECT currentDatabase()", self).get_result(), ) @contextmanager @@ -174,7 +170,7 @@ class ConnectionWrap(object): def __exit__(self, *args): if hasattr(self._connection, "close"): - return self._connection.close() + self._connection.close() def setup_connection(engine, conn_str=None, make_debug_request=True): @@ -263,7 +259,7 @@ class ExecResult: def assert_no_exception(self): if self.has_exception(): raise ProgramError( - f"request doesn't have a result set, it has the exception", + "request doesn't have a result set, it has the exception", parent=self._exception, ) diff --git a/tests/sqllogic/exceptions.py b/tests/sqllogic/exceptions.py index 30c8983d80f..2e4da3fd78b 100644 --- a/tests/sqllogic/exceptions.py +++ b/tests/sqllogic/exceptions.py @@ -1,8 +1,6 @@ #!/usr/bin/env python3 # -*- coding: utf-8 -*- -from enum import Enum - class Error(Exception): def __init__( @@ -45,16 +43,8 @@ class Error(Exception): @property def reason(self): - return ", ".join( - ( - str(x) - for x in [ - super().__str__(), - "details: {}".format(self._details) if self._details else "", - ] - if x - ) - ) + details = f"details: {self._details}" if self._details else "" + return ", ".join((str(x) for x in [super().__str__(), details] if x)) def set_details(self, file=None, name=None, pos=None, request=None, details=None): if file is not None: @@ -88,16 +78,8 @@ class ErrorWithParent(Error): @property def reason(self): - return ", ".join( - ( - str(x) - for x in [ - super().reason, - "exception: {}".format(str(self._parent)) if self._parent else "", - ] - if x - ) - ) + exception = f"exception: {self._parent}" if self._parent else "" + return ", ".join((str(x) for x in [super().reason, exception] if x)) class ProgramError(ErrorWithParent): diff --git a/tests/sqllogic/runner.py b/tests/sqllogic/runner.py index 5f4baf8e59b..2e8e098a099 100755 --- a/tests/sqllogic/runner.py +++ b/tests/sqllogic/runner.py @@ -2,20 +2,25 @@ # -*- coding: utf-8 -*- import argparse -import enum -import os -import logging import csv +import enum import json +import logging import multiprocessing +import os from functools import reduce -from deepdiff import DeepDiff -from connection import setup_connection, Engines, default_clickhouse_odbc_conn_str -from test_runner import TestRunner, Status, RequestType +# isort: off +from deepdiff import DeepDiff # pylint:disable=import-error; for style check +# isort: on -LEVEL_NAMES = [x.lower() for x in logging._nameToLevel.keys() if x != logging.NOTSET] +from connection import Engines, default_clickhouse_odbc_conn_str, setup_connection +from test_runner import RequestType, Status, TestRunner + +LEVEL_NAMES = [ # pylint:disable-next=protected-access + l.lower() for l, n in logging._nameToLevel.items() if n != logging.NOTSET +] def setup_logger(args): @@ -41,7 +46,7 @@ def __write_check_status(status_row, out_dir): if len(status_row) > 140: status_row = status_row[0:135] + "..." check_status_path = os.path.join(out_dir, "check_status.tsv") - with open(check_status_path, "a") as stream: + with open(check_status_path, "a", encoding="utf-8") as stream: writer = csv.writer(stream, delimiter="\t", lineterminator="\n") writer.writerow(status_row) @@ -60,7 +65,7 @@ def __write_test_result( ): all_stages = reports.keys() test_results_path = os.path.join(out_dir, "test_results.tsv") - with open(test_results_path, "a") as stream: + with open(test_results_path, "a", encoding="utf-8") as stream: writer = csv.writer(stream, delimiter="\t", lineterminator="\n") for stage in all_stages: report = reports[stage] @@ -182,7 +187,7 @@ def mode_check_statements(parser): input_dir, f"check statements:: not a dir {input_dir}" ) - reports = dict() + reports = {} out_stages_dir = os.path.join(out_dir, f"{args.mode}-stages") @@ -242,7 +247,7 @@ def mode_check_complete(parser): input_dir, f"check statements:: not a dir {input_dir}" ) - reports = dict() + reports = {} out_stages_dir = os.path.join(out_dir, f"{args.mode}-stages") @@ -286,9 +291,9 @@ def make_actual_report(reports): return {stage: report.get_map() for stage, report in reports.items()} -def write_actual_report(actial, out_dir): - with open(os.path.join(out_dir, "actual_report.json"), "w") as f: - f.write(json.dumps(actial)) +def write_actual_report(actual, out_dir): + with open(os.path.join(out_dir, "actual_report.json"), "w", encoding="utf-8") as f: + f.write(json.dumps(actual)) def read_canonic_report(input_dir): @@ -296,13 +301,15 @@ def read_canonic_report(input_dir): if not os.path.exists(file): return {} - with open(os.path.join(input_dir, "canonic_report.json"), "r") as f: + with open( + os.path.join(input_dir, "canonic_report.json"), "r", encoding="utf-8" + ) as f: data = f.read() return json.loads(data) def write_canonic_report(canonic, out_dir): - with open(os.path.join(out_dir, "canonic_report.json"), "w") as f: + with open(os.path.join(out_dir, "canonic_report.json"), "w", encoding="utf-8") as f: f.write(json.dumps(canonic)) @@ -370,7 +377,7 @@ def mode_self_test(parser): if not os.path.isdir(out_dir): raise NotADirectoryError(out_dir, f"self test: not a dir {out_dir}") - reports = dict() + reports = {} out_stages_dir = os.path.join(out_dir, f"{args.mode}-stages") diff --git a/tests/sqllogic/test_parser.py b/tests/sqllogic/test_parser.py index cb1144d7dd9..648fa9f6bf6 100755 --- a/tests/sqllogic/test_parser.py +++ b/tests/sqllogic/test_parser.py @@ -2,24 +2,27 @@ # -*- coding: utf-8 -*- import logging -import os - -from itertools import chain from enum import Enum -from hashlib import md5 from functools import reduce +from hashlib import md5 +from itertools import chain + +# isort: off +# pylint:disable=import-error; for style check import sqlglot -from sqlglot.expressions import PrimaryKeyColumnConstraint, ColumnDef +from sqlglot.expressions import ColumnDef, PrimaryKeyColumnConstraint + +# pylint:enable=import-error; for style check +# isort: on from exceptions import ( - Error, - ProgramError, - ErrorWithParent, DataResultDiffer, + Error, + ErrorWithParent, + ProgramError, QueryExecutionError, ) - logger = logging.getLogger("parser") logger.setLevel(logging.DEBUG) @@ -248,6 +251,7 @@ class FileBlockBase: ) block.with_result(result) return block + raise ValueError(f"Unknown block_type {block_type}") def dump_to(self, output): if output is None: @@ -258,9 +262,6 @@ class FileBlockBase: class FileBlockComments(FileBlockBase): - def __init__(self, parser, start, end): - super().__init__(parser, start, end) - def get_block_type(self): return BlockType.comments @@ -469,20 +470,18 @@ class QueryResult: ( str(x) for x in [ - "rows: {}".format(self.rows) if self.rows else "", - "values_count: {}".format(self.values_count) - if self.values_count - else "", - "data_hash: {}".format(self.data_hash) if self.data_hash else "", - "exception: {}".format(self.exception) if self.exception else "", - "hash_threshold: {}".format(self.hash_threshold) + f"rows: {self.rows}" if self.rows else "", + f"values_count: {self.values_count}" if self.values_count else "", + f"data_hash: {self.data_hash}" if self.data_hash else "", + f"exception: {self.exception}" if self.exception else "", + f"hash_threshold: {self.hash_threshold}" if self.hash_threshold else "", ] if x ) ) - return "QueryResult({})".format(params) + return f"QueryResult({params})" def __iter__(self): if self.rows is not None: @@ -491,12 +490,10 @@ class QueryResult: if self.values_count <= self.hash_threshold: return iter(self.rows) if self.data_hash is not None: - return iter( - [["{} values hashing to {}".format(self.values_count, self.data_hash)]] - ) + return iter([[f"{self.values_count} values hashing to {self.data_hash}"]]) if self.exception is not None: - return iter([["exception: {}".format(self.exception)]]) - raise ProgramError("Query result is empty", details="{}".format(self.__str__())) + return iter([[f"exception: {self.exception}"]]) + raise ProgramError("Query result is empty", details=str(self)) @staticmethod def __value_count(rows): @@ -528,7 +525,7 @@ class QueryResult: for row in rows: res_row = [] for c, t in zip(row, types): - logger.debug(f"Builging row. c:{c} t:{t}") + logger.debug("Builging row. c:%s t:%s", c, t) if c is None: res_row.append("NULL") continue @@ -541,7 +538,7 @@ class QueryResult: elif t == "I": try: res_row.append(str(int(c))) - except ValueError as ex: + except ValueError: # raise QueryExecutionError( # f"Got non-integer result '{c}' for I type." # ) @@ -549,7 +546,7 @@ class QueryResult: except OverflowError as ex: raise QueryExecutionError( f"Got overflowed result '{c}' for I type." - ) + ) from ex elif t == "R": res_row.append(f"{c:.3f}") @@ -567,6 +564,7 @@ class QueryResult: values = list(chain(*rows)) values.sort() return [values] if values else [] + return [] @staticmethod def __calculate_hash(rows): @@ -595,9 +593,9 @@ class QueryResult: # do not print details to the test file # but print original exception if isinstance(e, ErrorWithParent): - message = "{}, original is: {}".format(e, e.get_parent()) + message = f"{e}, original is: {e.get_parent()}" else: - message = "{}".format(e) + message = str(e) return QueryResult(exception=message) @@ -616,9 +614,8 @@ class QueryResult: "canonic and actual results have different exceptions", details=f"canonic: {canonic.exception}, actual: {actual.exception}", ) - else: - # exceptions are the same - return + # exceptions are the same + return elif canonic.exception is not None: raise DataResultDiffer( "canonic result has exception and actual result doesn't", @@ -639,9 +636,8 @@ class QueryResult: if canonic.values_count != actual.values_count: raise DataResultDiffer( "canonic and actual results have different value count", - details="canonic values count {}, actual {}".format( - canonic.values_count, actual.values_count - ), + details=f"canonic values count {canonic.values_count}, " + f"actual {actual.values_count}", ) if canonic.data_hash != actual.data_hash: raise DataResultDiffer( @@ -653,9 +649,8 @@ class QueryResult: if canonic.values_count != actual.values_count: raise DataResultDiffer( "canonic and actual results have different value count", - details="canonic values count {}, actual {}".format( - canonic.values_count, actual.values_count - ), + details=f"canonic values count {canonic.values_count}, " + f"actual {actual.values_count}", ) if canonic.rows != actual.rows: raise DataResultDiffer( @@ -665,5 +660,5 @@ class QueryResult: raise ProgramError( "Unable to compare results", - details="actual {}, canonic {}".format(actual, canonic), + details=f"actual {actual}, canonic {canonic}", ) diff --git a/tests/sqllogic/test_runner.py b/tests/sqllogic/test_runner.py index baec0dc7924..8f2242a45b9 100644 --- a/tests/sqllogic/test_runner.py +++ b/tests/sqllogic/test_runner.py @@ -1,25 +1,23 @@ #!/usr/bin/env python3 # -*- coding: utf-8 -*- import enum -import logging -import os -import traceback import io import json +import logging +import os import test_parser +from connection import execute_request from exceptions import ( + DataResultDiffer, Error, ProgramError, - DataResultDiffer, - StatementExecutionError, - StatementSuccess, QueryExecutionError, QuerySuccess, SchemeResultDiffer, + StatementExecutionError, + StatementSuccess, ) -from connection import execute_request - logger = logging.getLogger("parser") logger.setLevel(logging.DEBUG) @@ -55,6 +53,7 @@ class Status(str, enum.Enum): class TestStatus: def __init__(self): + self.name = None self.status = None self.file = None self.position = None @@ -155,7 +154,7 @@ class SimpleStats: self.success += 1 def get_map(self): - result = dict() + result = {} result["success"] = self.success result["fail"] = self.fail return result @@ -187,7 +186,7 @@ class Stats: choose.update(status) def get_map(self): - result = dict() + result = {} result["statements"] = self.statements.get_map() result["queries"] = self.queries.get_map() result["total"] = self.total.get_map() @@ -205,7 +204,7 @@ class OneReport: self.test_name = test_name self.test_file = test_file self.stats = Stats() - self.requests = dict() # type: dict(int, TestStatus) + self.requests = {} def update(self, status): if not isinstance(status, TestStatus): @@ -218,11 +217,11 @@ class OneReport: return str(self.get_map()) def get_map(self): - result = dict() + result = {} result["test_name"] = self.test_name result["test_file"] = self.test_file result["stats"] = self.stats.get_map() - result["requests"] = dict() + result["requests"] = {} requests = result["requests"] for pos, status in self.requests.items(): requests[pos] = status.get_map() @@ -233,7 +232,7 @@ class Report: def __init__(self, dbms_name, input_dir=None): self.dbms_name = dbms_name self.stats = Stats() - self.tests = dict() # type: dict(str, OneReport) + self.tests = {} self.input_dir = input_dir self.output_dir = None @@ -256,7 +255,7 @@ class Report: self.output_dir = res_dir def get_map(self): - result = dict() + result = {} result["dbms_name"] = self.dbms_name result["stats"] = self.stats.get_map() result["input_dir"] = self.input_dir @@ -264,7 +263,7 @@ class Report: result["input_dir"] = self.input_dir if self.output_dir is not None: result["output_dir"] = self.output_dir - result["tests"] = dict() + result["tests"] = {} tests = result["tests"] for test_name, one_report in self.tests.items(): tests.update({test_name: one_report.get_map()}) @@ -297,8 +296,8 @@ class Report: def write_report(self, report_dir): report_path = os.path.join(report_dir, "report.json") - logger.info(f"create file {report_path}") - with open(report_path, "w") as stream: + logger.info("create file %s", report_path) + with open(report_path, "w", encoding="utf-8") as stream: stream.write(json.dumps(self.get_map(), indent=4)) @@ -434,38 +433,34 @@ class TestRunner: details=f"expected error: {expected_error}", parent=exec_res.get_exception(), ) - else: - clogger.debug("errors matched") - raise QuerySuccess() - else: - clogger.debug("missed error") - raise QueryExecutionError( - "query is expected to fail with error", - details="expected error: {}".format(expected_error), + clogger.debug("errors matched") + raise QuerySuccess() + clogger.debug("missed error") + raise QueryExecutionError( + "query is expected to fail with error", + details=f"expected error: {expected_error}", + ) + clogger.debug("success is expected") + if exec_res.has_exception(): + clogger.debug("had error") + if self.verify: + clogger.debug("verify mode") + canonic = test_parser.QueryResult.parse_it( + block.get_result(), 10 ) - else: - clogger.debug("success is expected") - if exec_res.has_exception(): - clogger.debug("had error") - if self.verify: - clogger.debug("verify mode") - canonic = test_parser.QueryResult.parse_it( - block.get_result(), 10 - ) - exception = QueryExecutionError( - "query execution failed with an exception", - parent=exec_res.get_exception(), - ) - actual = test_parser.QueryResult.as_exception(exception) - test_parser.QueryResult.assert_eq(canonic, actual) - block.with_result(actual) - raise QuerySuccess() - else: - clogger.debug("completion mode") - raise QueryExecutionError( - "query execution failed with an exception", - parent=exec_res.get_exception(), - ) + exception = QueryExecutionError( + "query execution failed with an exception", + parent=exec_res.get_exception(), + ) + actual = test_parser.QueryResult.as_exception(exception) + test_parser.QueryResult.assert_eq(canonic, actual) + block.with_result(actual) + raise QuerySuccess() + clogger.debug("completion mode") + raise QueryExecutionError( + "query execution failed with an exception", + parent=exec_res.get_exception(), + ) canonic_types = block.get_types() clogger.debug("canonic types %s", canonic_types) @@ -476,9 +471,8 @@ class TestRunner: if canonic_columns_count != actual_columns_count: raise SchemeResultDiffer( "canonic and actual columns count differ", - details="expected columns {}, actual columns {}".format( - canonic_columns_count, actual_columns_count - ), + details=f"expected columns {canonic_columns_count}, " + f"actual columns {actual_columns_count}", ) actual = test_parser.QueryResult.make_it( @@ -528,7 +522,7 @@ class TestRunner: self.report = Report(self.dbms_name, self._input_dir) if self.results is None: - self.results = dict() + self.results = {} if self.dbms_name == "ClickHouse" and test_name in [ "test/select5.test", @@ -536,7 +530,7 @@ class TestRunner: "test/evidence/slt_lang_replace.test", "test/evidence/slt_lang_droptrigger.test", ]: - logger.info(f"Let's skip test %s for ClickHouse", test_name) + logger.info("Let's skip test %s for ClickHouse", test_name) return with self.connection.with_one_test_scope(): @@ -565,7 +559,7 @@ class TestRunner: test_name = os.path.relpath(test_file, start=self._input_dir) logger.debug("open file %s", test_name) - with open(test_file, "r") as stream: + with open(test_file, "r", encoding="utf-8") as stream: self.run_one_test(stream, test_name, test_file) def run_all_tests_from_dir(self, input_dir): @@ -582,10 +576,10 @@ class TestRunner: for test_name, stream in self.results.items(): test_file = os.path.join(dir_path, test_name) - logger.info(f"create file {test_file}") + logger.info("create file %s", test_file) result_dir = os.path.dirname(test_file) os.makedirs(result_dir, exist_ok=True) - with open(test_file, "w") as output: + with open(test_file, "w", encoding="utf-8") as output: output.write(stream.getvalue()) def write_report(self, report_dir): From 34bb40583cd0180f79c380b93808ca7ad441bc96 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 28 Feb 2024 22:58:33 +0100 Subject: [PATCH 150/197] Install stubs for python-requests --- docker/test/style/Dockerfile | 1 + tests/ci/.mypy.ini | 2 +- tests/ci/build_download_helper.py | 5 +++-- .../ci/cancel_and_rerun_workflow_lambda/app.py | 2 +- tests/ci/ci_runners_metrics_lambda/app.py | 2 +- tests/ci/clean_lost_instances_lambda/app.py | 2 +- tests/ci/clickhouse_helper.py | 2 +- tests/ci/download_release_packages.py | 2 +- tests/ci/get_previous_release_tag.py | 2 +- tests/ci/jepsen_check.py | 2 +- .../lambda_shared/__init__.py | 2 +- .../lambda_shared/token.py | 2 +- tests/ci/runner_token_rotation_lambda/app.py | 2 +- tests/ci/slack_bot_ci_lambda/app.py | 18 ++++++------------ tests/ci/team_keys_lambda/app.py | 2 +- tests/ci/workflow_approve_rerun_lambda/app.py | 2 +- 16 files changed, 23 insertions(+), 27 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 6153aec2144..122f558bab2 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -19,6 +19,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ yamllint \ locales \ && pip3 install black==23.12.0 boto3 codespell==2.2.1 mypy==1.8.0 PyGithub unidiff pylint==3.1.0 \ + requests types-requests \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ && rm -rf /root/.cache/pip diff --git a/tests/ci/.mypy.ini b/tests/ci/.mypy.ini index bcaac49f190..9bc44025826 100644 --- a/tests/ci/.mypy.ini +++ b/tests/ci/.mypy.ini @@ -14,4 +14,4 @@ warn_unused_ignores = False warn_return_any = True no_implicit_reexport = True strict_equality = True -strict_concatenate = True +extra_checks = True diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index c7a461e003b..66734f58b23 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -9,7 +9,7 @@ from pathlib import Path from typing import Any, Callable, List, Union # isort: off -import requests # type: ignore +import requests # isort: on @@ -90,7 +90,8 @@ def get_gh_api( ratelimit_exceeded = ( e.response.status_code == 403 and b"rate limit exceeded" - in e.response._content # pylint:disable=protected-access + # pylint:disable-next=protected-access + in (e.response._content or b"") ) try_auth = e.response.status_code == 404 if (ratelimit_exceeded or try_auth) and not token_is_set: diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 656198c6985..4b7a931f772 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -9,7 +9,7 @@ from queue import Queue from threading import Thread from typing import Any, Dict, List, Optional, Tuple -import requests # type: ignore +import requests from lambda_shared.pr import CATEGORY_TO_LABEL, check_pr_description from lambda_shared.token import get_cached_access_token diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index 5cb1e45dd14..147ff127275 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -11,7 +11,7 @@ import sys from datetime import datetime from typing import Dict, List -import requests # type: ignore +import requests import boto3 # type: ignore from botocore.exceptions import ClientError # type: ignore diff --git a/tests/ci/clean_lost_instances_lambda/app.py b/tests/ci/clean_lost_instances_lambda/app.py index 65f6ff78d4a..5ec929a59e0 100644 --- a/tests/ci/clean_lost_instances_lambda/app.py +++ b/tests/ci/clean_lost_instances_lambda/app.py @@ -12,7 +12,7 @@ from datetime import datetime from dataclasses import dataclass from typing import Dict, List -import requests # type: ignore +import requests import boto3 # type: ignore from botocore.exceptions import ClientError # type: ignore diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 3ab41673dec..637c4519d3d 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -6,7 +6,7 @@ import time from pathlib import Path from typing import Dict, List, Optional -import requests # type: ignore +import requests from get_robot_token import get_parameter_from_ssm from pr_info import PRInfo from report import TestResults diff --git a/tests/ci/download_release_packages.py b/tests/ci/download_release_packages.py index 26223de2f8a..550301e8fa2 100755 --- a/tests/ci/download_release_packages.py +++ b/tests/ci/download_release_packages.py @@ -3,7 +3,7 @@ import os import logging -import requests # type: ignore +import requests from requests.adapters import HTTPAdapter # type: ignore from urllib3.util.retry import Retry # type: ignore diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index b74432425d3..bc0cb975ef5 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -4,7 +4,7 @@ import logging import re from typing import List, Optional, Tuple -import requests # type: ignore +import requests CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags" CLICKHOUSE_PACKAGE_URL = ( diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 01263c3085d..011ecff635e 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -9,7 +9,7 @@ from pathlib import Path from typing import Any, List import boto3 # type: ignore -import requests # type: ignore +import requests from build_download_helper import ( download_build_with_progress, get_build_name_for_check, diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py index 043a0310d11..8b53f9dcb23 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/__init__.py +++ b/tests/ci/lambda_shared_package/lambda_shared/__init__.py @@ -8,7 +8,7 @@ from collections import namedtuple from typing import Any, Dict, Iterable, List, Optional import boto3 # type: ignore -import requests # type: ignore +import requests RUNNER_TYPE_LABELS = [ "builder", diff --git a/tests/ci/lambda_shared_package/lambda_shared/token.py b/tests/ci/lambda_shared_package/lambda_shared/token.py index 6d5653f6a58..f9860f6ad2a 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/token.py +++ b/tests/ci/lambda_shared_package/lambda_shared/token.py @@ -6,7 +6,7 @@ from typing import Tuple import boto3 # type: ignore import jwt -import requests # type: ignore +import requests from . import cached_value_is_valid diff --git a/tests/ci/runner_token_rotation_lambda/app.py b/tests/ci/runner_token_rotation_lambda/app.py index 6544eee9581..03d11809d07 100644 --- a/tests/ci/runner_token_rotation_lambda/app.py +++ b/tests/ci/runner_token_rotation_lambda/app.py @@ -4,7 +4,7 @@ import argparse import sys import boto3 # type: ignore -import requests # type: ignore +import requests from lambda_shared.token import get_cached_access_token, get_access_token_by_key_app diff --git a/tests/ci/slack_bot_ci_lambda/app.py b/tests/ci/slack_bot_ci_lambda/app.py index 45e14138335..9e39f937bf2 100755 --- a/tests/ci/slack_bot_ci_lambda/app.py +++ b/tests/ci/slack_bot_ci_lambda/app.py @@ -17,12 +17,12 @@ It's deployed to slack-bot-ci-lambda in CI/CD account See also: https://aretestsgreenyet.com/ """ -import os -import json import base64 +import json +import os import random -import requests # type: ignore +import requests DRY_RUN_MARK = "" @@ -139,13 +139,11 @@ def get_play_url(query): def run_clickhouse_query(query): - url = "https://play.clickhouse.com/?user=play&query=" + requests.utils.quote(query) + url = "https://play.clickhouse.com/?user=play&query=" + requests.compat.quote(query) res = requests.get(url) if res.status_code != 200: print("Failed to execute query: ", res.status_code, res.content) - raise Exception( - "Failed to execute query: {}: {}".format(res.status_code, res.content) - ) + res.raise_for_status() lines = res.text.strip().splitlines() return [x.split("\t") for x in lines] @@ -283,11 +281,7 @@ def send_to_slack_impl(message): res = requests.post(SLACK_URL, json.dumps(payload)) if res.status_code != 200: print("Failed to send a message to Slack: ", res.status_code, res.content) - raise Exception( - "Failed to send a message to Slack: {}: {}".format( - res.status_code, res.content - ) - ) + res.raise_for_status() def send_to_slack(message): diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py index f562fbe101d..8d3a3502861 100644 --- a/tests/ci/team_keys_lambda/app.py +++ b/tests/ci/team_keys_lambda/app.py @@ -7,7 +7,7 @@ from datetime import datetime from queue import Queue from threading import Thread -import requests # type: ignore +import requests import boto3 # type: ignore diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 5e68f2d4b53..c72deea7c7e 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -6,7 +6,7 @@ import time from collections import namedtuple from urllib.parse import quote -import requests # type: ignore +import requests from lambda_shared.pr import TRUSTED_CONTRIBUTORS from lambda_shared.token import get_cached_access_token From d2f92483f298491f6380a2754e3810998764628f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 29 Feb 2024 02:21:39 +0100 Subject: [PATCH 151/197] Add even more python code to pylint check, fix lambdas --- pyproject.toml | 3 +- tests/ci/autoscale_runners_lambda/app.py | 4 +- .../cancel_and_rerun_workflow_lambda/app.py | 17 ++++--- tests/ci/ci_runners_metrics_lambda/app.py | 17 ++----- tests/ci/clean_lost_instances_lambda/app.py | 10 ++--- tests/ci/runner_token_rotation_lambda/app.py | 7 +-- tests/ci/slack_bot_ci_lambda/app.py | 45 +++++++++---------- tests/ci/team_keys_lambda/app.py | 9 ++-- tests/ci/terminate_runner_lambda/app.py | 8 ++-- tests/ci/workflow_approve_rerun_lambda/app.py | 14 +++--- utils/check-style/check-style | 10 ++++- 11 files changed, 75 insertions(+), 69 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 277b24d4fd7..3d05abd9ec2 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -12,7 +12,8 @@ max-statements=200 [tool.pylint.'MESSAGES CONTROL'] # pytest.mark.parametrize is not callable (not-callable) -disable = '''missing-docstring, +disable = ''' + missing-docstring, too-few-public-methods, invalid-name, too-many-arguments, diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 26a05ab0af4..1aa2e424320 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -51,7 +51,7 @@ class Queue: label: str -def get_scales(runner_type: str) -> Tuple[int, int]: +def get_scales() -> Tuple[int, int]: "returns the multipliers for scaling down and up ASG by types" # Scaling down is quicker on the lack of running jobs than scaling up on # queue @@ -95,7 +95,7 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") - scale_down, scale_up = get_scales(runner_type) + scale_down, scale_up = get_scales() # With lyfecycle hooks some instances are actually free because some of # them are in 'Terminating:Wait' state effective_capacity = max( diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 4b7a931f772..625936ec5c8 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -1,16 +1,15 @@ #!/usr/bin/env python3 import json -import re import time from base64 import b64decode from collections import namedtuple from queue import Queue from threading import Thread -from typing import Any, Dict, List, Optional, Tuple +from typing import Any, Dict, List, Optional import requests -from lambda_shared.pr import CATEGORY_TO_LABEL, check_pr_description +from lambda_shared.pr import check_pr_description from lambda_shared.token import get_cached_access_token NEED_RERUN_OR_CANCELL_WORKFLOWS = { @@ -48,16 +47,18 @@ class Worker(Thread): def _exec_get_with_retry(url: str, token: str) -> dict: headers = {"Authorization": f"token {token}"} + e = Exception() for i in range(MAX_RETRY): try: - response = requests.get(url, headers=headers) + response = requests.get(url, headers=headers, timeout=30) response.raise_for_status() return response.json() # type: ignore except Exception as ex: print("Got exception executing request", ex) + e = ex time.sleep(i + 1) - raise Exception("Cannot execute GET request with retries") + raise requests.HTTPError("Cannot execute GET request with retries") from e WorkflowDescription = namedtuple( @@ -215,16 +216,18 @@ def get_workflow_description(workflow_url: str, token: str) -> WorkflowDescripti def _exec_post_with_retry(url: str, token: str, json: Optional[Any] = None) -> Any: headers = {"Authorization": f"token {token}"} + e = Exception() for i in range(MAX_RETRY): try: - response = requests.post(url, headers=headers, json=json) + response = requests.post(url, headers=headers, json=json, timeout=30) response.raise_for_status() return response.json() except Exception as ex: print("Got exception executing request", ex) + e = ex time.sleep(i + 1) - raise Exception("Cannot execute POST request with retry") + raise requests.HTTPError("Cannot execute POST request with retry") from e def exec_workflow_url(urls_to_post, token): diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index 147ff127275..47161215a97 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -8,23 +8,14 @@ Lambda function to: import argparse import sys -from datetime import datetime -from typing import Dict, List +from typing import Dict -import requests import boto3 # type: ignore -from botocore.exceptions import ClientError # type: ignore - -from lambda_shared import ( - RUNNER_TYPE_LABELS, - RunnerDescription, - RunnerDescriptions, - list_runners, -) +from lambda_shared import RUNNER_TYPE_LABELS, RunnerDescriptions, list_runners from lambda_shared.token import ( + get_access_token_by_key_app, get_cached_access_token, get_key_and_app_from_aws, - get_access_token_by_key_app, ) UNIVERSAL_LABEL = "universal" @@ -162,7 +153,7 @@ if __name__ == "__main__": if args.private_key: private_key = args.private_key elif args.private_key_path: - with open(args.private_key_path, "r") as key_file: + with open(args.private_key_path, "r", encoding="utf-8") as key_file: private_key = key_file.read() else: print("Attempt to get key and id from AWS secret manager") diff --git a/tests/ci/clean_lost_instances_lambda/app.py b/tests/ci/clean_lost_instances_lambda/app.py index 5ec929a59e0..4accc14f7ae 100644 --- a/tests/ci/clean_lost_instances_lambda/app.py +++ b/tests/ci/clean_lost_instances_lambda/app.py @@ -8,14 +8,13 @@ Lambda function to: import argparse import sys -from datetime import datetime from dataclasses import dataclass +from datetime import datetime from typing import Dict, List -import requests import boto3 # type: ignore +import requests from botocore.exceptions import ClientError # type: ignore - from lambda_shared import ( RUNNER_TYPE_LABELS, RunnerDescription, @@ -23,9 +22,9 @@ from lambda_shared import ( list_runners, ) from lambda_shared.token import ( + get_access_token_by_key_app, get_cached_access_token, get_key_and_app_from_aws, - get_access_token_by_key_app, ) UNIVERSAL_LABEL = "universal" @@ -140,6 +139,7 @@ def delete_runner(access_token: str, runner: RunnerDescription) -> bool: response = requests.delete( f"https://api.github.com/orgs/ClickHouse/actions/runners/{runner.id}", headers=headers, + timeout=30, ) response.raise_for_status() print(f"Response code deleting {runner.name} is {response.status_code}") @@ -325,7 +325,7 @@ if __name__ == "__main__": if args.private_key: private_key = args.private_key elif args.private_key_path: - with open(args.private_key_path, "r") as key_file: + with open(args.private_key_path, "r", encoding="utf-8") as key_file: private_key = key_file.read() else: print("Attempt to get key and id from AWS secret manager") diff --git a/tests/ci/runner_token_rotation_lambda/app.py b/tests/ci/runner_token_rotation_lambda/app.py index 03d11809d07..445704648a9 100644 --- a/tests/ci/runner_token_rotation_lambda/app.py +++ b/tests/ci/runner_token_rotation_lambda/app.py @@ -5,8 +5,7 @@ import sys import boto3 # type: ignore import requests - -from lambda_shared.token import get_cached_access_token, get_access_token_by_key_app +from lambda_shared.token import get_access_token_by_key_app, get_cached_access_token def get_runner_registration_token(access_token): @@ -17,6 +16,7 @@ def get_runner_registration_token(access_token): response = requests.post( "https://api.github.com/orgs/ClickHouse/actions/runners/registration-token", headers=headers, + timeout=30, ) response.raise_for_status() data = response.json() @@ -43,6 +43,7 @@ def main(access_token, push_to_ssm, ssm_parameter_name): def handler(event, context): + _, _ = event, context main(get_cached_access_token(), True, "github_runner_registration_token") @@ -85,7 +86,7 @@ if __name__ == "__main__": if args.private_key: private_key = args.private_key else: - with open(args.private_key_path, "r") as key_file: + with open(args.private_key_path, "r", encoding="utf-8") as key_file: private_key = key_file.read() token = get_access_token_by_key_app(private_key, args.app_id) diff --git a/tests/ci/slack_bot_ci_lambda/app.py b/tests/ci/slack_bot_ci_lambda/app.py index 9e39f937bf2..94b71724b1c 100755 --- a/tests/ci/slack_bot_ci_lambda/app.py +++ b/tests/ci/slack_bot_ci_lambda/app.py @@ -140,7 +140,7 @@ def get_play_url(query): def run_clickhouse_query(query): url = "https://play.clickhouse.com/?user=play&query=" + requests.compat.quote(query) - res = requests.get(url) + res = requests.get(url, timeout=30) if res.status_code != 200: print("Failed to execute query: ", res.status_code, res.content) res.raise_for_status() @@ -157,9 +157,9 @@ def split_broken_and_flaky_tests(failed_tests): flaky_tests = [] for name, report, count_prev_str, count_str in failed_tests: count_prev, count = int(count_prev_str), int(count_str) - if (2 <= count and count_prev < 2) or (count_prev == 1 and count == 1): + if (count_prev < 2 <= count) or (count_prev == count == 1): # It failed 2 times or more within extended time window, it's definitely broken. - # 2 <= count_prev means that it was not reported as broken on previous runs + # 2 <= count means that it was not reported as broken on previous runs broken_tests.append([name, report]) elif 0 < count and count_prev == 0: # It failed only once, can be a rare flaky test @@ -170,19 +170,18 @@ def split_broken_and_flaky_tests(failed_tests): def format_failed_tests_list(failed_tests, failure_type): if len(failed_tests) == 1: - res = "There is a new {} test:\n".format(failure_type) + res = f"There is a new {failure_type} test:\n" else: - res = "There are {} new {} tests:\n".format(len(failed_tests), failure_type) + res = f"There are {len(failed_tests)} new {failure_type} tests:\n" for name, report in failed_tests[:MAX_TESTS_TO_REPORT]: cidb_url = get_play_url(ALL_RECENT_FAILURES_QUERY.format(name)) - res += "- *{}* - <{}|Report> - <{}|CI DB> \n".format( - name, report, cidb_url - ) + res += f"- *{name}* - <{report}|Report> - <{cidb_url}|CI DB> \n" if MAX_TESTS_TO_REPORT < len(failed_tests): - res += "- and {} other tests... :this-is-fine-fire:".format( - len(failed_tests) - MAX_TESTS_TO_REPORT + res += ( + f"- and {len(failed_tests) - MAX_TESTS_TO_REPORT} other " + "tests... :this-is-fine-fire:" ) return res @@ -221,19 +220,16 @@ def get_too_many_failures_message_impl(failures_count): if random.random() < REPORT_NO_FAILURES_PROBABILITY: return None return "Wow, there are *no failures* at all... 0_o" - if curr_failures < MAX_FAILURES: + return_none = ( + curr_failures < MAX_FAILURES + or curr_failures < prev_failures + or (curr_failures - prev_failures) / prev_failures < 0.2 + ) + if return_none: return None if prev_failures < MAX_FAILURES: - return ":alert: *CI is broken: there are {} failures during the last 24 hours*".format( - curr_failures - ) - if curr_failures < prev_failures: - return None - if (curr_failures - prev_failures) / prev_failures < 0.2: - return None - return "CI is broken and it's getting worse: there are {} failures during the last 24 hours".format( - curr_failures - ) + return f":alert: *CI is broken: there are {curr_failures} failures during the last 24 hours*" + return "CI is broken and it's getting worse: there are {curr_failures} failures during the last 24 hours" def get_too_many_failures_message(failures_count): @@ -252,7 +248,7 @@ def get_failed_checks_percentage_message(percentage): return None msg = ":alert: " if p > 1 else "Only " if p < 0.5 else "" - msg += "*{0:.2f}%* of all checks in master have failed yesterday".format(p) + msg += f"*{p:.2f}%* of all checks in master have failed yesterday" return msg @@ -278,7 +274,7 @@ def send_to_slack_impl(message): payload = SLACK_MESSAGE_JSON.copy() payload["text"] = message - res = requests.post(SLACK_URL, json.dumps(payload)) + res = requests.post(SLACK_URL, json.dumps(payload), timeout=30) if res.status_code != 200: print("Failed to send a message to Slack: ", res.status_code, res.content) res.raise_for_status() @@ -297,7 +293,7 @@ def query_and_alert_if_needed(query, get_message_func): if msg is None: return - msg += "\nCI DB query: <{}|link>".format(get_play_url(query)) + msg += f"\nCI DB query: <{get_play_url(query)}|link>" print("Sending message to slack:", msg) send_to_slack(msg) @@ -311,6 +307,7 @@ def check_and_alert(): def handler(event, context): + _, _ = event, context try: check_and_alert() return {"statusCode": 200, "body": "OK"} diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py index 8d3a3502861..c056808a9b4 100644 --- a/tests/ci/team_keys_lambda/app.py +++ b/tests/ci/team_keys_lambda/app.py @@ -2,13 +2,12 @@ import argparse import json - from datetime import datetime from queue import Queue from threading import Thread -import requests import boto3 # type: ignore +import requests class Keys(set): @@ -34,7 +33,7 @@ class Worker(Thread): m = self.queue.get() if m == "": break - response = requests.get(f"https://github.com/{m}.keys") + response = requests.get(f"https://github.com/{m}.keys", timeout=30) self.results.add(f"# {m}\n{response.text}\n") self.queue.task_done() @@ -45,7 +44,9 @@ def get_org_team_members(token: str, org: str, team_slug: str) -> set: "Accept": "application/vnd.github.v3+json", } response = requests.get( - f"https://api.github.com/orgs/{org}/teams/{team_slug}/members", headers=headers + f"https://api.github.com/orgs/{org}/teams/{team_slug}/members", + headers=headers, + timeout=30, ) response.raise_for_status() data = response.json() diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 010f7dd6734..ab3e33d9e11 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -8,8 +8,7 @@ from dataclasses import dataclass from typing import Any, Dict, List import boto3 # type: ignore - -from lambda_shared import RunnerDescriptions, list_runners, cached_value_is_valid +from lambda_shared import RunnerDescriptions, cached_value_is_valid, list_runners from lambda_shared.token import get_access_token_by_key_app, get_cached_access_token @@ -134,7 +133,7 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: candidates = instances_by_zone[zone] total_to_kill += num_to_kill if num_to_kill > len(candidates): - raise Exception( + raise RuntimeError( f"Required to kill {num_to_kill}, but have only {len(candidates)}" f" candidates in AV {zone}" ) @@ -196,6 +195,7 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: def handler(event: dict, context: Any) -> Dict[str, List[str]]: + _ = context return main(get_cached_access_token(), event) @@ -226,7 +226,7 @@ if __name__ == "__main__": if args.private_key: private_key = args.private_key else: - with open(args.private_key_path, "r") as key_file: + with open(args.private_key_path, "r", encoding="utf-8") as key_file: private_key = key_file.read() token = get_access_token_by_key_app(private_key, args.app_id) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index c72deea7c7e..8ed47a54ab1 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -90,26 +90,29 @@ def is_trusted_contributor(pr_user_login, pr_user_orgs): def _exec_get_with_retry(url, token): headers = {"Authorization": f"token {token}"} + e = Exception() for i in range(MAX_RETRY): try: - response = requests.get(url, headers=headers) + response = requests.get(url, headers=headers, timeout=30) response.raise_for_status() return response.json() except Exception as ex: print("Got exception executing request", ex) + e = ex time.sleep(i + 1) - raise Exception("Cannot execute GET request with retries") + raise requests.HTTPError("Cannot execute GET request with retries") from e def _exec_post_with_retry(url, token, data=None): headers = {"Authorization": f"token {token}"} + e = Exception() for i in range(MAX_RETRY): try: if data: - response = requests.post(url, headers=headers, json=data) + response = requests.post(url, headers=headers, json=data, timeout=30) else: - response = requests.post(url, headers=headers) + response = requests.post(url, headers=headers, timeout=30) if response.status_code == 403: data = response.json() if ( @@ -123,9 +126,10 @@ def _exec_post_with_retry(url, token, data=None): return response.json() except Exception as ex: print("Got exception executing request", ex) + e = ex time.sleep(i + 1) - raise Exception("Cannot execute POST request with retry") + raise requests.HTTPError("Cannot execute POST request with retry") from e def _get_pull_requests_from(repo_url, owner, branch, token): diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 620aec2fda2..d1fb2d81a28 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -152,7 +152,15 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.xml' | grep -vP $EXCLUDE_DIRS | xargs xmllint --noout --nonet -pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test $ROOT_PATH/tests/**/*.py +function xargs-pylint { + # $1 is number maximum arguments per pylint process + sort | awk '$2=="text/x-script.python" {print $1}' | \ + xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/.pylintrc" --persistent=no --score=n +} + +find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 50 +# Beware, there lambdas are checked. All of them contain `app`, and it causes brain-cucumber-zalgo +find "$ROOT_PATH/tests/ci" -mindepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 1 find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*.yml' \) -type f | grep -vP $EXCLUDE_DIRS | From 859044221f0dc490d1464b8fe8adc887e8c842ac Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 29 Feb 2024 12:23:04 +0100 Subject: [PATCH 152/197] Fix tests/integration/runner linter issues --- tests/integration/runner | 40 ++++++++++++++++++---------------------- 1 file changed, 18 insertions(+), 22 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index b1193b5b471..f1d5198f545 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -39,9 +39,7 @@ def check_args_and_update_paths(args): else: CLICKHOUSE_ROOT = args.clickhouse_root else: - logging.info( - "ClickHouse root is not set. Will use %s" % (DEFAULT_CLICKHOUSE_ROOT) - ) + logging.info("ClickHouse root is not set. Will use %s", DEFAULT_CLICKHOUSE_ROOT) CLICKHOUSE_ROOT = DEFAULT_CLICKHOUSE_ROOT if not os.path.isabs(args.binary): @@ -74,9 +72,7 @@ def check_args_and_update_paths(args): args.base_configs_dir = os.path.abspath( os.path.join(CLICKHOUSE_ROOT, CONFIG_DIR_IN_REPO) ) - logging.info( - "Base configs dir is not set. Will use %s" % (args.base_configs_dir) - ) + logging.info("Base configs dir is not set. Will use %s", args.base_configs_dir) if args.cases_dir: if not os.path.isabs(args.cases_dir): @@ -87,7 +83,7 @@ def check_args_and_update_paths(args): args.cases_dir = os.path.abspath( os.path.join(CLICKHOUSE_ROOT, INTEGRATION_DIR_IN_REPO) ) - logging.info("Cases dir is not set. Will use %s" % (args.cases_dir)) + logging.info("Cases dir is not set. Will use %s", args.cases_dir) if args.utils_dir: if not os.path.isabs(args.utils_dir): @@ -98,12 +94,13 @@ def check_args_and_update_paths(args): args.utils_dir = os.path.abspath( os.path.join(CLICKHOUSE_ROOT, UTILS_DIR_IN_REPO) ) - logging.info("utils dir is not set. Will use %s" % (args.utils_dir)) + logging.info("utils dir is not set. Will use %s", args.utils_dir) logging.info( - "base_configs_dir: {}, binary: {}, cases_dir: {} ".format( - args.base_configs_dir, args.binary, args.cases_dir - ) + "base_configs_dir: %s, binary: %s, cases_dir: %s ", + args.base_configs_dir, + args.binary, + args.cases_dir, ) for path in [ @@ -115,7 +112,7 @@ def check_args_and_update_paths(args): CLICKHOUSE_ROOT, ]: if not os.path.exists(path): - raise Exception("Path {} doesn't exist".format(path)) + raise FileNotFoundError(f"Path {path} doesn't exist") if args.dockerd_volume: if not os.path.isabs(args.dockerd_volume): @@ -126,21 +123,22 @@ def check_args_and_update_paths(args): if (not os.path.exists(os.path.join(args.base_configs_dir, "config.xml"))) and ( not os.path.exists(os.path.join(args.base_configs_dir, "config.yaml")) ): - raise Exception( - "No config.xml or config.yaml in {}".format(args.base_configs_dir) + raise FileNotFoundError( + f"No config.xml or config.yaml in {args.base_configs_dir}" ) if (not os.path.exists(os.path.join(args.base_configs_dir, "users.xml"))) and ( not os.path.exists(os.path.join(args.base_configs_dir, "users.yaml")) ): - raise Exception( - "No users.xml or users.yaml in {}".format(args.base_configs_dir) + raise FileNotFoundError( + f"No users.xml or users.yaml in {args.base_configs_dir}" ) def docker_kill_handler_handler(signum, frame): + _, _ = signum, frame subprocess.check_call( - "docker ps --all --quiet --filter name={name}".format(name=CONTAINER_NAME), + f"docker ps --all --quiet --filter name={CONTAINER_NAME}", shell=True, ) raise KeyboardInterrupt("Killed by Ctrl+C") @@ -318,7 +316,7 @@ if __name__ == "__main__": parallel_args = "" if args.parallel: parallel_args += "--dist=loadfile" - parallel_args += " -n {}".format(args.parallel) + parallel_args += f" -n {args.parallel}".format() rand_args = "" # if not args.no_random: @@ -326,7 +324,7 @@ if __name__ == "__main__": net = "" if args.network: - net = "--net={}".format(args.network) + net = f"--net={args.network}" elif not args.disable_net_host: net = "--net=host" @@ -350,9 +348,7 @@ if __name__ == "__main__": dockerd_internal_volume = "--tmpfs /var/lib/docker -e DOCKER_RAMDISK=true" elif args.dockerd_volume: dockerd_internal_volume = ( - "--mount type=bind,source={},target=/var/lib/docker".format( - args.dockerd_volume - ) + f"--mount type=bind,source={args.dockerd_volume},target=/var/lib/docker" ) else: try: From c4009a16f90920e3ce59b1376d5bc7529aa8d450 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Tue, 5 Mar 2024 22:29:50 +0800 Subject: [PATCH 153/197] Refactor OptimizeDateOrDateTimeConverterWithPreimageVisitor The generateOptimizedDateFilter function is refactored to enhance the code readability. And this commit also fixes the duplicate creations of ColumnNode. --- ...ateOrDateTimeConverterWithPreimagePass.cpp | 88 ++-- ...mizations_ast_query_tree_rewrite.reference | 424 +++++++++--------- 2 files changed, 241 insertions(+), 271 deletions(-) diff --git a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp index cc334cde9c7..0c37749c706 100644 --- a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp +++ b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp @@ -94,7 +94,8 @@ public: if (!func_node || func_node->getArguments().getNodes().size() != 1) return; - const auto * column_id = func_node->getArguments().getNodes()[0]->as(); + const auto & argument_node = func_node->getArguments().getNodes()[0]; + const auto * column_id = argument_node->as(); if (!column_id) return; @@ -119,7 +120,7 @@ public: if (!preimage_range) return; - const auto new_node = generateOptimizedDateFilter(comparator, *column_id, *preimage_range); + const auto new_node = generateOptimizedDateFilter(comparator, argument_node, *preimage_range); if (!new_node) return; @@ -128,20 +129,22 @@ public: } private: - QueryTreeNodePtr - generateOptimizedDateFilter(const String & comparator, const ColumnNode & column_node, const std::pair & range) const + QueryTreeNodePtr generateOptimizedDateFilter( + const String & comparator, const QueryTreeNodePtr & column_node, const std::pair & range) const { const DateLUTImpl & date_lut = DateLUT::instance("UTC"); String start_date_or_date_time; String end_date_or_date_time; - if (isDateOrDate32(column_node.getColumnType().get())) + const auto & column_node_typed = column_node->as(); + const auto & column_type = column_node_typed.getColumnType().get(); + if (isDateOrDate32(column_type)) { start_date_or_date_time = date_lut.dateToString(range.first.get()); end_date_or_date_time = date_lut.dateToString(range.second.get()); } - else if (isDateTime(column_node.getColumnType().get()) || isDateTime64(column_node.getColumnType().get())) + else if (isDateTime(column_type) || isDateTime64(column_type)) { start_date_or_date_time = date_lut.timeToString(range.first.get()); end_date_or_date_time = date_lut.timeToString(range.second.get()); @@ -151,69 +154,29 @@ private: if (comparator == "equals") { - const auto lhs = std::make_shared("greaterOrEquals"); - lhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); - lhs->getArguments().getNodes().push_back(std::make_shared(start_date_or_date_time)); - resolveOrdinaryFunctionNode(*lhs, lhs->getFunctionName()); - - const auto rhs = std::make_shared("less"); - rhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); - rhs->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); - resolveOrdinaryFunctionNode(*rhs, rhs->getFunctionName()); - - const auto new_date_filter = std::make_shared("and"); - new_date_filter->getArguments().getNodes() = {lhs, rhs}; - resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); - - return new_date_filter; + return createFunctionNode( + "and", + createFunctionNode("greaterOrEquals", column_node, std::make_shared(start_date_or_date_time)), + createFunctionNode("less", column_node, std::make_shared(end_date_or_date_time))); } else if (comparator == "notEquals") { - const auto lhs = std::make_shared("less"); - lhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); - lhs->getArguments().getNodes().push_back(std::make_shared(start_date_or_date_time)); - resolveOrdinaryFunctionNode(*lhs, lhs->getFunctionName()); - - const auto rhs = std::make_shared("greaterOrEquals"); - rhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); - rhs->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); - resolveOrdinaryFunctionNode(*rhs, rhs->getFunctionName()); - - const auto new_date_filter = std::make_shared("or"); - new_date_filter->getArguments().getNodes() = {lhs, rhs}; - resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); - - return new_date_filter; + return createFunctionNode( + "or", + createFunctionNode("less", column_node, std::make_shared(start_date_or_date_time)), + createFunctionNode("greaterOrEquals", column_node, std::make_shared(end_date_or_date_time))); } else if (comparator == "greater") { - const auto new_date_filter = std::make_shared("greaterOrEquals"); - new_date_filter->getArguments().getNodes().push_back( - std::make_shared(column_node.getColumn(), column_node.getColumnSource())); - new_date_filter->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); - resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); - - return new_date_filter; + return createFunctionNode("greaterOrEquals", column_node, std::make_shared(end_date_or_date_time)); } else if (comparator == "lessOrEquals") { - const auto new_date_filter = std::make_shared("less"); - new_date_filter->getArguments().getNodes().push_back( - std::make_shared(column_node.getColumn(), column_node.getColumnSource())); - new_date_filter->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); - resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); - - return new_date_filter; + return createFunctionNode("less", column_node, std::make_shared(end_date_or_date_time)); } else if (comparator == "less" || comparator == "greaterOrEquals") { - const auto new_date_filter = std::make_shared(comparator); - new_date_filter->getArguments().getNodes().push_back( - std::make_shared(column_node.getColumn(), column_node.getColumnSource())); - new_date_filter->getArguments().getNodes().push_back(std::make_shared(start_date_or_date_time)); - resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); - - return new_date_filter; + return createFunctionNode(comparator, column_node, std::make_shared(start_date_or_date_time)); } else [[unlikely]] { @@ -224,10 +187,17 @@ private: } } - void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const + template + QueryTreeNodePtr createFunctionNode(const String & function_name, Args &&... args) const { auto function = FunctionFactory::instance().get(function_name, getContext()); - function_node.resolveAsFunction(function->build(function_node.getArgumentColumns())); + const auto function_node = std::make_shared(function_name); + auto & new_arguments = function_node->getArguments().getNodes(); + new_arguments.reserve(sizeof...(args)); + (new_arguments.push_back(std::forward(args)), ...); + function_node->resolveAsFunction(function->build(function_node->getArgumentColumns())); + + return function_node; } }; diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference index 63658890119..fca48238778 100644 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference @@ -24,21 +24,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -66,21 +66,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -244,21 +244,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 15, constant_value: \'1998-01-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 14, constant_value: \'1998-01-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -289,34 +289,34 @@ QUERY id: 0 FUNCTION id: 14, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 15, nodes: 2 - COLUMN id: 16, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 17, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 18, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 16, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 17, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - FUNCTION id: 20, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + FUNCTION id: 19, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 21, nodes: 2 - COLUMN id: 22, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 23, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 24, function_name: less, function_type: ordinary, result_type: UInt8 + LIST id: 20, nodes: 2 + COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 21, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 22, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 25, nodes: 2 - COLUMN id: 26, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 27, constant_value: \'1995-01-01\', constant_value_type: String - FUNCTION id: 28, function_name: and, function_type: ordinary, result_type: UInt8 + LIST id: 23, nodes: 2 + COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 24, constant_value: \'1995-01-01\', constant_value_type: String + FUNCTION id: 25, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 29, nodes: 2 - FUNCTION id: 30, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 26, nodes: 2 + FUNCTION id: 27, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 31, nodes: 2 - COLUMN id: 32, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 33, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 34, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 28, nodes: 2 + COLUMN id: 29, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 30, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 31, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 35, nodes: 2 - COLUMN id: 32, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 36, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 32, nodes: 2 + COLUMN id: 29, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 33, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1, @@ -346,26 +346,26 @@ QUERY id: 0 FUNCTION id: 11, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 12, nodes: 2 - COLUMN id: 13, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 14, constant_value: \'1993-01-01\', constant_value_type: String - FUNCTION id: 15, function_name: less, function_type: ordinary, result_type: UInt8 + COLUMN id: 6, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 13, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 14, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 16, nodes: 2 - COLUMN id: 17, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 18, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 19, function_name: and, function_type: ordinary, result_type: UInt8 + LIST id: 15, nodes: 2 + COLUMN id: 6, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 16, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 17, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 20, nodes: 2 - FUNCTION id: 21, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + FUNCTION id: 19, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 22, nodes: 2 - COLUMN id: 23, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 25, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 20, nodes: 2 + COLUMN id: 21, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 22, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 23, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 26, nodes: 2 - COLUMN id: 23, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 27, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 24, nodes: 2 + COLUMN id: 21, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 25, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -425,22 +425,22 @@ QUERY id: 0 FUNCTION id: 10, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 11, nodes: 2 - COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 13, constant_value: \'1994-01-01\', constant_value_type: String + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 12, constant_value: \'1994-01-01\', constant_value_type: String WHERE - FUNCTION id: 14, function_name: and, function_type: ordinary, result_type: UInt8 + FUNCTION id: 13, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 15, nodes: 2 - FUNCTION id: 16, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 14, nodes: 2 + FUNCTION id: 15, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 18, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 19, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 20, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + COLUMN id: 17, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 19, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 21, nodes: 2 - COLUMN id: 18, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 22, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 20, nodes: 2 + COLUMN id: 17, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -479,8 +479,8 @@ QUERY id: 0 FUNCTION id: 19, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 20, nodes: 2 - COLUMN id: 21, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 22, constant_value: \'1994-01-01\', constant_value_type: String + COLUMN id: 17, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 21, constant_value: \'1994-01-01\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -582,21 +582,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -624,21 +624,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 15, constant_value: \'1992-04-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 14, constant_value: \'1992-04-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -666,21 +666,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 15, constant_value: \'1992-04-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 14, constant_value: \'1992-04-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t @@ -847,26 +847,26 @@ QUERY id: 0 FUNCTION id: 14, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 15, nodes: 2 - COLUMN id: 16, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 17, constant_value: \'1993-01-01\', constant_value_type: String - FUNCTION id: 18, function_name: less, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 16, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 17, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 21, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 22, function_name: and, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 19, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 20, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - FUNCTION id: 24, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 21, nodes: 2 + FUNCTION id: 22, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 25, nodes: 2 - COLUMN id: 26, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 27, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 28, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 23, nodes: 2 + COLUMN id: 24, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 25, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 26, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 29, nodes: 2 - COLUMN id: 26, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 30, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 27, nodes: 2 + COLUMN id: 24, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 28, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime_t @@ -894,21 +894,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: DateTime, source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: DateTime, source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime_t @@ -936,21 +936,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: DateTime, source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: DateTime, source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date32_t @@ -978,21 +978,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date32, source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date32, source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date32_t @@ -1020,21 +1020,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: Date32, source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: Date32, source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime64_t @@ -1062,21 +1062,21 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: DateTime64(3), source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: DateTime64(3), source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime64_t @@ -1104,19 +1104,19 @@ QUERY id: 0 FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: date1, result_type: DateTime64(3), source_id: 3 - CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String - FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + COLUMN id: 10, column_name: date1, result_type: DateTime64(3), source_id: 3 + CONSTANT id: 14, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 15, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 16, nodes: 2 + FUNCTION id: 17, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 21, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + LIST id: 22, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 From 7fe7f3a79dbdf0e24a03bc63a57486e2ae14f7e8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 5 Mar 2024 16:03:02 +0000 Subject: [PATCH 154/197] address review comments --- src/Interpreters/MutationsInterpreter.cpp | 2 +- .../Merges/Algorithms/SummingSortedAlgorithm.cpp | 2 +- .../Optimizations/optimizeUseNormalProjection.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- src/Storages/IStorage.h | 6 +++--- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 4 +++- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- src/Storages/MergeTree/MergeTask.h | 2 +- src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- src/Storages/MergeTree/MergeTreeReadTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeSelectProcessor.cpp | 2 +- src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 2 +- .../{ => MergeTree}/MergeTreeVirtualColumns.cpp | 2 +- src/Storages/{ => MergeTree}/MergeTreeVirtualColumns.h | 0 src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 2 +- src/Storages/StorageLog.cpp | 1 - src/Storages/StorageSnapshot.cpp | 1 - src/Storages/StorageSnapshot.h | 3 --- src/Storages/StorageValues.cpp | 4 ++-- src/Storages/StorageValues.h | 2 +- src/Storages/System/StorageSystemDictionaries.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 10 +++++----- src/Storages/VirtualColumnsDescription.h | 4 +++- 30 files changed, 38 insertions(+), 39 deletions(-) rename src/Storages/{ => MergeTree}/MergeTreeVirtualColumns.cpp (96%) rename src/Storages/{ => MergeTree}/MergeTreeVirtualColumns.h (100%) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6641c6b740c..3b1a499255b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 6253d3058aa..28160b18269 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 73caf70627f..cac172a856f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -135,7 +135,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) std::list candidates; NormalProjectionCandidate * best_candidate = nullptr; - const Names & required_columns = reading->getRealColumnNames(); + const Names & required_columns = reading->getAllColumnNames(); const auto & parts = reading->getParts(); const auto & alter_conversions = reading->getAlterConvertionsForParts(); const auto & query_info = reading->getQueryInfo(); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 1d7c2d46361..5ed742a9bfd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -133,7 +133,7 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeIndexes(JSONBuilder::JSONMap & map) const override; - const Names & getRealColumnNames() const { return all_column_names; } + const Names & getAllColumnNames() const { return all_column_names; } StorageID getStorageID() const { return data.getStorageID(); } UInt64 getSelectedParts() const { return selected_parts; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index f8d73038e09..1108eafc6b6 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -214,9 +214,9 @@ public: metadata.set(std::make_unique(metadata_)); } - void setVirtuals(const VirtualColumnsDescription & virtuals_) + void setVirtuals(VirtualColumnsDescription virtuals_) { - virtuals.set(std::make_unique(virtuals_)); + virtuals.set(std::make_unique(std::move(virtuals_))); } /// Return list of virtual columns (like _part, _table, etc). In the vast @@ -275,7 +275,7 @@ private: /// Multiversion storage metadata. Allows to read/write storage metadata without locks. MultiVersionStorageMetadataPtr metadata; - /// TODO: + /// Description of virtual columns. Optional, may be set in constructor. MultiVersionVirtualsDescriptionPtr virtuals; protected: diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index cef385c6f98..476b0a7e183 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -220,7 +220,7 @@ StorageLiveView::StorageLiveView( VirtualColumnsDescription virtuals; virtuals.addEphemeral("_version", std::make_shared(), ""); - setVirtuals(virtuals); + setVirtuals(std::move(virtuals)); if (!query.select) throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName()); diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 30be1aa1c56..4936f1d33c6 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -68,6 +68,8 @@ const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints() void IMergeTreeReader::fillVirtualColumns(Columns & columns, size_t rows) const { + chassert(columns.size() == requested_columns.size()); + const auto * loaded_part_info = typeid_cast(data_part_info_for_read.get()); if (!loaded_part_info) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Filling of virtual columns is supported only for LoadedMergeTreeDataPartInfoForReader"); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 32877cd88eb..a5b84eba241 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -117,7 +117,7 @@ private: /// Actual columns description in part. const ColumnsDescription & part_columns; - /// TODO: + /// Fields of virtual columns that were filled in previous stages. VirtualFields virtual_fields; }; diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 28a3c671914..1f50e55f8a0 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -15,7 +15,7 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index e84ed0a8068..67f5e7a53e8 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7b1d08642e2..d98e961f96b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -67,7 +67,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 059caebcfc8..c19b4ddd8a2 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index ff86ec01efa..c78f7579637 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1153,13 +1153,13 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t size_t pos = read_sample_block.getPositionByName("_part_offset"); chassert(pos < result.columns.size()); chassert(result.columns[pos] == nullptr); - result.columns[pos] = fillPartOffsetColumn(result, leading_begin_part_offset, leading_end_part_offset); + result.columns[pos] = createPartOffsetColumn(result, leading_begin_part_offset, leading_end_part_offset); } return result; } -ColumnPtr MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) +ColumnPtr MergeTreeRangeReader::createPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset) { size_t num_rows = result.numReadRows(); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index ef861fc5b4d..688a6b0922b 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -308,7 +308,7 @@ private: ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); Columns continueReadingChain(const ReadResult & result, size_t & num_rows); void executePrewhereActionsAndFilterColumns(ReadResult & result) const; - ColumnPtr fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); + ColumnPtr createPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset); IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; diff --git a/src/Storages/MergeTree/MergeTreeReadTask.cpp b/src/Storages/MergeTree/MergeTreeReadTask.cpp index 64fd37d14b1..08b30e445e2 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.cpp +++ b/src/Storages/MergeTree/MergeTreeReadTask.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 8a24e150bae..fce733d47b7 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 0d6b1d88075..e5545a92aea 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTreeVirtualColumns.cpp b/src/Storages/MergeTree/MergeTreeVirtualColumns.cpp similarity index 96% rename from src/Storages/MergeTreeVirtualColumns.cpp rename to src/Storages/MergeTree/MergeTreeVirtualColumns.cpp index 8250ceda7fa..b87dccc2b18 100644 --- a/src/Storages/MergeTreeVirtualColumns.cpp +++ b/src/Storages/MergeTree/MergeTreeVirtualColumns.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Storages/MergeTreeVirtualColumns.h b/src/Storages/MergeTree/MergeTreeVirtualColumns.h similarity index 100% rename from src/Storages/MergeTreeVirtualColumns.h rename to src/Storages/MergeTree/MergeTreeVirtualColumns.h diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 56f832630b7..a5b8a2a2a6d 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 69e6085bab9..4749303e4c2 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -333,7 +333,7 @@ StorageKeeperMap::StorageKeeperMap( VirtualColumnsDescription virtuals; virtuals.addEphemeral(String(version_column_name), std::make_shared(), ""); - setVirtuals(virtuals); + setVirtuals(std::move(virtuals)); WriteBufferFromOwnString out; out << "KeeperMap metadata format version: 1\n" diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 7d959b05c94..5e275357022 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -35,7 +35,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index e5c1d3d1dea..222173fd66b 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index 390a5037780..63e4cbd99f6 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -84,9 +84,6 @@ struct StorageSnapshot /// If we have a projection then we should use its metadata. StorageMetadataPtr getMetadataForQuery() const { return projection ? projection->metadata : metadata; } - -private: - void init(); }; using StorageSnapshotPtr = std::shared_ptr; diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 191cdab1a40..a930ffd1307 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -12,13 +12,13 @@ StorageValues::StorageValues( const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_, - const VirtualColumnsDescription & virtuals_) + VirtualColumnsDescription virtuals_) : IStorage(table_id_), res_block(res_block_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - setVirtuals(virtuals_); + setVirtuals(std::move(virtuals_)); } Pipe StorageValues::read( diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index 0db3ecec888..ebb182ab667 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -18,7 +18,7 @@ public: const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_, - const VirtualColumnsDescription & virtuals_ = {}); + VirtualColumnsDescription virtuals_ = {}); std::string getName() const override { return "Values"; } diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index d60cfcafc13..99cdc019fe5 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -56,7 +56,7 @@ StorageSystemDictionaries::StorageSystemDictionaries(const StorageID & storage_i { VirtualColumnsDescription virtuals; virtuals.addEphemeral("key", std::make_shared(), ""); - setVirtuals(virtuals); + setVirtuals(std::move(virtuals)); } ColumnsDescription StorageSystemDictionaries::getColumnsDescription() diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 075e1c62323..78a17b974e1 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -262,7 +262,7 @@ StorageSystemPartsBase::StorageSystemPartsBase(const StorageID & table_id_, Colu VirtualColumnsDescription virtuals; virtuals.addEphemeral("_state", std::make_shared(), ""); - setVirtuals(virtuals); + setVirtuals(std::move(virtuals)); } } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 21f05953714..9c462217007 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -352,17 +352,17 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription { VirtualColumnsDescription desc; - auto add_virtual = [&](const auto & name, const auto & type, const auto & comment) + auto add_virtual = [&](const auto & name, const auto & type) { if (storage_columns.has(name)) return; - desc.addEphemeral(name, type, comment); + desc.addEphemeral(name, type, ""); }; - add_virtual("_path", std::make_shared(std::make_shared()), ""); - add_virtual("_file", std::make_shared(std::make_shared()), ""); - add_virtual("_size", makeNullable(std::make_shared()), ""); + add_virtual("_path", std::make_shared(std::make_shared())); + add_virtual("_file", std::make_shared(std::make_shared())); + add_virtual("_size", makeNullable(std::make_shared())); return desc; } diff --git a/src/Storages/VirtualColumnsDescription.h b/src/Storages/VirtualColumnsDescription.h index 2f46bbcab82..e11ba706718 100644 --- a/src/Storages/VirtualColumnsDescription.h +++ b/src/Storages/VirtualColumnsDescription.h @@ -7,7 +7,6 @@ namespace DB struct VirtualColumnDescription : public ColumnDescription { -public: using Self = VirtualColumnDescription; VirtualsKind kind; @@ -16,6 +15,9 @@ public: bool isEphemeral() const { return kind == VirtualsKind::Ephemeral; } bool isPersistent() const { return kind == VirtualsKind::Persistent; } + + /// This method is needed for boost::multi_index because field + /// of base class cannot be referenced in boost::multi_index::member. const String & getName() const { return name; } }; From 8b2bd2125b58074c3b7169f531a56a64a5769dcd Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 15 Dec 2023 18:21:14 +0100 Subject: [PATCH 155/197] no warnings at drop --- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++- .../0_stateless/02922_deduplication_with_zero_copy.sh | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 36e9ee22f6d..73c7e484bcb 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1115,7 +1115,10 @@ void StorageReplicatedMergeTree::dropZookeeperZeroCopyLockPaths(zkutil::ZooKeepe { LOG_INFO(logger, "Zero copy locks directory {} is absent on ZooKeeper.", zero_copy_locks_root); } - chassert(code == Coordination::Error::ZOK); + else + { + chassert(code == Coordination::Error::ZOK); + } } } diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh index 381311c5033..bb013dccb65 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -137,8 +137,8 @@ function list_keeper_nodes() { list_keeper_nodes "${table_shared_id}" -$CLICKHOUSE_CLIENT -nm -q "drop table r1;" & -$CLICKHOUSE_CLIENT -nm -q "drop table r2;" & +$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --send_logs_level="error" & wait list_keeper_nodes "${table_shared_id}" From e1851987d8c715ad039d092e6b398b722eda6c5b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 5 Mar 2024 19:35:39 +0100 Subject: [PATCH 156/197] Make tidy happy --- src/Parsers/ASTQueryWithOutput.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index e47f4dcf29d..c57aa759969 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -78,7 +78,7 @@ bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) { if (p) { - if (auto it = std::find(ast_with_output->children.begin(), ast_with_output->children.end(), p); + if (auto * it = std::find(ast_with_output->children.begin(), ast_with_output->children.end(), p); it != ast_with_output->children.end()) ast_with_output->children.erase(it); p.reset(); From 4f6cb21d51695aa6dd8fd7278d308f93a4d7b164 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 5 Mar 2024 18:57:24 +0100 Subject: [PATCH 157/197] set priority when scheduling task in thread pool --- src/Interpreters/AsynchronousInsertQueue.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index e25cedb916c..7d56dbabe3c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -254,11 +254,17 @@ AsynchronousInsertQueue::~AsynchronousInsertQueue() void AsynchronousInsertQueue::scheduleDataProcessingJob( const InsertQuery & key, InsertDataPtr data, ContextPtr global_context, size_t shard_num) { + /// Intuitively it seems reasonable to process first inserted blocks first. + /// We add new chunks in the end of entries list, so they are automatically ordered by creation time + chassert(!data->entries.empty()); + const auto priority = Priority{data->entries.front()->create_time.time_since_epoch().count()}; + /// Wrap 'unique_ptr' with 'shared_ptr' to make this /// lambda copyable and allow to save it to the thread pool. pool.scheduleOrThrowOnError( [this, key, global_context, shard_num, my_data = std::make_shared(std::move(data))]() mutable - { processData(key, std::move(*my_data), std::move(global_context), flush_time_history_per_queue_shard[shard_num]); }); + { processData(key, std::move(*my_data), std::move(global_context), flush_time_history_per_queue_shard[shard_num]); }, + priority); } void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const ContextPtr & query_context) @@ -382,6 +388,7 @@ AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr assert(data); auto size_in_bytes = data->size_in_bytes; data->size_in_bytes += entry_data_size; + /// We rely on the fact that entries are being added to the list in order of creation time in `scheduleDataProcessingJob()` data->entries.emplace_back(entry); insert_future = entry->getFuture(); From ba8a2eb10d1a2a2beb0fd72b5191c3f4b2fdbd34 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Wed, 6 Mar 2024 09:25:01 +0800 Subject: [PATCH 158/197] Fix functional test analyzer_preimage_null ColumnNodes with the same source are expected to share the exact id after the preimage transformation. --- .../02999_analyzer_preimage_null.reference | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02999_analyzer_preimage_null.reference b/tests/queries/0_stateless/02999_analyzer_preimage_null.reference index 6f9afedfd07..c5eb6b23d0d 100644 --- a/tests/queries/0_stateless/02999_analyzer_preimage_null.reference +++ b/tests/queries/0_stateless/02999_analyzer_preimage_null.reference @@ -108,14 +108,14 @@ QUERY id: 0 FUNCTION id: 14, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 15, nodes: 2 - COLUMN id: 16, column_name: date1, result_type: Date, source_id: 3 - CONSTANT id: 17, constant_value: \'1994-01-01\', constant_value_type: String - FUNCTION id: 18, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 16, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 17, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 - FUNCTION id: 21, function_name: toYear, function_type: ordinary, result_type: UInt16 + LIST id: 18, nodes: 2 + COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 + FUNCTION id: 20, function_name: toYear, function_type: ordinary, result_type: UInt16 ARGUMENTS - LIST id: 22, nodes: 1 - COLUMN id: 23, column_name: date1, result_type: Date, source_id: 3 + LIST id: 21, nodes: 1 + COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 SETTINGS optimize_time_filter_with_preimage=1 From cd2a418afd088563819ed723d7a1ae91110cb895 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 6 Mar 2024 02:31:21 +0000 Subject: [PATCH 159/197] add named collection grant doc --- docs/en/sql-reference/statements/grant.md | 24 +++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 4e5476210e3..879354d714b 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -202,6 +202,13 @@ Hierarchy of privileges: - `S3` - [dictGet](#grant-dictget) - [displaySecretsInShowAndSelect](#grant-display-secrets) +- [NAMED COLLECTION ADMIN](#grant-named-collection-admin) + - `CREATE NAMED COLLECTION` + - `DROP NAMED COLLECTION` + - `ALTER NAMED COLLECTION` + - `SHOW NAMED COLLECTIONS` + - `SHOW NAMED COLLECTIONS SECRETS` + - `NAMED COLLECTION` Examples of how this hierarchy is treated: @@ -498,6 +505,23 @@ and [`format_display_secrets_in_show_and_select` format setting](../../operations/settings/formats#format_display_secrets_in_show_and_select) are turned on. +### NAMED COLLECTION ADMIN + +Allows a certain operation on a specified named colleciton. + +- `NAMED COLLECTION ADMIN`. Level: `NAMED_COLLECTION`. Aliases: `NAMED COLLECTION CONTROL` + - `CREATE NAMED COLLECTION`. Level: `NAMED_COLLECTION` + - `DROP NAMED COLLECTION`. Level: `NAMED_COLLECTION` + - `ALTER NAMED COLLECTION`. Level: `NAMED_COLLECTION` + - `SHOW NAMED COLLECTIONS`. Level: `NAMED_COLLECTION`. Aliases: `SHOW NAMED COLLECTIONS` + - `SHOW NAMED COLLECTIONS SECRETS`. Level: `NAMED_COLLECTION`. Aliases: `SHOW NAMED COLLECTIONS SECRETS` + - `NAMED COLLECTION`. Level: `NAMED_COLLECTION`. Aliases: `NAMED COLLECTION USAGE, USE NAMED COLLECTION` + +**Examples** + +Assuming a named collecion is called abc, we grant privilege CREATE NAMED COLLECTION to user john. +- `GRANT CREATE NAMED COLLECTION ON abc TO john` + ### ALL Grants all the privileges on regulated entity to a user account or a role. From e4b78e7dac963e9fa8d10fcd9956f8df00b3a933 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 6 Mar 2024 15:10:45 +0800 Subject: [PATCH 160/197] fix bugs --- src/Functions/multiIf.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index f5cb2375c53..81304f3afbd 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -422,7 +422,7 @@ private: res_data.resize_exact(rows); if constexpr (nullable_result) { - if (res_null_map) + if (!res_null_map) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid result null_map while result type is nullable"); res_null_map->resize_exact(rows); From 655f9cb1953c371d7b9a9ba6a58fb4a4fc3ca40e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 6 Mar 2024 08:57:20 +0100 Subject: [PATCH 161/197] Revert "Fix wacky primary key sorting in `SHOW INDEX`" --- .../InterpreterShowIndexesQuery.cpp | 4 ++-- .../0_stateless/02724_show_indexes.reference | 24 +++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 5be72dc8ce6..e8005ead91e 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -37,7 +37,7 @@ FROM ( name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - row_number() over (order by null) AS seq_in_index, + row_number() over (order by column_name) AS seq_in_index, arrayJoin(splitByString(', ', primary_key)) AS column_name, 'A' AS collation, 0 AS cardinality, @@ -75,7 +75,7 @@ FROM ( database = '{0}' AND table = '{1}')) {2} -ORDER BY index_type, expression, seq_in_index;)", database, table, where_expression); +ORDER BY index_type, expression, column_name, seq_in_index;)", database, table, where_expression); /// Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to /// sort the output of SHOW INDEXES otherwise (SELECT * FROM (SHOW INDEXES ...) ORDER BY ...) is rejected) and 3. some diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index ac0461fc506..e41f2521f5c 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -2,33 +2,33 @@ tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- EXTENDED tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- WHERE --- Check with weird table names @@ -40,8 +40,8 @@ NULL 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- Equally named table in other database tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b From 06c2492b92ed89c62ae0e3cd3d11478b1aa0f75e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Mar 2024 09:29:16 +0000 Subject: [PATCH 162/197] Clarify subtle detail in docs --- docs/en/sql-reference/statements/alter/column.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 902eae2d67d..0989c151d18 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -278,9 +278,9 @@ Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mut For columns with a new or updated `MATERIALIZED` value expression, all existing rows are rewritten. -For columns with a new or updated `DEFAULT` value expression, the behavior changed in ClickHouse v24.2: -- In ClickHouse < v24.2, all existing rows are changed to the new `DEFAULT` value expression. -- In ClickHouse >= v24.2, only rows containing the previous default value are changed to the new `DEFAULT` value expression. Rows with non-default values are kept as is. +For columns with a new or updated `DEFAULT` value expression, the behavior depends on the ClickHouse version: +- In ClickHouse < v24.2, all existing rows are rewritten. +- ClickHouse >= v24.2 distinguishes if a row value in a column with `DEFAULT` value expression was explicitly specified when it was inserted, or not, i.e. calculated from the `DEFAULT` value expression. If the value was explicitly specified, ClickHouse keeps it as is. If the value was was calculated, ClickHouse changes it to the new or updated `MATERIALIZED` value expression. Syntax: From 93e2ed824816ba5215d3ff5c12c72d9726538a1a Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 6 Mar 2024 11:31:24 +0100 Subject: [PATCH 163/197] Remove useless message from AWS library --- contrib/aws | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index 5f0542b3ad7..32870e234ca 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 5f0542b3ad7eef25b0540d37d778207e0345ea8f +Subproject commit 32870e234cac03e0ac46370c26858b0ffdf14200 From 083a251951e60b8ca4c4d3e12f78a12cff330237 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Mar 2024 12:04:13 +0000 Subject: [PATCH 164/197] Fixes for HashedArrayDictionary --- src/Dictionaries/HashedArrayDictionary.cpp | 15 ++++++++++-- .../HashedDictionaryParallelLoader.h | 24 ++++++++++++++++--- 2 files changed, 34 insertions(+), 5 deletions(-) diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 56d42a38c8e..eae00c297b1 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -1078,7 +1078,7 @@ void HashedArrayDictionary::calculateBytesAllocate bytes_allocated += container.allocated_bytes(); } - bucket_count = container.capacity(); + bucket_count += container.capacity(); } }; @@ -1089,6 +1089,13 @@ void HashedArrayDictionary::calculateBytesAllocate bytes_allocated += container.size(); } + /// `bucket_count` should be a sum over all shards, + /// but it should not be a sum over all attributes, since it is used to + /// calculate load_factor like this: `element_count / bucket_count` + /// While element_count is a sum over all shards, not over all attributes. + if (attributes.size()) + bucket_count /= attributes.size(); + if (update_field_loaded_block) bytes_allocated += update_field_loaded_block->allocatedBytes(); @@ -1167,11 +1174,15 @@ void registerDictionaryArrayHashed(DictionaryFactory & factory) if (shards <= 0 || 128 < shards) throw Exception(ErrorCodes::BAD_ARGUMENTS,"{}: SHARDS parameter should be within [1, 128]", full_name); - HashedArrayDictionaryStorageConfiguration configuration{require_nonempty, dict_lifetime, static_cast(shards)}; + Int64 shard_load_queue_backlog = config.getInt(config_prefix + dictionary_layout_prefix + ".shard_load_queue_backlog", 10000); + if (shard_load_queue_backlog <= 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: SHARD_LOAD_QUEUE_BACKLOG parameter should be greater then zero", full_name); if (source_ptr->hasUpdateField() && shards > 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: SHARDS parameter does not supports for updatable source (UPDATE_FIELD)", full_name); + HashedArrayDictionaryStorageConfiguration configuration{require_nonempty, dict_lifetime, static_cast(shards), static_cast(shard_load_queue_backlog)}; + ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); const auto & settings = context->getSettingsRef(); diff --git a/src/Dictionaries/HashedDictionaryParallelLoader.h b/src/Dictionaries/HashedDictionaryParallelLoader.h index a256f6de0e0..1b8b7b7f555 100644 --- a/src/Dictionaries/HashedDictionaryParallelLoader.h +++ b/src/Dictionaries/HashedDictionaryParallelLoader.h @@ -62,7 +62,11 @@ public: shards_queues[shard].emplace(backlog); pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()] { + WorkerStatistic statistic; SCOPE_EXIT_SAFE( + LOG_TRACE(dictionary.log, "Finished worker for dictionary {} shard {}, processed {} blocks, {} rows, total time {}ms", + dictionary_name, shard, statistic.total_blocks, statistic.total_rows, statistic.total_elapsed_ms); + if (thread_group) CurrentThread::detachFromGroupIfNotDetached(); ); @@ -74,7 +78,9 @@ public: CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictLoad"); - threadWorker(shard); + LOG_TRACE(dictionary.log, "Starting worker for dictionary {}, shard {}", dictionary_name, shard); + + threadWorker(shard, statistic); }); } } @@ -128,7 +134,14 @@ private: std::vector shards_slots; DictionaryKeysArenaHolder arena_holder; - void threadWorker(size_t shard) + struct WorkerStatistic + { + UInt64 total_elapsed_ms = 0; + UInt64 total_blocks = 0; + UInt64 total_rows = 0; + }; + + void threadWorker(size_t shard, WorkerStatistic & statistic) { Block block; DictionaryKeysArenaHolder arena_holder_; @@ -139,8 +152,13 @@ private: Stopwatch watch; dictionary.blockToAttributes(block, arena_holder_, shard); UInt64 elapsed_ms = watch.elapsedMilliseconds(); + + statistic.total_elapsed_ms += elapsed_ms; + statistic.total_blocks += 1; + statistic.total_rows += block.rows(); + if (elapsed_ms > 1'000) - LOG_TRACE(dictionary.log, "Block processing for shard #{} is slow {}ms (rows {}).", shard, elapsed_ms, block.rows()); + LOG_TRACE(dictionary.log, "Block processing for shard #{} is slow {}ms (rows {})", shard, elapsed_ms, block.rows()); } if (!shard_queue.isFinished()) From 70750cb10882ff0eaf15da52baa94603cdffc03b Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Mar 2024 12:04:57 +0000 Subject: [PATCH 165/197] Fix possible stuck on error in HashedDictionaryParallelLoader --- .../HashedDictionaryParallelLoader.h | 32 +++++++++++++++---- 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/src/Dictionaries/HashedDictionaryParallelLoader.h b/src/Dictionaries/HashedDictionaryParallelLoader.h index 1b8b7b7f555..c0b4aa73adb 100644 --- a/src/Dictionaries/HashedDictionaryParallelLoader.h +++ b/src/Dictionaries/HashedDictionaryParallelLoader.h @@ -93,8 +93,21 @@ public: for (size_t shard = 0; shard < shards; ++shard) { - if (!shards_queues[shard]->push(std::move(shards_blocks[shard]))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to shards queue #{}", shard); + const auto & current_block = shards_blocks[shard]; + while (!shards_queues[shard]->tryPush(current_block, /* milliseconds= */ 100)) + { + if (shards_queues[shard]->isFinished()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to finished shards queue #{}, dictionary {}", shard, dictionary_name); + + /// We need to check if some workers failed + if (pool.active() != shards) + { + LOG_DEBUG(dictionary.log, "Some workers for dictionary {} failed, stopping all workers", dictionary_name); + stop_all_workers = true; + pool.wait(); /// We expect exception to be thrown from the failed worker thread + throw Exception(ErrorCodes::LOGICAL_ERROR, "Worker threads for dictionary {} are not active", dictionary_name); + } + } } } @@ -130,6 +143,7 @@ private: String dictionary_name; const size_t shards; ThreadPool pool; + std::atomic_bool stop_all_workers{false}; std::vector>> shards_queues; std::vector shards_slots; DictionaryKeysArenaHolder arena_holder; @@ -147,8 +161,17 @@ private: DictionaryKeysArenaHolder arena_holder_; auto & shard_queue = *shards_queues[shard]; - while (shard_queue.pop(block)) + while (true) { + if (!shard_queue.tryPop(block, /* milliseconds= */ 100)) + { + /// Check if we need to stop + if (stop_all_workers || shard_queue.isFinished()) + break; + /// Timeout expired, but the queue is not finished yet, try again + continue; + } + Stopwatch watch; dictionary.blockToAttributes(block, arena_holder_, shard); UInt64 elapsed_ms = watch.elapsedMilliseconds(); @@ -160,9 +183,6 @@ private: if (elapsed_ms > 1'000) LOG_TRACE(dictionary.log, "Block processing for shard #{} is slow {}ms (rows {})", shard, elapsed_ms, block.rows()); } - - if (!shard_queue.isFinished()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pull non finished shards queue #{}", shard); } /// Split block to shards smaller block, using 'selector'. From d9f7ac881593dc577b7da864bdabc482337ea378 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 1 Mar 2024 15:41:42 +0000 Subject: [PATCH 166/197] Analyzer: Fix bug with join_use_nulls and PREWHERE --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 23 ++++++++++++++++--- ...ence => 02534_join_prewhere_bug.reference} | 7 ++++++ ..._44062.sql => 02534_join_prewhere_bug.sql} | 5 ++++ 3 files changed, 32 insertions(+), 3 deletions(-) rename tests/queries/0_stateless/{02534_join_prewhere_bug_44062.reference => 02534_join_prewhere_bug.reference} (88%) rename tests/queries/0_stateless/{02534_join_prewhere_bug_44062.sql => 02534_join_prewhere_bug.sql} (92%) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 907a732493d..eb578d17c04 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -718,6 +718,8 @@ struct IdentifierResolveScope group_by_use_nulls = context->getSettingsRef().group_by_use_nulls && (query_node->isGroupByWithGroupingSets() || query_node->isGroupByWithRollup() || query_node->isGroupByWithCube()); } + + join_use_nulls = context->getSettingsRef().join_use_nulls; } QueryTreeNodePtr scope_node; @@ -772,6 +774,8 @@ struct IdentifierResolveScope /// Apply nullability to aggregation keys bool group_by_use_nulls = false; + /// Join retutns NULLs instead of default values + bool join_use_nulls = false; /// JOINs count size_t joins_count = 0; @@ -3286,7 +3290,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo QueryTreeNodePtr resolved_identifier; JoinKind join_kind = from_join_node.getKind(); - bool join_use_nulls = scope.context->getSettingsRef().join_use_nulls; /// If columns from left or right table were missed Object(Nullable('json')) subcolumns, they will be replaced /// to ConstantNode(NULL), which can't be cast to ColumnNode, so we resolve it here. @@ -3451,7 +3454,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo if (join_node_in_resolve_process || !resolved_identifier) return resolved_identifier; - if (join_use_nulls) + if (scope.join_use_nulls) { resolved_identifier = resolved_identifier->clone(); convertJoinedColumnTypeToNullIfNeeded(resolved_identifier, join_kind, resolved_side); @@ -4439,7 +4442,7 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I else matched_expression_nodes_with_names = resolveUnqualifiedMatcher(matcher_node, scope); - if (scope.context->getSettingsRef().join_use_nulls) + if (scope.join_use_nulls) { /** If we are resolving matcher came from the result of JOIN and `join_use_nulls` is set, * we need to convert joined column type to Nullable. @@ -7558,8 +7561,22 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier } if (query_node_typed.getPrewhere()) + { + /** Expression in PREWHERE with JOIN should not be modified by join_use_nulls. + * Example: SELECT * FROM t1 JOIN t2 USING (id) PREWHERE a = 1 + * Column `a` should be resolved from table and should not change its type to Nullable. + */ + bool join_use_nulls = scope.join_use_nulls; + bool use_identifier_lookup_to_result_cache = scope.use_identifier_lookup_to_result_cache; + scope.join_use_nulls = false; + scope.use_identifier_lookup_to_result_cache = false; + resolveExpressionNode(query_node_typed.getPrewhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + scope.join_use_nulls = join_use_nulls; + scope.use_identifier_lookup_to_result_cache = use_identifier_lookup_to_result_cache; + } + if (query_node_typed.getWhere()) resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); diff --git a/tests/queries/0_stateless/02534_join_prewhere_bug_44062.reference b/tests/queries/0_stateless/02534_join_prewhere_bug.reference similarity index 88% rename from tests/queries/0_stateless/02534_join_prewhere_bug_44062.reference rename to tests/queries/0_stateless/02534_join_prewhere_bug.reference index aaef17371d8..115ea994de1 100644 --- a/tests/queries/0_stateless/02534_join_prewhere_bug_44062.reference +++ b/tests/queries/0_stateless/02534_join_prewhere_bug.reference @@ -34,5 +34,12 @@ ORDER BY test2.col1 ; 5600 123 123 5601 321 -32 +SELECT col2, col2 + 1 FROM test1 +FULL OUTER JOIN test2 USING (col1) +PREWHERE (col2 * 2) :: UInt8 +; +123 124 +-32 -31 +-30 -29 DROP TABLE IF EXISTS test1; DROP TABLE IF EXISTS test2; diff --git a/tests/queries/0_stateless/02534_join_prewhere_bug_44062.sql b/tests/queries/0_stateless/02534_join_prewhere_bug.sql similarity index 92% rename from tests/queries/0_stateless/02534_join_prewhere_bug_44062.sql rename to tests/queries/0_stateless/02534_join_prewhere_bug.sql index 073f81e4ff3..016c92597ec 100644 --- a/tests/queries/0_stateless/02534_join_prewhere_bug_44062.sql +++ b/tests/queries/0_stateless/02534_join_prewhere_bug.sql @@ -42,5 +42,10 @@ WHERE test2.col1 IS NOT NULL ORDER BY test2.col1 ; +SELECT col2, col2 + 1 FROM test1 +FULL OUTER JOIN test2 USING (col1) +PREWHERE (col2 * 2) :: UInt8 +; + DROP TABLE IF EXISTS test1; DROP TABLE IF EXISTS test2; From 4dede601efe1fd651f4d4cd311fb52895890f936 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Mar 2024 12:08:03 +0000 Subject: [PATCH 167/197] fix --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index eb578d17c04..576a0e68966 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -719,7 +719,10 @@ struct IdentifierResolveScope (query_node->isGroupByWithGroupingSets() || query_node->isGroupByWithRollup() || query_node->isGroupByWithCube()); } - join_use_nulls = context->getSettingsRef().join_use_nulls; + if (context) + join_use_nulls = context->getSettingsRef().join_use_nulls; + else if (parent_scope) + join_use_nulls = parent_scope->join_use_nulls; } QueryTreeNodePtr scope_node; From 381c0753915f43d33b7f3105eb2a5a1072350d0b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 6 Mar 2024 11:48:33 +0100 Subject: [PATCH 168/197] Throw on query timeout --- src/Common/ZooKeeper/ZooKeeperRetries.h | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperRetries.h b/src/Common/ZooKeeper/ZooKeeperRetries.h index d411549346a..d8d234ba913 100644 --- a/src/Common/ZooKeeper/ZooKeeperRetries.h +++ b/src/Common/ZooKeeper/ZooKeeperRetries.h @@ -5,8 +5,6 @@ #include #include -#include - namespace DB { @@ -220,8 +218,8 @@ private: return false; } - if (process_list_element && !process_list_element->checkTimeLimitSoft()) - return false; + if (process_list_element) + process_list_element->checkTimeLimit(); /// retries logLastError("will retry due to error"); From 314e27a06ebf5df04b211f92e1eaaa34d5b5d9f6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Mar 2024 12:52:03 +0000 Subject: [PATCH 169/197] Fix groupArraySorted documentation --- .../reference/grouparraysorted.md | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/grouparraysorted.md b/docs/en/sql-reference/aggregate-functions/reference/grouparraysorted.md index cc601c097fe..9bee0c29e7a 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/grouparraysorted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/grouparraysorted.md @@ -14,8 +14,6 @@ - `N` – The number of elements to return. - If the parameter is omitted, default value is the size of input. - - `column` – The value (Integer, String, Float and other Generic types). **Example** @@ -36,13 +34,12 @@ Gets all the String implementations of all numbers in column: ``` sql -SELECT groupArraySorted(str) FROM (SELECT toString(number) as str FROM numbers(5)); +SELECT groupArraySorted(5)(str) FROM (SELECT toString(number) as str FROM numbers(5)); ``` ``` text - ┌─groupArraySorted(str)────────┠- │ ['0','1','2','3','4'] │ - └──────────────────────────────┘ - ``` - \ No newline at end of file +┌─groupArraySorted(5)(str)─┠+│ ['0','1','2','3','4'] │ +└──────────────────────────┘ + ``` \ No newline at end of file From 7e908711d0ff0895a5fbe13c28077f0a3042d49c Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Mar 2024 12:54:39 +0000 Subject: [PATCH 170/197] Respect max_execution_time for dictionary reloading time --- src/Dictionaries/HashedArrayDictionary.cpp | 3 +++ src/Dictionaries/HashedArrayDictionary.h | 1 + src/Dictionaries/HashedDictionary.h | 1 + src/Dictionaries/HashedDictionaryParallelLoader.h | 14 +++++++++++++- src/Dictionaries/registerHashedDictionary.cpp | 1 + 5 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index eae00c297b1..d09f402143e 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -1189,6 +1189,9 @@ void registerDictionaryArrayHashed(DictionaryFactory & factory) const auto * clickhouse_source = dynamic_cast(source_ptr.get()); configuration.use_async_executor = clickhouse_source && clickhouse_source->isLocal() && settings.dictionary_use_async_executor; + if (settings.max_execution_time.totalSeconds() > 0) + configuration.load_timeout = std::chrono::seconds(settings.max_execution_time.totalSeconds()); + if (dictionary_key_type == DictionaryKeyType::Simple) { if (shards > 1) diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h index 4b2570ad928..9877d92d457 100644 --- a/src/Dictionaries/HashedArrayDictionary.h +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -29,6 +29,7 @@ struct HashedArrayDictionaryStorageConfiguration size_t shards = 1; size_t shard_load_queue_backlog = 10000; bool use_async_executor = false; + std::chrono::seconds load_timeout{0}; }; template diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 3a5e4ff6306..b3b8cc56868 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -67,6 +67,7 @@ struct HashedDictionaryConfiguration const bool require_nonempty; const DictionaryLifetime lifetime; bool use_async_executor = false; + const std::chrono::seconds load_timeout{0}; }; template diff --git a/src/Dictionaries/HashedDictionaryParallelLoader.h b/src/Dictionaries/HashedDictionaryParallelLoader.h index c0b4aa73adb..d88ee88f9a9 100644 --- a/src/Dictionaries/HashedDictionaryParallelLoader.h +++ b/src/Dictionaries/HashedDictionaryParallelLoader.h @@ -31,6 +31,7 @@ template clas namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int TIMEOUT_EXCEEDED; } } @@ -50,9 +51,10 @@ public: , shards(dictionary.configuration.shards) , pool(CurrentMetrics::HashedDictionaryThreads, CurrentMetrics::HashedDictionaryThreadsActive, CurrentMetrics::HashedDictionaryThreadsScheduled, shards) , shards_queues(shards) + , loading_timeout(dictionary.configuration.load_timeout) { UInt64 backlog = dictionary.configuration.shard_load_queue_backlog; - LOG_TRACE(dictionary.log, "Will load the {} dictionary using {} threads (with {} backlog)", dictionary_name, shards, backlog); + LOG_TRACE(dictionary.log, "Will load the {} dictionary using {} threads (with {} backlog and timeout {} sec)", dictionary_name, shards, backlog, loading_timeout.count()); shards_slots.resize(shards); iota(shards_slots.data(), shards_slots.size(), UInt64(0)); @@ -107,6 +109,13 @@ public: pool.wait(); /// We expect exception to be thrown from the failed worker thread throw Exception(ErrorCodes::LOGICAL_ERROR, "Worker threads for dictionary {} are not active", dictionary_name); } + + if (loading_timeout.count() && std::chrono::milliseconds(total_loading_time.elapsedMilliseconds()) > loading_timeout) + { + stop_all_workers = true; + pool.wait(); + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout {} sec for dictionary {} loading is expired", loading_timeout.count(), dictionary_name); + } } } } @@ -145,6 +154,9 @@ private: ThreadPool pool; std::atomic_bool stop_all_workers{false}; std::vector>> shards_queues; + std::chrono::seconds loading_timeout; + Stopwatch total_loading_time; + std::vector shards_slots; DictionaryKeysArenaHolder arena_holder; diff --git a/src/Dictionaries/registerHashedDictionary.cpp b/src/Dictionaries/registerHashedDictionary.cpp index 6b980e2d534..5fc4f5d5cb6 100644 --- a/src/Dictionaries/registerHashedDictionary.cpp +++ b/src/Dictionaries/registerHashedDictionary.cpp @@ -77,6 +77,7 @@ void registerDictionaryHashed(DictionaryFactory & factory) require_nonempty, dict_lifetime, use_async_executor, + std::chrono::seconds(settings.max_execution_time.totalSeconds()), }; if (source_ptr->hasUpdateField() && shards > 1) From ee468b0f2ae6e30b01704aebbfed440c8bec6f11 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 6 Mar 2024 14:26:23 +0000 Subject: [PATCH 171/197] CI: wait less in await to not exceed GH timeout #do_not_test --- tests/ci/ci.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 1bd607696de..b222e81ad73 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -645,7 +645,8 @@ class CiCache: if not jobs_with_params: return {} poll_interval_sec = 300 - TIMEOUT = 3590 + # TIMEOUT * MAX_ROUNDS_TO_WAIT must be less than 6h (GH job timeout) with a room for rest RunConfig work + TIMEOUT = 3000 # 50 min MAX_ROUNDS_TO_WAIT = 6 MAX_JOB_NUM_TO_WAIT = 3 await_finished: Dict[str, List[int]] = {} From 1c0aed896cde47e409ce254d2b46fa050f738655 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 6 Mar 2024 15:25:06 +0000 Subject: [PATCH 172/197] fix build --- src/Storages/System/IStorageSystemOneBlock.h | 2 +- .../System/StorageSystemDictionaries.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 49 ------------------- 3 files changed, 2 insertions(+), 51 deletions(-) diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 8123143a345..a20434fd97e 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -1,4 +1,4 @@ -// #pragma once +#pragma once #include diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index b33d8f4fa2e..7b733c872c4 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -52,7 +52,7 @@ catch (const DB::Exception &) } StorageSystemDictionaries::StorageSystemDictionaries(const StorageID & storage_id_) - : DB::IStorageSystemOneBlock(storage_id_) + : IStorageSystemOneBlock(storage_id_, getColumnsDescription()) { VirtualColumnsDescription virtuals; virtuals.addEphemeral("key", std::make_shared(), ""); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index fdafbd61336..22466b3d4c2 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -109,55 +109,6 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) } } -void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast) -{ - if (block.rows() == 0) - return; - - if (!expression_ast) - prepareFilterBlockWithQuery(query, context, block, expression_ast); - - if (!expression_ast) - return; - - /// Let's analyze and calculate the prepared expression. - auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); - ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); - - makeSets(actions, context); - - Block block_with_filter = block; - actions->execute(block_with_filter); - - /// Filter the block. - String filter_column_name = expression_ast->getColumnName(); - ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullIfNeeded(); - if (filter_column->getDataType() != TypeIndex::UInt8) - return; - - ConstantFilterDescription constant_filter(*filter_column); - - if (constant_filter.always_true) - { - return; - } - - if (constant_filter.always_false) - { - block = block.cloneEmpty(); - return; - } - - FilterDescription filter(*filter_column); - - for (size_t i = 0; i < block.columns(); ++i) - { - ColumnPtr & column = block.safeGetByPosition(i).column; - column = column->filter(*filter.data, -1); - } -} - NameSet getVirtualNamesForFileLikeStorage() { return {"_path", "_file", "_size"}; From d6325e267c51756dd8f4b98638e89ab05f0fd007 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Mar 2024 16:43:53 +0100 Subject: [PATCH 173/197] Use pyproject.toml for pylynt configuration --- .pylintrc | 43 ----------------------------------- tests/sqllogic/connection.py | 5 +++- utils/check-style/check-style | 2 +- 3 files changed, 5 insertions(+), 45 deletions(-) delete mode 100644 .pylintrc diff --git a/.pylintrc b/.pylintrc deleted file mode 100644 index b672cbfdfad..00000000000 --- a/.pylintrc +++ /dev/null @@ -1,43 +0,0 @@ -# vim: ft=config - -[BASIC] -max-module-lines=2000 -# due to SQL -max-line-length=200 -# Drop/decrease them one day: -max-branches=50 -max-nested-blocks=10 -max-statements=200 - -[FORMAT] -ignore-long-lines = (# )??$ - -[MESSAGES CONTROL] -disable = missing-docstring, - too-few-public-methods, - invalid-name, - too-many-arguments, - keyword-arg-before-vararg, - too-many-locals, - too-many-instance-attributes, - cell-var-from-loop, - fixme, - too-many-public-methods, - wildcard-import, - unused-wildcard-import, - singleton-comparison, - # pytest.mark.parametrize is not callable (not-callable) - not-callable, - # https://github.com/PyCQA/pylint/issues/3882 - # [Python 3.9] Value 'Optional' is unsubscriptable (unsubscriptable-object) (also Union) - unsubscriptable-object, - # Drop them one day: - redefined-outer-name, - broad-except, - bare-except, - no-else-return, - global-statement - -[SIMILARITIES] -# due to SQL -min-similarity-lines=1000 diff --git a/tests/sqllogic/connection.py b/tests/sqllogic/connection.py index 8bbb76d83df..169e0f0f440 100644 --- a/tests/sqllogic/connection.py +++ b/tests/sqllogic/connection.py @@ -59,7 +59,10 @@ def default_clickhouse_odbc_conn_str(): OdbcConnectingArgs.create_from_kw( dsn="ClickHouse DSN (ANSI)", Timeout="300", - Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1&create_index_ignore_unique=1", + Url="http://localhost:8123/query?default_format=ODBCDriver2&" + "default_table_engine=MergeTree&union_default_mode=DISTINCT&" + "group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1&" + "create_index_ignore_unique=1", ) ) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index d1fb2d81a28..3a5d0c053ea 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -155,7 +155,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.xml' | function xargs-pylint { # $1 is number maximum arguments per pylint process sort | awk '$2=="text/x-script.python" {print $1}' | \ - xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/.pylintrc" --persistent=no --score=n + xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/pyproject.toml" --persistent=no --score=n } find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 50 From fb1a684d98ab33d9260b236fb28c50372571030c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 6 Mar 2024 08:46:15 -0800 Subject: [PATCH 174/197] Fix race in PageCache (#60878) * Fix race in PageCache * Transportation security administration * Fix test flakiness, remove accidentally left over debug logging --- src/Common/PageCache.cpp | 42 ++++++++++--------- src/Common/PageCache.h | 20 ++++----- src/IO/CachedInMemoryReadBufferFromFile.cpp | 2 - src/IO/ReadSettings.h | 1 - .../0_stateless/02867_page_cache.reference | 16 ++++--- .../queries/0_stateless/02867_page_cache.sql | 13 +++--- 6 files changed, 47 insertions(+), 47 deletions(-) diff --git a/src/Common/PageCache.cpp b/src/Common/PageCache.cpp index 511ec23d431..d4598d4683b 100644 --- a/src/Common/PageCache.cpp +++ b/src/Common/PageCache.cpp @@ -191,7 +191,7 @@ size_t PageCache::maxChunks() const { return chunks_per_mmap_target * max_mmaps; size_t PageCache::getPinnedSize() const { - std::unique_lock lock(global_mutex); + std::lock_guard lock(global_mutex); return (total_chunks - lru.size()) * bytes_per_page * pages_per_chunk; } @@ -202,8 +202,11 @@ PageCache::MemoryStats PageCache::getResidentSetSize() const if (use_madv_free) { std::unordered_set cache_mmap_addrs; - for (const auto & m : mmaps) - cache_mmap_addrs.insert(reinterpret_cast(m.ptr)); + { + std::lock_guard lock(global_mutex); + for (const auto & m : mmaps) + cache_mmap_addrs.insert(reinterpret_cast(m.ptr)); + } ReadBufferFromFile in("/proc/self/smaps"); @@ -283,6 +286,7 @@ PageCache::MemoryStats PageCache::getResidentSetSize() const } #endif + std::lock_guard lock(global_mutex); stats.page_cache_rss = bytes_per_page * pages_per_chunk * total_chunks; return stats; } @@ -294,12 +298,12 @@ PinnedPageChunk PageCache::getOrSet(PageCacheKey key, bool detached_if_missing, bool incremented_profile_events = false; { - std::unique_lock lock(global_mutex); + std::lock_guard lock(global_mutex); auto * it = chunk_by_key.find(key); if (it == chunk_by_key.end()) { - chunk = getFreeChunk(lock); + chunk = getFreeChunk(); chassert(!chunk->key.has_value()); if (!detached_if_missing) @@ -331,14 +335,14 @@ PinnedPageChunk PageCache::getOrSet(PageCacheKey key, bool detached_if_missing, /// otherwise we may detach a chunk pinned by someone else, which may be unexpected /// for that someone else. Or maybe the latter is fine, dropCache() already does it.) if (chunk->pages_populated.get(0) && reinterpret_cast*>(chunk->data)->load(std::memory_order_relaxed) == 0) - evictChunk(chunk, lock); + evictChunk(chunk); } if (inject_eviction && chunk->key.has_value() && rng() % 10 == 0) { /// Simulate eviction of the chunk or some of its pages. if (rng() % 2 == 0) - evictChunk(chunk, lock); + evictChunk(chunk); else for (size_t i = 0; i < 20; ++i) chunk->pages_populated.unset(rng() % (chunk->size / chunk->page_size)); @@ -353,7 +357,7 @@ PinnedPageChunk PageCache::getOrSet(PageCacheKey key, bool detached_if_missing, } { - std::unique_lock chunk_lock(chunk->chunk_mutex); + std::lock_guard chunk_lock(chunk->chunk_mutex); if (chunk->pages_state == PageChunkState::Limbo) { @@ -383,7 +387,7 @@ void PageCache::removeRef(PageChunk * chunk) noexcept return; { - std::unique_lock lock(global_mutex); + std::lock_guard lock(global_mutex); prev_pin_count = chunk->pin_count.fetch_sub(1); if (prev_pin_count > 1) @@ -398,7 +402,7 @@ void PageCache::removeRef(PageChunk * chunk) noexcept } { - std::unique_lock chunk_lock(chunk->chunk_mutex); + std::lock_guard chunk_lock(chunk->chunk_mutex); /// Need to be extra careful here because we unlocked global_mutex above, so other /// getOrSet()/removeRef() calls could have happened during this brief period. @@ -421,7 +425,7 @@ static void logUnexpectedSyscallError(std::string name) #endif } -void PageCache::sendChunkToLimbo(PageChunk * chunk [[maybe_unused]], std::unique_lock & /* chunk_mutex */) const noexcept +void PageCache::sendChunkToLimbo(PageChunk * chunk [[maybe_unused]], std::lock_guard & /* chunk_mutex */) const noexcept { #ifdef MADV_FREE // if we're not on a very old version of Linux chassert(chunk->size == bytes_per_page * pages_per_chunk); @@ -454,7 +458,7 @@ void PageCache::sendChunkToLimbo(PageChunk * chunk [[maybe_unused]], std::unique #endif } -std::pair PageCache::restoreChunkFromLimbo(PageChunk * chunk, std::unique_lock & /* chunk_mutex */) const noexcept +std::pair PageCache::restoreChunkFromLimbo(PageChunk * chunk, std::lock_guard & /* chunk_mutex */) const noexcept { static_assert(sizeof(std::atomic) == 1, "char is not atomic?"); // Make sure our strategic memory reads/writes are not reordered or optimized out. @@ -505,10 +509,10 @@ std::pair PageCache::restoreChunkFromLimbo(PageChunk * chunk, st return {pages_restored, pages_evicted}; } -PageChunk * PageCache::getFreeChunk(std::unique_lock & lock /* global_mutex */) +PageChunk * PageCache::getFreeChunk() { if (lru.empty() || (mmaps.size() < max_mmaps && lru.front().key.has_value())) - addMmap(lock); + addMmap(); if (lru.empty()) throw Exception(ErrorCodes::MEMORY_LIMIT_EXCEEDED, "All chunks in the entire page cache ({:.3} GiB) are pinned.", bytes_per_page * pages_per_chunk * total_chunks * 1. / (1l << 30)); @@ -519,12 +523,12 @@ PageChunk * PageCache::getFreeChunk(std::unique_lock & lock /* globa size_t prev_pin_count = chunk->pin_count.fetch_add(1); chassert(prev_pin_count == 0); - evictChunk(chunk, lock); + evictChunk(chunk); return chunk; } -void PageCache::evictChunk(PageChunk * chunk, std::unique_lock & /* global_mutex */) +void PageCache::evictChunk(PageChunk * chunk) { if (chunk->key.has_value()) { @@ -548,7 +552,7 @@ void PageCache::evictChunk(PageChunk * chunk, std::unique_lock & /* chunk->pages_populated.unsetAll(); } -void PageCache::addMmap(std::unique_lock & /* global_mutex */) +void PageCache::addMmap() { /// ASLR by hand. void * address_hint = reinterpret_cast(std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(rng)); @@ -564,13 +568,13 @@ void PageCache::addMmap(std::unique_lock & /* global_mutex */) void PageCache::dropCache() { - std::unique_lock lock(global_mutex); + std::lock_guard lock(global_mutex); /// Detach and free unpinned chunks. bool logged_error = false; for (PageChunk & chunk : lru) { - evictChunk(&chunk, lock); + evictChunk(&chunk); if (use_madv_free) { diff --git a/src/Common/PageCache.h b/src/Common/PageCache.h index a3f465a82f5..04411e6fc24 100644 --- a/src/Common/PageCache.h +++ b/src/Common/PageCache.h @@ -270,28 +270,28 @@ private: mutable std::mutex global_mutex; - pcg64 rng; + pcg64 rng TSA_GUARDED_BY(global_mutex); - std::vector mmaps; - size_t total_chunks = 0; + std::vector mmaps TSA_GUARDED_BY(global_mutex); + size_t total_chunks TSA_GUARDED_BY(global_mutex) = 0; /// All non-pinned chunks, including ones not assigned to any file. Least recently used is begin(). - boost::intrusive::list, boost::intrusive::constant_time_size> lru; + boost::intrusive::list, boost::intrusive::constant_time_size> lru TSA_GUARDED_BY(global_mutex); - HashMap chunk_by_key; + HashMap chunk_by_key TSA_GUARDED_BY(global_mutex); /// Get a usable chunk, doing eviction or allocation if needed. /// Caller is responsible for clearing pages_populated. - PageChunk * getFreeChunk(std::unique_lock & /* global_mutex */); - void addMmap(std::unique_lock & /* global_mutex */); - void evictChunk(PageChunk * chunk, std::unique_lock & /* global_mutex */); + PageChunk * getFreeChunk() TSA_REQUIRES(global_mutex); + void addMmap() TSA_REQUIRES(global_mutex); + void evictChunk(PageChunk * chunk) TSA_REQUIRES(global_mutex); void removeRef(PageChunk * chunk) noexcept; /// These may run in parallel with getFreeChunk(), so be very careful about which fields of the PageChunk we touch here. - void sendChunkToLimbo(PageChunk * chunk, std::unique_lock & /* chunk_mutex */) const noexcept; + void sendChunkToLimbo(PageChunk * chunk, std::lock_guard & /* chunk_mutex */) const noexcept; /// Returns {pages_restored, pages_evicted}. - std::pair restoreChunkFromLimbo(PageChunk * chunk, std::unique_lock & /* chunk_mutex */) const noexcept; + std::pair restoreChunkFromLimbo(PageChunk * chunk, std::lock_guard & /* chunk_mutex */) const noexcept; }; using PageCachePtr = std::shared_ptr; diff --git a/src/IO/CachedInMemoryReadBufferFromFile.cpp b/src/IO/CachedInMemoryReadBufferFromFile.cpp index 384d2229f14..ceaf0ca4752 100644 --- a/src/IO/CachedInMemoryReadBufferFromFile.cpp +++ b/src/IO/CachedInMemoryReadBufferFromFile.cpp @@ -136,7 +136,6 @@ bool CachedInMemoryReadBufferFromFile::nextImpl() char * piece_start = chunk->getChunk()->data + pos; size_t piece_size = chunk_size - pos; in->set(piece_start, piece_size); - LOG_INFO(&Poco::Logger::get("asdqwe"), "this {:x}, in {:x}, path {}, size {}, offset {:x}, pos {:x}", reinterpret_cast(this), reinterpret_cast(in.get()), cache_key.path, file_size.value(), cache_key.offset, pos); if (pos == 0) in->seek(cache_key.offset, SEEK_SET); else @@ -155,7 +154,6 @@ bool CachedInMemoryReadBufferFromFile::nextImpl() memmove(piece_start, in->position(), n); in->position() += n; pos += n; - LOG_INFO(&Poco::Logger::get("asdqwe"), "this {:x}, got {:x} bytes", reinterpret_cast(this), n); } chunk->markPrefixPopulated(chunk_size); diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index f4dc7880be4..31ea45d92a9 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -103,7 +103,6 @@ struct ReadSettings bool avoid_readthrough_cache_outside_query_context = true; size_t filesystem_cache_segments_batch_size = 20; - //asdqwe assign these two bool use_page_cache_for_disks_without_file_cache = false; bool read_from_page_cache_if_exists_otherwise_bypass_cache = false; bool page_cache_inject_eviction = false; diff --git a/tests/queries/0_stateless/02867_page_cache.reference b/tests/queries/0_stateless/02867_page_cache.reference index 5502059508a..c3d6484a175 100644 --- a/tests/queries/0_stateless/02867_page_cache.reference +++ b/tests/queries/0_stateless/02867_page_cache.reference @@ -1,23 +1,21 @@ -54975576145920 +cold read 54975576145920 PageCacheBytesUnpinnedRoundedToHugePages 1 PageCacheBytesUnpinnedRoundedToPages 1 PageCacheChunkMisses 1 ReadBufferFromS3Bytes 1 -54975576145920 +repeat read 1 54975576145920 PageCacheBytesUnpinnedRoundedToHugePages 1 PageCacheBytesUnpinnedRoundedToPages 1 PageCacheChunkDataHits 1 -54975576145920 +dropped and bypassed cache 54975576145920 +PageCacheChunkMisses 1 +ReadBufferFromS3Bytes 1 +repeat read 2 54975576145920 PageCacheBytesUnpinnedRoundedToHugePages 1 PageCacheBytesUnpinnedRoundedToPages 1 PageCacheChunkMisses 1 ReadBufferFromS3Bytes 1 -54975576145920 -PageCacheBytesUnpinnedRoundedToHugePages 1 -PageCacheBytesUnpinnedRoundedToPages 1 -PageCacheChunkMisses 1 -ReadBufferFromS3Bytes 1 -54975576145920 +repeat read 3 54975576145920 PageCacheBytesUnpinnedRoundedToHugePages 1 PageCacheBytesUnpinnedRoundedToPages 1 PageCacheChunkDataHits 1 diff --git a/tests/queries/0_stateless/02867_page_cache.sql b/tests/queries/0_stateless/02867_page_cache.sql index 8765b30ebc3..f1882de4af6 100644 --- a/tests/queries/0_stateless/02867_page_cache.sql +++ b/tests/queries/0_stateless/02867_page_cache.sql @@ -46,7 +46,7 @@ insert into events_snapshot select * from system.events; -- Cold read, should miss cache. (Populating cache on write is not implemented yet.) -select sum(k) from page_cache_03055; +select 'cold read', sum(k) from page_cache_03055; select * from events_diff where event not in ('PageCacheChunkDataHits'); truncate table events_snapshot; @@ -54,7 +54,7 @@ insert into events_snapshot select * from system.events; -- Repeat read, should hit cache. -select sum(k) from page_cache_03055; +select 'repeat read 1', sum(k) from page_cache_03055; select * from events_diff; truncate table events_snapshot; @@ -64,16 +64,17 @@ insert into events_snapshot select * from system.events; system drop page cache; -select sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1; +select 'dropped and bypassed cache', sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1; -- Data could be read multiple times because we're not writing to cache. -select event, if(event in ('PageCacheChunkMisses', 'ReadBufferFromS3Bytes'), diff >= 1, diff) from events_diff where event not in ('PageCacheChunkDataHits'); +-- (Not checking PageCacheBytesUnpinned* because it's unreliable in this case because of an intentional race condition, see PageCache::evictChunk.) +select event, if(event in ('PageCacheChunkMisses', 'ReadBufferFromS3Bytes'), diff >= 1, diff) from events_diff where event not in ('PageCacheChunkDataHits', 'PageCacheBytesUnpinnedRoundedToPages', 'PageCacheBytesUnpinnedRoundedToHugePages'); truncate table events_snapshot; insert into events_snapshot select * from system.events; -- Repeat read, should still miss, but populate cache. -select sum(k) from page_cache_03055; +select 'repeat read 2', sum(k) from page_cache_03055; select * from events_diff where event not in ('PageCacheChunkDataHits'); truncate table events_snapshot; @@ -81,7 +82,7 @@ insert into events_snapshot select * from system.events; -- Read again, hit the cache. -select sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1; +select 'repeat read 3', sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1; select * from events_diff; truncate table events_snapshot; From c8878332ca8a07d4c2a6bb0f4af4580d47dba779 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 6 Mar 2024 16:46:59 +0000 Subject: [PATCH 175/197] fix StorageSystemDictionaries --- src/Storages/System/StorageSystemDictionaries.cpp | 4 ++-- src/Storages/System/StorageSystemDictionaries.h | 4 +--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index 7b733c872c4..a19741f92d1 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -51,8 +51,8 @@ catch (const DB::Exception &) } -StorageSystemDictionaries::StorageSystemDictionaries(const StorageID & storage_id_) - : IStorageSystemOneBlock(storage_id_, getColumnsDescription()) +StorageSystemDictionaries::StorageSystemDictionaries(const StorageID & storage_id_, ColumnsDescription columns_description_) + : IStorageSystemOneBlock(storage_id_, std::move(columns_description_)) { VirtualColumnsDescription virtuals; virtuals.addEphemeral("key", std::make_shared(), ""); diff --git a/src/Storages/System/StorageSystemDictionaries.h b/src/Storages/System/StorageSystemDictionaries.h index 5a8d7eae167..058b8b163d9 100644 --- a/src/Storages/System/StorageSystemDictionaries.h +++ b/src/Storages/System/StorageSystemDictionaries.h @@ -12,15 +12,13 @@ class Context; class StorageSystemDictionaries final : public IStorageSystemOneBlock { public: - explicit StorageSystemDictionaries(const StorageID & storage_id_); + StorageSystemDictionaries(const StorageID & storage_id_, ColumnsDescription columns_description_); std::string getName() const override { return "SystemDictionaries"; } static ColumnsDescription getColumnsDescription(); protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; }; From 2730f0b54fa5d7e81ca6e502bcb7567266cff4d9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Mar 2024 16:48:03 +0000 Subject: [PATCH 176/197] Allow subqueries for IN with parallel replicas under a setting. --- src/Core/Settings.h | 1 + src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Planner/Planner.cpp | 2 +- src/Planner/findParallelReplicasQuery.cpp | 12 +- ...49_parallel_replicas_in_subquery.reference | 5 + .../02949_parallel_replicas_in_subquery.sql | 12 +- ...llel_replicas_joins_and_analyzer.reference | 176 +++++++++++++++++- ...arallel_replicas_joins_and_analyzer.sql.j2 | 38 ++++ 8 files changed, 238 insertions(+), 10 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 99a48d043d0..b1cabe66aaf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -186,6 +186,7 @@ class IColumn; \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ + M(Bool, parallel_replicas_allow_subqueries_for_in, true, "If true, subquery for IN will be executed on every follower replica.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index c53e54573c5..bb3bd120303 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -216,7 +216,7 @@ private: if (enable_parallel_processing_of_joins) { /// We don't enable parallel replicas for IN (subquery) - if (ast->as()) + if (!settings.parallel_replicas_allow_subqueries_for_in && ast->as()) { if (settings.allow_experimental_parallel_reading_from_replicas == 1) { diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index efd8f4faa42..219f67ecbd8 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1373,7 +1373,7 @@ void Planner::buildPlanForQueryNode() const auto & settings = query_context->getSettingsRef(); if (query_context->canUseTaskBasedParallelReplicas()) { - if (planner_context->getPreparedSets().hasSubqueries()) + if (!settings.parallel_replicas_allow_subqueries_for_in && planner_context->getPreparedSets().hasSubqueries()) { if (settings.allow_experimental_parallel_reading_from_replicas >= 2) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "IN with subquery is not supported with parallel replicas"); diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 95afcb605b3..e0e47915047 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -156,7 +157,8 @@ QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, co /// Otherwise we can execute current query up to WithMergableStage only. const QueryNode * findQueryForParallelReplicas( std::stack stack, - const std::unordered_map & mapping) + const std::unordered_map & mapping, + const Settings & settings) { const QueryPlan::Node * prev_checked_node = nullptr; const QueryNode * res = nullptr; @@ -192,7 +194,11 @@ const QueryNode * findQueryForParallelReplicas( { const auto * expression = typeid_cast(step); const auto * filter = typeid_cast(step); - if (!expression && !filter) + + const auto * creating_sets = typeid_cast(step); + bool allowed_creating_sets = settings.parallel_replicas_allow_subqueries_for_in && creating_sets; + + if (!expression && !filter && !allowed_creating_sets) can_distribute_full_node = false; next_node_to_check = children.front(); @@ -274,7 +280,7 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr /// So that we build a list of candidates again, and call findQueryForParallelReplicas for it. auto new_stack = getSupportingParallelReplicasQuery(updated_query_tree.get()); const auto & mapping = planner.getQueryNodeToPlanStepMapping(); - const auto * res = findQueryForParallelReplicas(new_stack, mapping); + const auto * res = findQueryForParallelReplicas(new_stack, mapping, context->getSettingsRef()); /// Now, return a query from initial stack. if (res) diff --git a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.reference b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.reference index 4d33751c699..8ae3cdf8f3a 100644 --- a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.reference +++ b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.reference @@ -2,7 +2,12 @@ 2 test2 8 3 test3 8 4 test4 1985 +2 test2 8 +3 test3 8 +4 test4 1985 --- 1 test1 42 +1 test1 42 --- 3 test3 +3 test3 diff --git a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql index 53b8a761cda..9000d37c801 100644 --- a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql +++ b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql @@ -8,19 +8,23 @@ INSERT INTO merge_tree_in_subqueries VALUES(5, 'test5', 0); SET max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost', parallel_replicas_for_non_replicated_merge_tree=1; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2; -- { serverError SUPPORT_IS_DISABLED } +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; -- { serverError SUPPORT_IS_DISABLED } +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=1; SELECT '---'; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=1; SELECT '---'; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=1; -- IN with tuples is allowed SELECT '---'; -SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2; +SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; +SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; DROP TABLE IF EXISTS merge_tree_in_subqueries; diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index 6b1fdfd42a2..edd99058bd9 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -237,7 +237,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1;-- { echoOn } +select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; Expression Sorting Expression @@ -250,6 +250,93 @@ Expression ReadFromRemoteParallelReplicas Expression ReadFromRemoteParallelReplicas +-- +-- Subqueries for IN allowed +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +0 0 0 0 0 0 +1 1 0 0 0 0 +3 3 0 0 0 0 +4 4 0 0 0 0 +5 5 0 0 0 0 +6 6 6 6 0 0 +7 7 0 0 0 0 +8 8 8 8 0 0 +9 9 0 0 0 0 +10 10 10 10 0 0 +11 11 0 0 0 0 +12 12 12 12 12 12 +13 13 0 0 0 0 +14 14 14 14 0 0 +15 15 0 0 0 0 +explain description=0 +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +Expression + Sorting + Expression + ReadFromRemoteParallelReplicas +-- +-- Subqueries for IN are not allowed +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +0 0 0 0 0 0 +1 1 0 0 0 0 +3 3 0 0 0 0 +4 4 0 0 0 0 +5 5 0 0 0 0 +6 6 6 6 0 0 +7 7 0 0 0 0 +8 8 8 8 0 0 +9 9 0 0 0 0 +10 10 10 10 0 0 +11 11 0 0 0 0 +12 12 12 12 12 12 +13 13 0 0 0 0 +14 14 14 14 0 0 +15 15 0 0 0 0 +explain description=0 +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0;-- { echoOn } +Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + ReadFromRemoteParallelReplicas + Expression + ReadFromRemoteParallelReplicas set parallel_replicas_prefer_local_join = 1; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; @@ -500,3 +587,90 @@ Expression ReadFromRemoteParallelReplicas Expression ReadFromRemoteParallelReplicas +-- +-- Subqueries for IN allowed +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +0 0 0 0 0 0 +1 1 0 0 0 0 +3 3 0 0 0 0 +4 4 0 0 0 0 +5 5 0 0 0 0 +6 6 6 6 0 0 +7 7 0 0 0 0 +8 8 8 8 0 0 +9 9 0 0 0 0 +10 10 10 10 0 0 +11 11 0 0 0 0 +12 12 12 12 12 12 +13 13 0 0 0 0 +14 14 14 14 0 0 +15 15 0 0 0 0 +explain description=0 +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +Expression + Sorting + Expression + ReadFromRemoteParallelReplicas +-- +-- Subqueries for IN are not allowed +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +0 0 0 0 0 0 +1 1 0 0 0 0 +3 3 0 0 0 0 +4 4 0 0 0 0 +5 5 0 0 0 0 +6 6 6 6 0 0 +7 7 0 0 0 0 +8 8 8 8 0 0 +9 9 0 0 0 0 +10 10 10 10 0 0 +11 11 0 0 0 0 +12 12 12 12 12 12 +13 13 0 0 0 0 +14 14 14 14 0 0 +15 15 0 0 0 0 +explain description=0 +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + ReadFromRemoteParallelReplicas + Expression + ReadFromRemoteParallelReplicas diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 index 7d2766d52f8..e0de8c64950 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 @@ -126,4 +126,42 @@ sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +-- +-- Subqueries for IN allowed +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; + +explain description=0 +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; + +-- +-- Subqueries for IN are not allowed +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; + +explain description=0 +with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), +sub2 as (select y, z from tab2 where y != 4), +sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), +sub4 as (select z, a from tab3 where z != 8), +sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) +select * from sub5 order by x +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; + {%- endfor %} From 4dd5e4fe993b8ca2a5f693f8f809fa0c2fc76fa2 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 6 Mar 2024 16:30:47 +0000 Subject: [PATCH 177/197] CI: fix stage config (unit test release issue) #no_ci_cache --- tests/ci/ci_config.py | 16 ++++++++++++---- tests/ci/test_ci_config.py | 20 +++++++++++++++++++- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 308a9098c29..4d944d24765 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -11,7 +11,7 @@ from ci_utils import WithIter from integration_test_images import IMAGES -class CIStages: +class CIStages(metaclass=WithIter): NA = "UNKNOWN" BUILDS_1 = "Builds_1" BUILDS_2 = "Builds_2" @@ -547,9 +547,17 @@ class CIConfig: stage_type = CIStages.TESTS_2 elif self.is_test_job(job_name): stage_type = CIStages.TESTS_1 - if job_name == JobNames.LIBFUZZER_TEST: - # since fuzzers build in Builds_2, test must be in Tests_2 - stage_type = CIStages.TESTS_2 + if job_name in CI_CONFIG.test_configs: + required_build = CI_CONFIG.test_configs[job_name].required_build + assert required_build + if required_build in CI_CONFIG.get_builds_for_report( + JobNames.BUILD_CHECK + ): + stage_type = CIStages.TESTS_1 + else: + stage_type = CIStages.TESTS_2 + else: + stage_type = CIStages.TESTS_1 assert stage_type, f"BUG [{job_name}]" return stage_type diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 04c90105276..badbc4c5dcf 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 import unittest -from ci_config import JobNames, CI_CONFIG, Runners +from ci_config import CIStages, JobNames, CI_CONFIG, Runners class TestCIConfig(unittest.TestCase): @@ -10,3 +10,21 @@ class TestCIConfig(unittest.TestCase): for job in JobNames: runner = CI_CONFIG.get_runner_type(job) self.assertIn(runner, Runners) + + def test_job_stage_config(self): + """check runner is provided w/o exception""" + for job in JobNames: + stage = CI_CONFIG.get_job_ci_stage(job) + if job in [ + JobNames.STYLE_CHECK, + JobNames.FAST_TEST, + JobNames.JEPSEN_KEEPER, + JobNames.BUILD_CHECK, + JobNames.BUILD_CHECK_SPECIAL, + ]: + assert ( + stage == CIStages.NA + ), "These jobs are not in CI stages, must be NA" + else: + assert stage != CIStages.NA, f"stage not found for [{job}]" + self.assertIn(stage, CIStages) From 5e4c51ec05fbb6d2c21b8b5371170b1454a4f16c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Mar 2024 17:02:08 +0000 Subject: [PATCH 178/197] Update settings history --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 7b35c9fb239..3f816586085 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -90,6 +90,7 @@ static std::map sett {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, + {"parallel_replicas_allow_subqueries_for_in", false, true, "If true, subquery for IN will be executed on every follower replica"}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From a0d161704a44c74c9a19ed3e8c7ac9ba79f8a2db Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 6 Mar 2024 17:55:04 +0000 Subject: [PATCH 179/197] CI: remove sqllancer, sqllogic, sqltest from pr wf #do_not_test --- tests/ci/ci_config.py | 43 ++++++++++++++++++++++--------------------- 1 file changed, 22 insertions(+), 21 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 4d944d24765..2b19e0e5c13 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -473,21 +473,24 @@ perf_test_common_params = { "digest": perf_check_digest, "run_command": "performance_comparison_check.py", } -sqllancer_test_common_params = { - "digest": sqllancer_check_digest, - "run_command": "sqlancer_check.py", - "run_always": True, -} -sqllogic_test_params = { - "digest": sqllogic_check_digest, - "run_command": "sqllogic_test.py", - "timeout": 10800, -} -sql_test_params = { - "digest": sqltest_check_digest, - "run_command": "sqltest.py", - "timeout": 10800, -} +sqllancer_test_common_params = JobConfig( + digest=sqllancer_check_digest, + run_command="sqlancer_check.py", + release_only=True, + run_always=True, +) +sqllogic_test_params = JobConfig( + digest=sqllogic_check_digest, + run_command="sqllogic_test.py", + timeout=10800, + release_only=True, +) +sql_test_params = JobConfig( + digest=sqltest_check_digest, + run_command="sqltest.py", + timeout=10800, + release_only=True, +) clickbench_test_params = { "digest": DigestConfig( include_paths=[ @@ -1256,17 +1259,15 @@ CI_CONFIG = CIConfig( job_config=JobConfig(num_batches=4, run_by_label="pr-performance", **perf_test_common_params), # type: ignore ), JobNames.SQLANCER: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**sqllancer_test_common_params) # type: ignore + Build.PACKAGE_RELEASE, job_config=sqllancer_test_common_params ), JobNames.SQLANCER_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**sqllancer_test_common_params) # type: ignore + Build.PACKAGE_DEBUG, job_config=sqllancer_test_common_params ), JobNames.SQL_LOGIC_TEST: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**sqllogic_test_params) # type: ignore - ), - JobNames.SQLTEST: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**sql_test_params) # type: ignore + Build.PACKAGE_RELEASE, job_config=sqllogic_test_params ), + JobNames.SQLTEST: TestConfig(Build.PACKAGE_RELEASE, job_config=sql_test_params), JobNames.CLCIKBENCH_TEST: TestConfig( Build.PACKAGE_RELEASE, job_config=JobConfig(**clickbench_test_params) # type: ignore ), From d008b4eb4f32aa9d5473699dccfc094b2a47d1da Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 6 Mar 2024 20:24:38 +0100 Subject: [PATCH 180/197] fix test --- src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index c6407a99a4e..ff9941ee808 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -623,6 +623,15 @@ void DataPartStorageOnDiskBase::remove( } } + if (!disk->exists(from)) + { + LOG_ERROR(log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(disk, from)); + /// We will never touch this part again, so unlocking it from zero-copy + if (!can_remove_description) + can_remove_description.emplace(can_remove_callback()); + return; + } + try { disk->moveDirectory(from, to); From 4e1257974c597fe97a4a024eb2cd454f152c762a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 6 Mar 2024 18:26:20 +0000 Subject: [PATCH 181/197] CI: disable rerun check helper for build report jobs #do_not_test --- tests/ci/ci.py | 31 ++++++++++++++++++------------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index b222e81ad73..5049c4a6558 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1796,24 +1796,29 @@ def main() -> int: print(build_result.as_json()) print("::endgroup::") else: - # this is a test job - check if GH commit status is present - - # rerun helper check - # FIXME: remove rerun_helper check and rely on ci cache only + # this is a test job - check if GH commit status or cache record is present commit = get_commit( Github(get_best_robot_token(), per_page=100), pr_info.sha ) - rerun_helper = RerunHelper(commit, check_name_with_group) - if rerun_helper.is_already_finished_by_status(): - status = rerun_helper.get_finished_status() - assert status - previous_status = status.state - print("::group::Commit Status") - print(status) - print("::endgroup::") + + # rerun helper check + # FIXME: remove rerun_helper check and rely on ci cache only + if check_name not in ( + # we might want to rerun reports' jobs - disable rerun check for them + JobNames.BUILD_CHECK, + JobNames.BUILD_CHECK_SPECIAL, + ): + rerun_helper = RerunHelper(commit, check_name_with_group) + if rerun_helper.is_already_finished_by_status(): + status = rerun_helper.get_finished_status() + assert status + previous_status = status.state + print("::group::Commit Status") + print(status) + print("::endgroup::") # ci cache check - elif not indata["ci_flags"][Labels.NO_CI_CACHE]: + if not previous_status and not indata["ci_flags"][Labels.NO_CI_CACHE]: ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() job_config = CI_CONFIG.get_job_config(check_name) if ci_cache.is_successful( From ae7772e5873d3db874dcf1a0e10cc448fdae1e3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Mar 2024 00:58:13 +0100 Subject: [PATCH 182/197] Fix a typo --- .../02908_many_requests_to_system_replicas.reference | 2 +- .../0_stateless/02908_many_requests_to_system_replicas.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference index af0e50ec332..f1ca07ef408 100644 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference @@ -1,5 +1,5 @@ Creating 300 tables -Making making 200 requests to system.replicas +Making 200 requests to system.replicas Query system.replicas while waiting for other concurrent requests to finish 0 900 diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh index f93175529c0..d3eed891ab9 100755 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -33,7 +33,7 @@ done wait; -echo "Making making $CONCURRENCY requests to system.replicas" +echo "Making $CONCURRENCY requests to system.replicas" for i in `seq 1 $CONCURRENCY`; do From 0f4cd3198df59ba7e60730d6a815f8d40f90c913 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 7 Mar 2024 01:09:18 +0000 Subject: [PATCH 183/197] add version specification --- docs/en/sql-reference/statements/grant.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 879354d714b..b9c69b16e85 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -507,7 +507,7 @@ are turned on. ### NAMED COLLECTION ADMIN -Allows a certain operation on a specified named colleciton. +Allows a certain operation on a specified named colleciton. Before version 23.7 it was called NAMED COLLECTION CONTROL, and after 23.7 NAMED COLLECTION ADMIN as added and NAMED COLLECTION CONTROL is preserved as an alias. - `NAMED COLLECTION ADMIN`. Level: `NAMED_COLLECTION`. Aliases: `NAMED COLLECTION CONTROL` - `CREATE NAMED COLLECTION`. Level: `NAMED_COLLECTION` @@ -517,9 +517,11 @@ Allows a certain operation on a specified named colleciton. - `SHOW NAMED COLLECTIONS SECRETS`. Level: `NAMED_COLLECTION`. Aliases: `SHOW NAMED COLLECTIONS SECRETS` - `NAMED COLLECTION`. Level: `NAMED_COLLECTION`. Aliases: `NAMED COLLECTION USAGE, USE NAMED COLLECTION` +Unlike all other grants (CREATE, DROP, ALTER, SHOW) grant NAMED COLLECTION was added only in 23.7, while all others were added earlier - in 22.12. + **Examples** -Assuming a named collecion is called abc, we grant privilege CREATE NAMED COLLECTION to user john. +Assuming a named collection is called abc, we grant privilege CREATE NAMED COLLECTION to user john. - `GRANT CREATE NAMED COLLECTION ON abc TO john` ### ALL From c901c9c1f31a24e4b28f579a6d704de6be7231d7 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 7 Mar 2024 01:47:10 +0000 Subject: [PATCH 184/197] fix typo --- docs/en/sql-reference/statements/grant.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index b9c69b16e85..c7ce7ffe5e4 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -507,7 +507,7 @@ are turned on. ### NAMED COLLECTION ADMIN -Allows a certain operation on a specified named colleciton. Before version 23.7 it was called NAMED COLLECTION CONTROL, and after 23.7 NAMED COLLECTION ADMIN as added and NAMED COLLECTION CONTROL is preserved as an alias. +Allows a certain operation on a specified named collection. Before version 23.7 it was called NAMED COLLECTION CONTROL, and after 23.7 NAMED COLLECTION ADMIN as added and NAMED COLLECTION CONTROL is preserved as an alias. - `NAMED COLLECTION ADMIN`. Level: `NAMED_COLLECTION`. Aliases: `NAMED COLLECTION CONTROL` - `CREATE NAMED COLLECTION`. Level: `NAMED_COLLECTION` From de7f7e196edd5fdb6b9ab2c2712acda918d02ddd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 7 Mar 2024 13:55:22 +0800 Subject: [PATCH 185/197] Update docs/en/sql-reference/statements/grant.md --- docs/en/sql-reference/statements/grant.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index c7ce7ffe5e4..a93db29e82c 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -507,7 +507,7 @@ are turned on. ### NAMED COLLECTION ADMIN -Allows a certain operation on a specified named collection. Before version 23.7 it was called NAMED COLLECTION CONTROL, and after 23.7 NAMED COLLECTION ADMIN as added and NAMED COLLECTION CONTROL is preserved as an alias. +Allows a certain operation on a specified named collection. Before version 23.7 it was called NAMED COLLECTION CONTROL, and after 23.7 NAMED COLLECTION ADMIN was added and NAMED COLLECTION CONTROL is preserved as an alias. - `NAMED COLLECTION ADMIN`. Level: `NAMED_COLLECTION`. Aliases: `NAMED COLLECTION CONTROL` - `CREATE NAMED COLLECTION`. Level: `NAMED_COLLECTION` From 9e7894d8cbf4ea08657326083cf677699ed0be12 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 7 Mar 2024 16:38:07 +0800 Subject: [PATCH 186/197] Reduce the number of read rows from `system.numbers` (#60546) * Fix read more data for system.numbers * Fix tests --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 22 ++++++++++++++----- .../integration/test_storage_numbers/test.py | 16 ++++++++++++++ 2 files changed, 32 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index c72c63d09c4..a294683c640 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -26,9 +26,11 @@ namespace class NumbersSource : public ISource { public: - NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_) + NumbersSource(UInt64 block_size_, UInt64 offset_, std::optional limit_, UInt64 step_) : ISource(createHeader()), block_size(block_size_), next(offset_), step(step_) { + if (limit_.has_value()) + end = limit_.value() + offset_; } String getName() const override { return "Numbers"; } @@ -38,24 +40,32 @@ public: protected: Chunk generate() override { - auto column = ColumnUInt64::create(block_size); + UInt64 real_block_size = block_size; + if (end.has_value()) + { + if (end.value() <= next) + return {}; + real_block_size = std::min(block_size, end.value() - next); + } + auto column = ColumnUInt64::create(real_block_size); ColumnUInt64::Container & vec = column->getData(); UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 * pos = vec.data(); /// This also accelerates the code. - UInt64 * end = &vec[block_size]; - iota(pos, static_cast(end - pos), curr); + UInt64 * end_ = &vec[real_block_size]; + iota(pos, static_cast(end_ - pos), curr); next += step; progress(column->size(), column->byteSize()); - return {Columns{std::move(column)}, block_size}; + return {Columns{std::move(column)}, real_block_size}; } private: UInt64 block_size; UInt64 next; + std::optional end; /// not included UInt64 step; }; @@ -478,7 +488,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() for (size_t i = 0; i < num_streams; ++i) { auto source - = std::make_shared(max_block_size, numbers_storage.offset + i * max_block_size, num_streams * max_block_size); + = std::make_shared(max_block_size, numbers_storage.offset + i * max_block_size, numbers_storage.limit, num_streams * max_block_size); if (numbers_storage.limit && i == 0) { diff --git a/tests/integration/test_storage_numbers/test.py b/tests/integration/test_storage_numbers/test.py index 61fe8719ea2..cbd7793fd8c 100644 --- a/tests/integration/test_storage_numbers/test.py +++ b/tests/integration/test_storage_numbers/test.py @@ -242,3 +242,19 @@ def test_overflow(started_cluster): ) assert response == "(18446744073709551614),(18446744073709551615),(0),(1),(2)" check_read_rows("test_overflow", 5) + + +def test_non_number_filter(started_cluster): + response = node.query( + "SELECT toString(number) as a FROM numbers(3) WHERE a = '1' FORMAT Values", + query_id="test_non_number_filter", + ) + assert response == "('1')" + check_read_rows("test_non_number_filter", 3) + + response = node.query( + "SELECT toString(number) as a FROM numbers(1, 4) WHERE a = '1' FORMAT Values SETTINGS max_block_size = 3", + query_id="test_non_number_filter2", + ) + assert response == "('1')" + check_read_rows("test_non_number_filter2", 4) From a70b09b4e570f5a90bbb66ca6ff140bef1737b6f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 29 Feb 2024 23:33:36 +0100 Subject: [PATCH 187/197] Investigate an issue with MarkReleaseReady tags #no_ci_cache #job_package_release #job_package_aarch64 tags #job_binary_darwin #job_binary_darwin_aarch64 --- .github/workflows/master.yml | 18 +++++++++++++++++- .github/workflows/release_branches.yml | 18 +++++++++++++++++- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f18a83e1b97..2853adff48a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -305,7 +305,7 @@ jobs: runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} MarkReleaseReady: - if: ${{ ! (contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} + if: ${{ !failure() && !cancelled() }} needs: - BuilderBinDarwin - BuilderBinDarwinAarch64 @@ -313,9 +313,25 @@ jobs: - BuilderDebAarch64 runs-on: [self-hosted, style-checker] steps: + - name: Debug + run: | + echo need with different filters + cat << 'EOF' + ${{ toJSON(needs) }} + ${{ toJSON(needs.*.result) }} + no failures ${{ !contains(needs.*.result, 'failure') }} + no skips ${{ !contains(needs.*.result, 'skipped') }} + no both ${{ !(contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} + EOF + - name: Not ready + # fail the job to be able restart it + if: ${{ contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure') }} + run: exit 1 - name: Check out repository code + if: ${{ ! (contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} uses: ClickHouse/checkout@v1 - name: Mark Commit Release Ready + if: ${{ ! (contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 mark_release_ready.py diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index bdb045a70a6..9e95b3d3d8f 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -206,7 +206,7 @@ jobs: runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} MarkReleaseReady: - if: ${{ ! (contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} + if: ${{ !failure() && !cancelled() }} needs: - BuilderBinDarwin - BuilderBinDarwinAarch64 @@ -214,9 +214,25 @@ jobs: - BuilderDebAarch64 runs-on: [self-hosted, style-checker-aarch64] steps: + - name: Debug + run: | + echo need with different filters + cat << 'EOF' + ${{ toJSON(needs) }} + ${{ toJSON(needs.*.result) }} + no failures ${{ !contains(needs.*.result, 'failure') }} + no skips ${{ !contains(needs.*.result, 'skipped') }} + no both ${{ !(contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} + EOF + - name: Not ready + # fail the job to be able restart it + if: ${{ contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure') }} + run: exit 1 - name: Check out repository code + if: ${{ ! (contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} uses: ClickHouse/checkout@v1 - name: Mark Commit Release Ready + if: ${{ ! (contains(needs.*.result, 'skipped') || contains(needs.*.result, 'failure')) }} run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 mark_release_ready.py From e3f9fb273300997651c017f7da208899ff7f6b83 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Mar 2024 12:45:59 +0100 Subject: [PATCH 188/197] Trigger Mergeable check unconditionally in the final_check.py --- tests/ci/finish_check.py | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index e5268947304..eebc846f4b1 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -11,7 +11,7 @@ from commit_status_helper import ( get_commit, get_commit_filtered_statuses, post_commit_status, - update_mergeable_check, + trigger_mergeable_check, ) from get_robot_token import get_best_robot_token from pr_info import PRInfo @@ -24,14 +24,11 @@ def main(): pr_info = PRInfo(need_orgs=True) gh = Github(get_best_robot_token(), per_page=100) commit = get_commit(gh, pr_info.sha) - # Update the Mergeable Check at the final step - update_mergeable_check(commit, pr_info, CI_STATUS_NAME) + # Unconditionally update the Mergeable Check at the final step + statuses = get_commit_filtered_statuses(commit) + trigger_mergeable_check(commit, statuses) - statuses = [ - status - for status in get_commit_filtered_statuses(commit) - if status.context == CI_STATUS_NAME - ] + statuses = [s for s in statuses if s.context == CI_STATUS_NAME] if not statuses: return # Take the latest status From b037726180467e84fb96ba5c1513f3f3bdd699ad Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 7 Mar 2024 13:07:32 +0100 Subject: [PATCH 189/197] Update README.md Adding upcoming events, fixing release call links --- README.md | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9ada350d173..0ca338a3317 100644 --- a/README.md +++ b/README.md @@ -33,11 +33,21 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -Keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. +Keep an eye out for upcoming meetups and eventsaround the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. + +* [ClickHouse Meetup in Bellevue](https://www.meetup.com/clickhouse-seattle-user-group/events/298650371/) - Mar 11 +* [ClickHouse Meetup at Ramp's Offices in NYC](https://www.meetup.com/clickhouse-new-york-user-group/events/298640542/) - Mar 19 +* [ClickHouse Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/299479750/) - Mar 20 +* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/298997115/) - Mar 21 +* [ClickHouse Meetup in Bengaluru](https://www.meetup.com/clickhouse-bangalore-user-group/events/299479850/) - Mar 23 +* [ClickHouse Meetup in Zurich](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/299628922/) - Apr 16 +* [ClickHouse Meetup in Copenhagen](https://www.meetup.com/clickhouse-denmark-meetup-group/events/299629133/) - Apr 23 +* [ClickHouse Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28 + ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v24.1 Release Webinar**](https://www.youtube.com/watch?v=pBF9g0wGAGs) All the features of 24.1, one convenient video! Watch it now! +* **Recording available**: [**v24.2 Release Call**](https://www.youtube.com/watch?v=iN2y-TK8f3A) All the features of 24.2, one convenient video! Watch it now! * **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) From 62f9a00da833559562587294c09a1b5d88f31c81 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 6 Mar 2024 20:28:38 +0100 Subject: [PATCH 190/197] shuffle resolved IPs --- programs/copier/Internals.cpp | 2 +- src/Access/Common/AllowedClientHosts.cpp | 2 +- src/Client/ConnectionParameters.cpp | 2 +- src/Common/DNSResolver.cpp | 11 +++++++++-- src/Common/DNSResolver.h | 1 + src/Coordination/KeeperStateManager.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- 7 files changed, 15 insertions(+), 7 deletions(-) diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 0cfff7e3f6c..dcd199c6b38 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -259,7 +259,7 @@ ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std res.is_remote = 1; for (const auto & replica : replicas) { - if (isLocalAddress(DNSResolver::instance().resolveHost(replica.host_name))) + if (isLocalAddress(DNSResolver::instance().resolveHostAllInOriginOrder(replica.host_name).front())) { res.is_remote = 0; break; diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index c677465a7a1..bee0cdd7264 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -55,7 +55,7 @@ namespace { IPAddress addr_v6 = toIPv6(address); - auto host_addresses = DNSResolver::instance().resolveHostAll(host); + auto host_addresses = DNSResolver::instance().resolveHostAllInOriginOrder(host); for (const auto & addr : host_addresses) { diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 5c22b6c6d3f..16911f97e84 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -115,7 +115,7 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati /// At the same time, I want clickhouse-local to always work, regardless. /// TODO: get rid of glibc, or replace getaddrinfo to c-ares. - compression = config.getBool("compression", host != "localhost" && !isLocalAddress(DNSResolver::instance().resolveHost(host))) + compression = config.getBool("compression", host != "localhost" && !isLocalAddress(DNSResolver::instance().resolveHostAllInOriginOrder(host).front())) ? Protocol::Compression::Enable : Protocol::Compression::Disable; timeouts = ConnectionTimeouts() diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 250a8b3fc49..5b5f5369d5e 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -202,10 +202,10 @@ DNSResolver::DNSResolver() : impl(std::make_unique()), log(ge Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host) { - return pickAddress(resolveHostAll(host)); + return pickAddress(resolveHostAll(host)); // random order -> random pick } -DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host) +DNSResolver::IPAddresses DNSResolver::resolveHostAllInOriginOrder(const std::string & host) { if (impl->disable_cache) return resolveIPAddressImpl(host); @@ -214,6 +214,13 @@ DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host) return resolveIPAddressWithCache(impl->cache_host, host); } +DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host) +{ + auto addresses = resolveHostAllInOriginOrder(host); + std::shuffle(addresses.begin(), addresses.end(), thread_local_rng); + return addresses; +} + Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host_and_port) { if (impl->disable_cache) diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 27d81c9442a..6309eba888b 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -34,6 +34,7 @@ public: Poco::Net::IPAddress resolveHost(const std::string & host); /// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolves all its IPs + IPAddresses resolveHostAllInOriginOrder(const std::string & host); IPAddresses resolveHostAll(const std::string & host); /// Accepts host names like 'example.com:port' or '127.0.0.1:port' or '[::1]:port' and resolves its IP and port diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 87c56909387..c30df0b6313 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -30,7 +30,7 @@ bool isLocalhost(const std::string & hostname) { try { - return isLocalAddress(DNSResolver::instance().resolveHost(hostname)); + return isLocalAddress(DNSResolver::instance().resolveHostAllInOriginOrder(hostname).front()); } catch (...) { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cdeaa46cff2..a81392cb3d8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3270,7 +3270,7 @@ bool checkZooKeeperConfigIsLocal(const Poco::Util::AbstractConfiguration & confi if (startsWith(key, "node")) { String host = config.getString(config_name + "." + key + ".host"); - if (isLocalAddress(DNSResolver::instance().resolveHost(host))) + if (isLocalAddress(DNSResolver::instance().resolveHostAllInOriginOrder(host).front())) return true; } } From 2a62ec26557ff7ec199e749244314eada857de39 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Mar 2024 12:05:47 +0100 Subject: [PATCH 191/197] add comment to ping the CI --- src/Common/DNSResolver.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 6309eba888b..e3030e51a96 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -34,7 +34,9 @@ public: Poco::Net::IPAddress resolveHost(const std::string & host); /// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolves all its IPs + /// resolveHostAllInOriginOrder returns addresses with the same order as system call returns it IPAddresses resolveHostAllInOriginOrder(const std::string & host); + /// resolveHostAll returns addresses in random order IPAddresses resolveHostAll(const std::string & host); /// Accepts host names like 'example.com:port' or '127.0.0.1:port' or '[::1]:port' and resolves its IP and port From 61d64fd7e0ae10fa8a4573722e8221f8cbfef263 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 7 Mar 2024 12:23:52 +0000 Subject: [PATCH 192/197] CI: test build's checkout step issue #no_ci_cache #job_package_debug #job_style_check --- .github/workflows/reusable_build.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/reusable_build.yml b/.github/workflows/reusable_build.yml index 80d78d93e1b..d2fe6f5dbe7 100644 --- a/.github/workflows/reusable_build.yml +++ b/.github/workflows/reusable_build.yml @@ -43,7 +43,8 @@ jobs: runs-on: [self-hosted, '${{inputs.runner_type}}'] steps: - name: Check out repository code - uses: ClickHouse/checkout@v1 + # WIP: temporary try commit with limited perallelization of checkout + uses: ClickHouse/checkout@0be3f7b3098bae494d3ef5d29d2e0676fb606232 with: clear-repository: true ref: ${{ fromJson(inputs.data).git_ref }} From 5ae203ce4d212cb57527ba92712ac9479842101d Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 7 Mar 2024 14:03:21 +0100 Subject: [PATCH 193/197] Update README.md --- README.md | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 0ca338a3317..e00ce42a60b 100644 --- a/README.md +++ b/README.md @@ -31,6 +31,13 @@ curl https://clickhouse.com/ | sh * [Static Analysis (SonarCloud)](https://sonarcloud.io/project/issues?resolved=false&id=ClickHouse_ClickHouse) proposes C++ quality improvements. * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. +## Monthly Release & Community Call + +Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. + +* [v24.3 Community Call](https://clickhouse.com/company/events/v24-3-community-release-call) - Mar 26 +* [v24.4 Community Call](https://clickhouse.com/company/events/v24-4-community-release-call) - Apr 30 + ## Upcoming Events Keep an eye out for upcoming meetups and eventsaround the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. @@ -48,8 +55,6 @@ Keep an eye out for upcoming meetups and eventsaround the world. Somewhere else ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" * **Recording available**: [**v24.2 Release Call**](https://www.youtube.com/watch?v=iN2y-TK8f3A) All the features of 24.2, one convenient video! Watch it now! -* **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) - ## Interested in joining ClickHouse and making it your full-time job? From cd4b60b51a26f3fda2a772d8949b521c19d688ec Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Mar 2024 13:38:00 +0000 Subject: [PATCH 194/197] Rename the setting --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Planner/Planner.cpp | 2 +- src/Planner/findParallelReplicasQuery.cpp | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b1cabe66aaf..9f22d35bb9e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -186,7 +186,7 @@ class IColumn; \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ - M(Bool, parallel_replicas_allow_subqueries_for_in, true, "If true, subquery for IN will be executed on every follower replica.", 0) \ + M(Bool, parallel_replicas_allow_in_with_subquery, true, "If true, subquery for IN will be executed on every follower replica.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3f816586085..e7b96cee9d3 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -90,7 +90,7 @@ static std::map sett {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"parallel_replicas_allow_subqueries_for_in", false, true, "If true, subquery for IN will be executed on every follower replica"}, + {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index bb3bd120303..64b6eb5dce9 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -216,7 +216,7 @@ private: if (enable_parallel_processing_of_joins) { /// We don't enable parallel replicas for IN (subquery) - if (!settings.parallel_replicas_allow_subqueries_for_in && ast->as()) + if (!settings.parallel_replicas_allow_in_with_subquery && ast->as()) { if (settings.allow_experimental_parallel_reading_from_replicas == 1) { diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 219f67ecbd8..8082a01d41b 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1373,7 +1373,7 @@ void Planner::buildPlanForQueryNode() const auto & settings = query_context->getSettingsRef(); if (query_context->canUseTaskBasedParallelReplicas()) { - if (!settings.parallel_replicas_allow_subqueries_for_in && planner_context->getPreparedSets().hasSubqueries()) + if (!settings.parallel_replicas_allow_in_with_subquery && planner_context->getPreparedSets().hasSubqueries()) { if (settings.allow_experimental_parallel_reading_from_replicas >= 2) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "IN with subquery is not supported with parallel replicas"); diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index e0e47915047..ef640bcd42d 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -196,7 +196,7 @@ const QueryNode * findQueryForParallelReplicas( const auto * filter = typeid_cast(step); const auto * creating_sets = typeid_cast(step); - bool allowed_creating_sets = settings.parallel_replicas_allow_subqueries_for_in && creating_sets; + bool allowed_creating_sets = settings.parallel_replicas_allow_in_with_subquery && creating_sets; if (!expression && !filter && !allowed_creating_sets) can_distribute_full_node = false; From 82ba97c3a730a422a01b7765ccd7aca69887bf71 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 7 Mar 2024 17:16:13 +0100 Subject: [PATCH 195/197] More explicit template instantiations (#60730) --- base/base/CMakeLists.txt | 1 + base/base/Decimal.cpp | 87 ++++++ base/base/Decimal.h | 94 +++++-- base/base/extended_types.h | 38 +++ src/Columns/ColumnUnique.cpp | 25 ++ src/Columns/ColumnUnique.h | 21 ++ src/Common/FieldVisitorConvertToNumber.cpp | 23 ++ src/Common/FieldVisitorConvertToNumber.h | 15 + src/DataTypes/DataTypeDecimalBase.h | 6 + src/DataTypes/DataTypesDecimal.cpp | 256 +++++++++++++++++- src/DataTypes/DataTypesDecimal.h | 237 ++++++++-------- src/DataTypes/DataTypesNumber.cpp | 17 ++ src/DataTypes/DataTypesNumber.h | 16 ++ src/DataTypes/IDataType.cpp | 87 ++++++ src/DataTypes/IDataType.h | 142 +++++----- .../Serializations/SerializationDecimalBase.h | 6 + src/Functions/FunctionBase64Conversion.h | 2 +- src/Functions/FunctionHelpers.h | 6 +- src/Functions/FunctionStringReplace.h | 6 +- src/Functions/FunctionTokens.h | 10 +- src/Functions/FunctionUnixTimestamp64.h | 2 +- src/Functions/FunctionsAES.h | 20 +- src/Functions/FunctionsConversion.h | 12 +- src/Functions/JSONArrayLength.cpp | 2 +- src/Functions/URL/URLHierarchy.cpp | 2 +- src/Functions/URL/URLPathHierarchy.cpp | 2 +- .../URL/extractURLParameterNames.cpp | 2 +- src/Functions/URL/extractURLParameters.cpp | 2 +- src/Functions/array/arrayJaccardIndex.cpp | 4 +- src/Functions/array/arrayRandomSample.cpp | 4 +- src/Functions/array/arrayShingles.cpp | 4 +- src/Functions/arrayStringConcat.cpp | 4 +- src/Functions/castOrDefault.cpp | 4 +- src/Functions/countMatches.h | 4 +- src/Functions/extractAll.cpp | 4 +- src/Functions/extractAllGroups.h | 4 +- src/Functions/extractGroups.cpp | 4 +- src/Functions/formatQuery.cpp | 2 +- src/Functions/fromDaysSinceYearZero.cpp | 2 +- src/Functions/makeDate.cpp | 54 ++-- src/Functions/parseDateTime.cpp | 6 +- src/Functions/regexpExtract.cpp | 6 +- src/Functions/repeat.cpp | 4 +- src/Functions/seriesDecomposeSTL.cpp | 4 +- src/Functions/seriesOutliersDetectTukey.cpp | 8 +- src/Functions/seriesPeriodDetectFFT.cpp | 2 +- src/Functions/snowflake.cpp | 12 +- src/Functions/space.cpp | 2 +- src/Functions/sqid.cpp | 2 +- src/Functions/timestamp.cpp | 4 +- src/Functions/toDecimalString.cpp | 4 +- .../Impl/PrometheusTextOutputFormat.cpp | 24 +- 52 files changed, 962 insertions(+), 349 deletions(-) create mode 100644 base/base/Decimal.cpp create mode 100644 src/Columns/ColumnUnique.cpp create mode 100644 src/Common/FieldVisitorConvertToNumber.cpp diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 548ba01d86a..610877eae73 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -13,6 +13,7 @@ set (SRCS cgroupsv2.cpp coverage.cpp demangle.cpp + Decimal.cpp getAvailableMemoryAmount.cpp getFQDNOrHostName.cpp getMemoryAmount.cpp diff --git a/base/base/Decimal.cpp b/base/base/Decimal.cpp new file mode 100644 index 00000000000..7e65c0eb8d1 --- /dev/null +++ b/base/base/Decimal.cpp @@ -0,0 +1,87 @@ +#include +#include + +namespace DB +{ + +/// Explicit template instantiations. + +#define FOR_EACH_UNDERLYING_DECIMAL_TYPE(M) \ + M(Int32) \ + M(Int64) \ + M(Int128) \ + M(Int256) + +#define FOR_EACH_UNDERLYING_DECIMAL_TYPE_PASS(M, X) \ + M(Int32, X) \ + M(Int64, X) \ + M(Int128, X) \ + M(Int256, X) + +template const Decimal & Decimal::operator += (const T & x) { value += x; return *this; } +template const Decimal & Decimal::operator -= (const T & x) { value -= x; return *this; } +template const Decimal & Decimal::operator *= (const T & x) { value *= x; return *this; } +template const Decimal & Decimal::operator /= (const T & x) { value /= x; return *this; } +template const Decimal & Decimal::operator %= (const T & x) { value %= x; return *this; } + +template void NO_SANITIZE_UNDEFINED Decimal::addOverflow(const T & x) { value += x; } + +/// Maybe this explicit instantiation affects performance since operators cannot be inlined. + +template template const Decimal & Decimal::operator += (const Decimal & x) { value += static_cast(x.value); return *this; } +template template const Decimal & Decimal::operator -= (const Decimal & x) { value -= static_cast(x.value); return *this; } +template template const Decimal & Decimal::operator *= (const Decimal & x) { value *= static_cast(x.value); return *this; } +template template const Decimal & Decimal::operator /= (const Decimal & x) { value /= static_cast(x.value); return *this; } +template template const Decimal & Decimal::operator %= (const Decimal & x) { value %= static_cast(x.value); return *this; } + +#define DISPATCH(TYPE_T, TYPE_U) \ + template const Decimal & Decimal::operator += (const Decimal & x); \ + template const Decimal & Decimal::operator -= (const Decimal & x); \ + template const Decimal & Decimal::operator *= (const Decimal & x); \ + template const Decimal & Decimal::operator /= (const Decimal & x); \ + template const Decimal & Decimal::operator %= (const Decimal & x); +#define INVOKE(X) FOR_EACH_UNDERLYING_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_UNDERLYING_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + +#define DISPATCH(TYPE) template struct Decimal; +FOR_EACH_UNDERLYING_DECIMAL_TYPE(DISPATCH) +#undef DISPATCH + +template bool operator< (const Decimal & x, const Decimal & y) { return x.value < y.value; } +template bool operator> (const Decimal & x, const Decimal & y) { return x.value > y.value; } +template bool operator<= (const Decimal & x, const Decimal & y) { return x.value <= y.value; } +template bool operator>= (const Decimal & x, const Decimal & y) { return x.value >= y.value; } +template bool operator== (const Decimal & x, const Decimal & y) { return x.value == y.value; } +template bool operator!= (const Decimal & x, const Decimal & y) { return x.value != y.value; } + +#define DISPATCH(TYPE) \ +template bool operator< (const Decimal & x, const Decimal & y); \ +template bool operator> (const Decimal & x, const Decimal & y); \ +template bool operator<= (const Decimal & x, const Decimal & y); \ +template bool operator>= (const Decimal & x, const Decimal & y); \ +template bool operator== (const Decimal & x, const Decimal & y); \ +template bool operator!= (const Decimal & x, const Decimal & y); +FOR_EACH_UNDERLYING_DECIMAL_TYPE(DISPATCH) +#undef DISPATCH + + +template Decimal operator+ (const Decimal & x, const Decimal & y) { return x.value + y.value; } +template Decimal operator- (const Decimal & x, const Decimal & y) { return x.value - y.value; } +template Decimal operator* (const Decimal & x, const Decimal & y) { return x.value * y.value; } +template Decimal operator/ (const Decimal & x, const Decimal & y) { return x.value / y.value; } +template Decimal operator- (const Decimal & x) { return -x.value; } + +#define DISPATCH(TYPE) \ +template Decimal operator+ (const Decimal & x, const Decimal & y); \ +template Decimal operator- (const Decimal & x, const Decimal & y); \ +template Decimal operator* (const Decimal & x, const Decimal & y); \ +template Decimal operator/ (const Decimal & x, const Decimal & y); \ +template Decimal operator- (const Decimal & x); +FOR_EACH_UNDERLYING_DECIMAL_TYPE(DISPATCH) +#undef DISPATCH + +#undef FOR_EACH_UNDERLYING_DECIMAL_TYPE_PASS +#undef FOR_EACH_UNDERLYING_DECIMAL_TYPE +} diff --git a/base/base/Decimal.h b/base/base/Decimal.h index 66ff623217c..42f9e67c49d 100644 --- a/base/base/Decimal.h +++ b/base/base/Decimal.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -10,6 +11,18 @@ namespace DB template struct Decimal; class DateTime64; +#define FOR_EACH_UNDERLYING_DECIMAL_TYPE(M) \ + M(Int32) \ + M(Int64) \ + M(Int128) \ + M(Int256) + +#define FOR_EACH_UNDERLYING_DECIMAL_TYPE_PASS(M, X) \ + M(Int32, X) \ + M(Int64, X) \ + M(Int128, X) \ + M(Int256, X) + using Decimal32 = Decimal; using Decimal64 = Decimal; using Decimal128 = Decimal; @@ -50,36 +63,73 @@ struct Decimal return static_cast(value); } - const Decimal & operator += (const T & x) { value += x; return *this; } - const Decimal & operator -= (const T & x) { value -= x; return *this; } - const Decimal & operator *= (const T & x) { value *= x; return *this; } - const Decimal & operator /= (const T & x) { value /= x; return *this; } - const Decimal & operator %= (const T & x) { value %= x; return *this; } + const Decimal & operator += (const T & x); + const Decimal & operator -= (const T & x); + const Decimal & operator *= (const T & x); + const Decimal & operator /= (const T & x); + const Decimal & operator %= (const T & x); - template const Decimal & operator += (const Decimal & x) { value += x.value; return *this; } - template const Decimal & operator -= (const Decimal & x) { value -= x.value; return *this; } - template const Decimal & operator *= (const Decimal & x) { value *= x.value; return *this; } - template const Decimal & operator /= (const Decimal & x) { value /= x.value; return *this; } - template const Decimal & operator %= (const Decimal & x) { value %= x.value; return *this; } + template const Decimal & operator += (const Decimal & x); + template const Decimal & operator -= (const Decimal & x); + template const Decimal & operator *= (const Decimal & x); + template const Decimal & operator /= (const Decimal & x); + template const Decimal & operator %= (const Decimal & x); /// This is to avoid UB for sumWithOverflow() - void NO_SANITIZE_UNDEFINED addOverflow(const T & x) { value += x; } + void NO_SANITIZE_UNDEFINED addOverflow(const T & x); T value; }; -template inline bool operator< (const Decimal & x, const Decimal & y) { return x.value < y.value; } -template inline bool operator> (const Decimal & x, const Decimal & y) { return x.value > y.value; } -template inline bool operator<= (const Decimal & x, const Decimal & y) { return x.value <= y.value; } -template inline bool operator>= (const Decimal & x, const Decimal & y) { return x.value >= y.value; } -template inline bool operator== (const Decimal & x, const Decimal & y) { return x.value == y.value; } -template inline bool operator!= (const Decimal & x, const Decimal & y) { return x.value != y.value; } +#define DISPATCH(TYPE) extern template struct Decimal; +FOR_EACH_UNDERLYING_DECIMAL_TYPE(DISPATCH) +#undef DISPATCH -template inline Decimal operator+ (const Decimal & x, const Decimal & y) { return x.value + y.value; } -template inline Decimal operator- (const Decimal & x, const Decimal & y) { return x.value - y.value; } -template inline Decimal operator* (const Decimal & x, const Decimal & y) { return x.value * y.value; } -template inline Decimal operator/ (const Decimal & x, const Decimal & y) { return x.value / y.value; } -template inline Decimal operator- (const Decimal & x) { return -x.value; } +#define DISPATCH(TYPE_T, TYPE_U) \ + extern template const Decimal & Decimal::operator += (const Decimal & x); \ + extern template const Decimal & Decimal::operator -= (const Decimal & x); \ + extern template const Decimal & Decimal::operator *= (const Decimal & x); \ + extern template const Decimal & Decimal::operator /= (const Decimal & x); \ + extern template const Decimal & Decimal::operator %= (const Decimal & x); +#define INVOKE(X) FOR_EACH_UNDERLYING_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_UNDERLYING_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + +template bool operator< (const Decimal & x, const Decimal & y); +template bool operator> (const Decimal & x, const Decimal & y); +template bool operator<= (const Decimal & x, const Decimal & y); +template bool operator>= (const Decimal & x, const Decimal & y); +template bool operator== (const Decimal & x, const Decimal & y); +template bool operator!= (const Decimal & x, const Decimal & y); + +#define DISPATCH(TYPE) \ +extern template bool operator< (const Decimal & x, const Decimal & y); \ +extern template bool operator> (const Decimal & x, const Decimal & y); \ +extern template bool operator<= (const Decimal & x, const Decimal & y); \ +extern template bool operator>= (const Decimal & x, const Decimal & y); \ +extern template bool operator== (const Decimal & x, const Decimal & y); \ +extern template bool operator!= (const Decimal & x, const Decimal & y); +FOR_EACH_UNDERLYING_DECIMAL_TYPE(DISPATCH) +#undef DISPATCH + +template Decimal operator+ (const Decimal & x, const Decimal & y); +template Decimal operator- (const Decimal & x, const Decimal & y); +template Decimal operator* (const Decimal & x, const Decimal & y); +template Decimal operator/ (const Decimal & x, const Decimal & y); +template Decimal operator- (const Decimal & x); + +#define DISPATCH(TYPE) \ +extern template Decimal operator+ (const Decimal & x, const Decimal & y); \ +extern template Decimal operator- (const Decimal & x, const Decimal & y); \ +extern template Decimal operator* (const Decimal & x, const Decimal & y); \ +extern template Decimal operator/ (const Decimal & x, const Decimal & y); \ +extern template Decimal operator- (const Decimal & x); +FOR_EACH_UNDERLYING_DECIMAL_TYPE(DISPATCH) +#undef DISPATCH + +#undef FOR_EACH_UNDERLYING_DECIMAL_TYPE_PASS +#undef FOR_EACH_UNDERLYING_DECIMAL_TYPE /// Distinguishable type to allow function resolution/deduction based on value type, /// but also relatively easy to convert to/from Decimal64. diff --git a/base/base/extended_types.h b/base/base/extended_types.h index b58df45a97e..796167ab45d 100644 --- a/base/base/extended_types.h +++ b/base/base/extended_types.h @@ -64,6 +64,44 @@ template <> struct is_arithmetic { static constexpr bool value = true; template inline constexpr bool is_arithmetic_v = is_arithmetic::value; +#define FOR_EACH_ARITHMETIC_TYPE(M) \ + M(DataTypeDate) \ + M(DataTypeDate32) \ + M(DataTypeDateTime) \ + M(DataTypeInt8) \ + M(DataTypeUInt8) \ + M(DataTypeInt16) \ + M(DataTypeUInt16) \ + M(DataTypeInt32) \ + M(DataTypeUInt32) \ + M(DataTypeInt64) \ + M(DataTypeUInt64) \ + M(DataTypeInt128) \ + M(DataTypeUInt128) \ + M(DataTypeInt256) \ + M(DataTypeUInt256) \ + M(DataTypeFloat32) \ + M(DataTypeFloat64) + +#define FOR_EACH_ARITHMETIC_TYPE_PASS(M, X) \ + M(DataTypeDate, X) \ + M(DataTypeDate32, X) \ + M(DataTypeDateTime, X) \ + M(DataTypeInt8, X) \ + M(DataTypeUInt8, X) \ + M(DataTypeInt16, X) \ + M(DataTypeUInt16, X) \ + M(DataTypeInt32, X) \ + M(DataTypeUInt32, X) \ + M(DataTypeInt64, X) \ + M(DataTypeUInt64, X) \ + M(DataTypeInt128, X) \ + M(DataTypeUInt128, X) \ + M(DataTypeInt256, X) \ + M(DataTypeUInt256, X) \ + M(DataTypeFloat32, X) \ + M(DataTypeFloat64, X) + template struct make_unsigned // NOLINT(readability-identifier-naming) { diff --git a/src/Columns/ColumnUnique.cpp b/src/Columns/ColumnUnique.cpp new file mode 100644 index 00000000000..edfee69a752 --- /dev/null +++ b/src/Columns/ColumnUnique.cpp @@ -0,0 +1,25 @@ +#include + +namespace DB +{ + +/// Explicit template instantiations. +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; +template class ColumnUnique; + +} diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 34f1ffc15cd..76bbbbacdbf 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -15,6 +15,8 @@ #include #include #include +#include "Columns/ColumnsDateTime.h" +#include "Columns/ColumnsNumber.h" #include #include @@ -736,4 +738,23 @@ UInt128 ColumnUnique::IncrementalHash::getHash(const ColumnType & co return cur_hash; } + +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; +extern template class ColumnUnique; + } diff --git a/src/Common/FieldVisitorConvertToNumber.cpp b/src/Common/FieldVisitorConvertToNumber.cpp new file mode 100644 index 00000000000..75b3fbfe02a --- /dev/null +++ b/src/Common/FieldVisitorConvertToNumber.cpp @@ -0,0 +1,23 @@ +#include +#include "base/Decimal.h" + +namespace DB +{ + +/// Explicit template instantiations. +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; +template class FieldVisitorConvertToNumber; + +} diff --git a/src/Common/FieldVisitorConvertToNumber.h b/src/Common/FieldVisitorConvertToNumber.h index 47a1e669969..86e03cb5d77 100644 --- a/src/Common/FieldVisitorConvertToNumber.h +++ b/src/Common/FieldVisitorConvertToNumber.h @@ -117,4 +117,19 @@ public: T operator() (const bool & x) const { return T(x); } }; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; +extern template class FieldVisitorConvertToNumber; + } diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index adbe9c95b14..9887dfabcdb 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -207,4 +207,10 @@ inline DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value) return std::make_shared>(precision_value, scale_value); } +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; + } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 7ad9f0b6fd8..77a7a3e7237 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -112,6 +112,256 @@ static DataTypePtr createExact(const ASTPtr & arguments) return createDecimal(precision, scale); } +template +requires (IsDataTypeDecimal && IsDataTypeDecimal) +ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) +{ + using FromFieldType = typename FromDataType::FieldType; + using ToFieldType = typename ToDataType::FieldType; + using MaxFieldType = std::conditional_t<(sizeof(FromFieldType) > sizeof(ToFieldType)), FromFieldType, ToFieldType>; + using MaxNativeType = typename MaxFieldType::NativeType; + + static constexpr bool throw_exception = std::is_same_v; + + MaxNativeType converted_value; + if (scale_to > scale_from) + { + converted_value = DecimalUtils::scaleMultiplier(scale_to - scale_from); + if (common::mulOverflow(static_cast(value.value), converted_value, converted_value)) + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow while multiplying {} by scale {}", + std::string(ToDataType::family_name), toString(value.value), toString(converted_value)); + else + return ReturnType(false); + } + } + else if (scale_to == scale_from) + { + converted_value = value.value; + } + else + { + converted_value = value.value / DecimalUtils::scaleMultiplier(scale_from - scale_to); + } + + if constexpr (sizeof(FromFieldType) > sizeof(ToFieldType)) + { + if (converted_value < std::numeric_limits::min() || + converted_value > std::numeric_limits::max()) + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow: {} is not in range ({}, {})", + std::string(ToDataType::family_name), toString(converted_value), + toString(std::numeric_limits::min()), + toString(std::numeric_limits::max())); + else + return ReturnType(false); + } + } + + result = static_cast(converted_value); + + return ReturnType(true); +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template void convertDecimalsImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename TO_DATA_TYPE::FieldType & result); \ + template bool convertDecimalsImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef DISPATCH + + +template +requires (IsDataTypeDecimal && IsDataTypeDecimal) +typename ToDataType::FieldType convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to) +{ + using ToFieldType = typename ToDataType::FieldType; + ToFieldType result; + + convertDecimalsImpl(value, scale_from, scale_to, result); + + return result; +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template typename TO_DATA_TYPE::FieldType convertDecimals(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef DISPATCH + + +template +requires (IsDataTypeDecimal && IsDataTypeDecimal) +bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) +{ + return convertDecimalsImpl(value, scale_from, scale_to, result); +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template bool tryConvertDecimals(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef DISPATCH + + +template +requires (IsDataTypeDecimal && is_arithmetic_v) +ReturnType convertFromDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType & result) +{ + using FromFieldType = typename FromDataType::FieldType; + using ToFieldType = typename ToDataType::FieldType; + + return DecimalUtils::convertToImpl(value, scale, result); +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template void convertFromDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); \ + template bool convertFromDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_ARITHMETIC_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + + +template +requires (IsDataTypeDecimal && is_arithmetic_v) +inline typename ToDataType::FieldType convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale) +{ + typename ToDataType::FieldType result; + + convertFromDecimalImpl(value, scale, result); + + return result; +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template typename TO_DATA_TYPE::FieldType convertFromDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_ARITHMETIC_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + + +template +requires (IsDataTypeDecimal && is_arithmetic_v) +inline bool tryConvertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result) +{ + return convertFromDecimalImpl(value, scale, result); +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template bool tryConvertFromDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType& result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_ARITHMETIC_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + + +template +requires (is_arithmetic_v && IsDataTypeDecimal) +ReturnType convertToDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType & result) +{ + using FromFieldType = typename FromDataType::FieldType; + using ToFieldType = typename ToDataType::FieldType; + using ToNativeType = typename ToFieldType::NativeType; + + static constexpr bool throw_exception = std::is_same_v; + + if constexpr (std::is_floating_point_v) + { + if (!std::isfinite(value)) + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow. Cannot convert infinity or NaN to decimal", ToDataType::family_name); + else + return ReturnType(false); + } + + auto out = value * static_cast(DecimalUtils::scaleMultiplier(scale)); + + if (out <= static_cast(std::numeric_limits::min()) || + out >= static_cast(std::numeric_limits::max())) + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow. Float is out of Decimal range", ToDataType::family_name); + else + return ReturnType(false); + } + + result = static_cast(out); + return ReturnType(true); + } + else + { + if constexpr (is_big_int_v) + return ReturnType(convertDecimalsImpl, ToDataType, ReturnType>(static_cast(value), 0, scale, result)); + else if constexpr (std::is_same_v) + return ReturnType(convertDecimalsImpl, ToDataType, ReturnType>(static_cast(value), 0, scale, result)); + else + return ReturnType(convertDecimalsImpl, ToDataType, ReturnType>(static_cast(value), 0, scale, result)); + } +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template void convertToDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); \ + template bool convertToDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_ARITHMETIC_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + + +template +requires (is_arithmetic_v && IsDataTypeDecimal) +inline typename ToDataType::FieldType convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale) +{ + typename ToDataType::FieldType result; + convertToDecimalImpl(value, scale, result); + return result; +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template typename TO_DATA_TYPE::FieldType convertToDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale); +#define INVOKE(X) FOR_EACH_ARITHMETIC_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + + +template +requires (is_arithmetic_v && IsDataTypeDecimal) +inline bool tryConvertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result) +{ + return convertToDecimalImpl(value, scale, result); +} + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + template bool tryConvertToDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType& result); +#define INVOKE(X) FOR_EACH_ARITHMETIC_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + + +template +DataTypePtr createDecimalMaxPrecision(UInt64 scale) +{ + return std::make_shared>(DecimalUtils::max_precision, scale); +} + +template DataTypePtr createDecimalMaxPrecision(UInt64 scale); +template DataTypePtr createDecimalMaxPrecision(UInt64 scale); +template DataTypePtr createDecimalMaxPrecision(UInt64 scale); +template DataTypePtr createDecimalMaxPrecision(UInt64 scale); + +/// Explicit template instantiations. +template class DataTypeDecimal; +template class DataTypeDecimal; +template class DataTypeDecimal; +template class DataTypeDecimal; + void registerDataTypeDecimal(DataTypeFactory & factory) { factory.registerDataType("Decimal32", createExact, DataTypeFactory::CaseInsensitive); @@ -125,10 +375,4 @@ void registerDataTypeDecimal(DataTypeFactory & factory) factory.registerAlias("FIXED", "Decimal", DataTypeFactory::CaseInsensitive); } -/// Explicit template instantiations. -template class DataTypeDecimal; -template class DataTypeDecimal; -template class DataTypeDecimal; -template class DataTypeDecimal; - } diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index e2b433cbe2f..badefc4c75a 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -3,7 +3,11 @@ #include #include #include +#include +#include #include +#include +#include #include #include @@ -13,7 +17,6 @@ namespace DB namespace ErrorCodes { - extern const int DECIMAL_OVERFLOW; extern const int LOGICAL_ERROR; } @@ -99,171 +102,145 @@ inline UInt32 getDecimalScale(const DataTypeDecimal & data_type) return data_type.getScale(); } +#define FOR_EACH_DECIMAL_TYPE(M) \ + M(DataTypeDecimal) \ + M(DataTypeDateTime64) \ + M(DataTypeDecimal32) \ + M(DataTypeDecimal64) \ + M(DataTypeDecimal128) \ + M(DataTypeDecimal256) + +#define FOR_EACH_DECIMAL_TYPE_PASS(M, X) \ + M(DataTypeDecimal, X) \ + M(DataTypeDateTime64, X) \ + M(DataTypeDecimal32, X) \ + M(DataTypeDecimal64, X) \ + M(DataTypeDecimal128, X) \ + M(DataTypeDecimal256, X) + + template requires (IsDataTypeDecimal && IsDataTypeDecimal) -inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) -{ - using FromFieldType = typename FromDataType::FieldType; - using ToFieldType = typename ToDataType::FieldType; - using MaxFieldType = std::conditional_t<(sizeof(FromFieldType) > sizeof(ToFieldType)), FromFieldType, ToFieldType>; - using MaxNativeType = typename MaxFieldType::NativeType; +ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result); - static constexpr bool throw_exception = std::is_same_v; +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template void convertDecimalsImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename TO_DATA_TYPE::FieldType & result); \ + extern template bool convertDecimalsImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH - MaxNativeType converted_value; - if (scale_to > scale_from) - { - converted_value = DecimalUtils::scaleMultiplier(scale_to - scale_from); - if (common::mulOverflow(static_cast(value.value), converted_value, converted_value)) - { - if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow while multiplying {} by scale {}", - std::string(ToDataType::family_name), toString(value.value), toString(converted_value)); - else - return ReturnType(false); - } - } - else if (scale_to == scale_from) - { - converted_value = value.value; - } - else - { - converted_value = value.value / DecimalUtils::scaleMultiplier(scale_from - scale_to); - } - - if constexpr (sizeof(FromFieldType) > sizeof(ToFieldType)) - { - if (converted_value < std::numeric_limits::min() || - converted_value > std::numeric_limits::max()) - { - if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow: {} is not in range ({}, {})", - std::string(ToDataType::family_name), toString(converted_value), - toString(std::numeric_limits::min()), - toString(std::numeric_limits::max())); - else - return ReturnType(false); - } - } - - result = static_cast(converted_value); - - return ReturnType(true); -} template requires (IsDataTypeDecimal && IsDataTypeDecimal) -inline typename ToDataType::FieldType convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to) -{ - using ToFieldType = typename ToDataType::FieldType; - ToFieldType result; +typename ToDataType::FieldType convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to); - convertDecimalsImpl(value, scale_from, scale_to, result); +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template typename TO_DATA_TYPE::FieldType convertDecimals(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH - return result; -} template requires (IsDataTypeDecimal && IsDataTypeDecimal) -inline bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) -{ - return convertDecimalsImpl(value, scale_from, scale_to, result); -} +bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result); + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template bool tryConvertDecimals(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + template requires (IsDataTypeDecimal && is_arithmetic_v) -inline ReturnType convertFromDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result) -{ - using FromFieldType = typename FromDataType::FieldType; - using ToFieldType = typename ToDataType::FieldType; +ReturnType convertFromDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType & result); + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template void convertFromDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); \ + extern template bool convertFromDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_ARITHMETIC_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH - return DecimalUtils::convertToImpl(value, scale, result); -} template requires (IsDataTypeDecimal && is_arithmetic_v) -inline typename ToDataType::FieldType convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale) -{ - typename ToDataType::FieldType result; +typename ToDataType::FieldType convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale); - convertFromDecimalImpl(value, scale, result); +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template typename TO_DATA_TYPE::FieldType convertFromDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_ARITHMETIC_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH - return result; -} template requires (IsDataTypeDecimal && is_arithmetic_v) -inline bool tryConvertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result) -{ - return convertFromDecimalImpl(value, scale, result); -} +bool tryConvertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result); + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template bool tryConvertFromDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType& result); +#define INVOKE(X) FOR_EACH_DECIMAL_TYPE_PASS(DISPATCH, X) +FOR_EACH_ARITHMETIC_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + template requires (is_arithmetic_v && IsDataTypeDecimal) -inline ReturnType convertToDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result) -{ - using FromFieldType = typename FromDataType::FieldType; - using ToFieldType = typename ToDataType::FieldType; - using ToNativeType = typename ToFieldType::NativeType; +ReturnType convertToDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result); - static constexpr bool throw_exception = std::is_same_v; +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template void convertToDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); \ + extern template bool convertToDecimalImpl(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType & result); +#define INVOKE(X) FOR_EACH_ARITHMETIC_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH - if constexpr (std::is_floating_point_v) - { - if (!std::isfinite(value)) - { - if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow. Cannot convert infinity or NaN to decimal", ToDataType::family_name); - else - return ReturnType(false); - } - - auto out = value * static_cast(DecimalUtils::scaleMultiplier(scale)); - - if (out <= static_cast(std::numeric_limits::min()) || - out >= static_cast(std::numeric_limits::max())) - { - if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow. Float is out of Decimal range", ToDataType::family_name); - else - return ReturnType(false); - } - - result = static_cast(out); - return ReturnType(true); - } - else - { - if constexpr (is_big_int_v) - return ReturnType(convertDecimalsImpl, ToDataType, ReturnType>(static_cast(value), 0, scale, result)); - else if constexpr (std::is_same_v) - return ReturnType(convertDecimalsImpl, ToDataType, ReturnType>(static_cast(value), 0, scale, result)); - else - return ReturnType(convertDecimalsImpl, ToDataType, ReturnType>(static_cast(value), 0, scale, result)); - } -} template requires (is_arithmetic_v && IsDataTypeDecimal) -inline typename ToDataType::FieldType convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale) -{ - typename ToDataType::FieldType result; - convertToDecimalImpl(value, scale, result); - return result; -} +typename ToDataType::FieldType convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale); + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template typename TO_DATA_TYPE::FieldType convertToDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale); +#define INVOKE(X) FOR_EACH_ARITHMETIC_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + template requires (is_arithmetic_v && IsDataTypeDecimal) -inline bool tryConvertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result) -{ - return convertToDecimalImpl(value, scale, result); -} +bool tryConvertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result); + +#define DISPATCH(FROM_DATA_TYPE, TO_DATA_TYPE) \ + extern template bool tryConvertToDecimal(const typename FROM_DATA_TYPE::FieldType & value, UInt32 scale, typename TO_DATA_TYPE::FieldType& result); +#define INVOKE(X) FOR_EACH_ARITHMETIC_TYPE_PASS(DISPATCH, X) +FOR_EACH_DECIMAL_TYPE(INVOKE); +#undef INVOKE +#undef DISPATCH + template -inline DataTypePtr createDecimalMaxPrecision(UInt64 scale) -{ - return std::make_shared>(DecimalUtils::max_precision, scale); -} +DataTypePtr createDecimalMaxPrecision(UInt64 scale); + +extern template DataTypePtr createDecimalMaxPrecision(UInt64 scale); +extern template DataTypePtr createDecimalMaxPrecision(UInt64 scale); +extern template DataTypePtr createDecimalMaxPrecision(UInt64 scale); +extern template DataTypePtr createDecimalMaxPrecision(UInt64 scale); + +extern template class DataTypeDecimal; +extern template class DataTypeDecimal; +extern template class DataTypeDecimal; +extern template class DataTypeDecimal; } diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index 008fa287064..99446d24eed 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -102,4 +102,21 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive); } +/// Explicit template instantiations. +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; + +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; +template class DataTypeNumber; + } diff --git a/src/DataTypes/DataTypesNumber.h b/src/DataTypes/DataTypesNumber.h index 0c1f88a7925..d550ceababc 100644 --- a/src/DataTypes/DataTypesNumber.h +++ b/src/DataTypes/DataTypesNumber.h @@ -55,6 +55,22 @@ private: bool unsigned_can_be_signed = false; }; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; + +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; +extern template class DataTypeNumber; + using DataTypeUInt8 = DataTypeNumber; using DataTypeUInt16 = DataTypeNumber; using DataTypeUInt32 = DataTypeNumber; diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 392c56343e3..40915418aea 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -267,4 +267,91 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column) return column.type->getDefaultSerialization(); } +#define FOR_TYPES_OF_TYPE(M) \ + M(TypeIndex) \ + M(const IDataType &) \ + M(const DataTypePtr &) \ + M(WhichDataType) + +#define DISPATCH(TYPE) \ +bool isUInt8(TYPE data_type) { return WhichDataType(data_type).isUInt8(); } \ +bool isUInt16(TYPE data_type) { return WhichDataType(data_type).isUInt16(); } \ +bool isUInt32(TYPE data_type) { return WhichDataType(data_type).isUInt32(); } \ +bool isUInt64(TYPE data_type) { return WhichDataType(data_type).isUInt64(); } \ +bool isNativeUInt(TYPE data_type) { return WhichDataType(data_type).isNativeUInt(); } \ +bool isUInt(TYPE data_type) { return WhichDataType(data_type).isUInt(); } \ +\ +bool isInt8(TYPE data_type) { return WhichDataType(data_type).isInt8(); } \ +bool isInt16(TYPE data_type) { return WhichDataType(data_type).isInt16(); } \ +bool isInt32(TYPE data_type) { return WhichDataType(data_type).isInt32(); } \ +bool isInt64(TYPE data_type) { return WhichDataType(data_type).isInt64(); } \ +bool isNativeInt(TYPE data_type) { return WhichDataType(data_type).isNativeInt(); } \ +bool isInt(TYPE data_type) { return WhichDataType(data_type).isInt(); } \ +\ +bool isInteger(TYPE data_type) { return WhichDataType(data_type).isInteger(); } \ +bool isNativeInteger(TYPE data_type) { return WhichDataType(data_type).isNativeInteger(); } \ +\ +bool isDecimal(TYPE data_type) { return WhichDataType(data_type).isDecimal(); } \ +\ +bool isFloat(TYPE data_type) { return WhichDataType(data_type).isFloat(); } \ +\ +bool isNativeNumber(TYPE data_type) { return WhichDataType(data_type).isNativeNumber(); } \ +bool isNumber(TYPE data_type) { return WhichDataType(data_type).isNumber(); } \ +\ +bool isEnum8(TYPE data_type) { return WhichDataType(data_type).isEnum8(); } \ +bool isEnum16(TYPE data_type) { return WhichDataType(data_type).isEnum16(); } \ +bool isEnum(TYPE data_type) { return WhichDataType(data_type).isEnum(); } \ +\ +bool isDate(TYPE data_type) { return WhichDataType(data_type).isDate(); } \ +bool isDate32(TYPE data_type) { return WhichDataType(data_type).isDate32(); } \ +bool isDateOrDate32(TYPE data_type) { return WhichDataType(data_type).isDateOrDate32(); } \ +bool isDateTime(TYPE data_type) { return WhichDataType(data_type).isDateTime(); } \ +bool isDateTime64(TYPE data_type) { return WhichDataType(data_type).isDateTime64(); } \ +bool isDateTimeOrDateTime64(TYPE data_type) { return WhichDataType(data_type).isDateTimeOrDateTime64(); } \ +bool isDateOrDate32OrDateTimeOrDateTime64(TYPE data_type) { return WhichDataType(data_type).isDateOrDate32OrDateTimeOrDateTime64(); } \ +\ +bool isString(TYPE data_type) { return WhichDataType(data_type).isString(); } \ +bool isFixedString(TYPE data_type) { return WhichDataType(data_type).isFixedString(); } \ +bool isStringOrFixedString(TYPE data_type) { return WhichDataType(data_type).isStringOrFixedString(); } \ +\ +bool isUUID(TYPE data_type) { return WhichDataType(data_type).isUUID(); } \ +bool isIPv4(TYPE data_type) { return WhichDataType(data_type).isIPv4(); } \ +bool isIPv6(TYPE data_type) { return WhichDataType(data_type).isIPv6(); } \ +bool isArray(TYPE data_type) { return WhichDataType(data_type).isArray(); } \ +bool isTuple(TYPE data_type) { return WhichDataType(data_type).isTuple(); } \ +bool isMap(TYPE data_type) {return WhichDataType(data_type).isMap(); } \ +bool isInterval(TYPE data_type) {return WhichDataType(data_type).isInterval(); } \ +bool isObject(TYPE data_type) { return WhichDataType(data_type).isObject(); } \ +bool isVariant(TYPE data_type) { return WhichDataType(data_type).isVariant(); } \ +bool isNothing(TYPE data_type) { return WhichDataType(data_type).isNothing(); } \ +\ +bool isColumnedAsNumber(TYPE data_type) \ +{ \ + WhichDataType which(data_type); \ + return which.isInteger() || which.isFloat() || which.isDateOrDate32OrDateTimeOrDateTime64() || which.isUUID() || which.isIPv4() || which.isIPv6(); \ +} \ +\ +bool isColumnedAsDecimal(TYPE data_type) \ +{ \ + WhichDataType which(data_type); \ + return which.isDecimal() || which.isDateTime64(); \ +} \ +\ +bool isNotCreatable(TYPE data_type) \ +{ \ + WhichDataType which(data_type); \ + return which.isNothing() || which.isFunction() || which.isSet(); \ +} \ +\ +bool isNotDecimalButComparableToDecimal(TYPE data_type) \ +{ \ + WhichDataType which(data_type); \ + return which.isInt() || which.isUInt() || which.isFloat(); \ +} \ + +FOR_TYPES_OF_TYPE(DISPATCH) + +#undef DISPATCH +#undef FOR_TYPES_OF_TYPE + } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 220658afda5..55f584ef1e0 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -424,71 +424,76 @@ struct WhichDataType /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) -template inline bool isUInt8(const T & data_type) { return WhichDataType(data_type).isUInt8(); } -template inline bool isUInt16(const T & data_type) { return WhichDataType(data_type).isUInt16(); } -template inline bool isUInt32(const T & data_type) { return WhichDataType(data_type).isUInt32(); } -template inline bool isUInt64(const T & data_type) { return WhichDataType(data_type).isUInt64(); } -template inline bool isNativeUInt(const T & data_type) { return WhichDataType(data_type).isNativeUInt(); } -template inline bool isUInt(const T & data_type) { return WhichDataType(data_type).isUInt(); } +#define FOR_TYPES_OF_TYPE(M) \ + M(TypeIndex) \ + M(const IDataType &) \ + M(const DataTypePtr &) \ + M(WhichDataType) -template inline bool isInt8(const T & data_type) { return WhichDataType(data_type).isInt8(); } -template inline bool isInt16(const T & data_type) { return WhichDataType(data_type).isInt16(); } -template inline bool isInt32(const T & data_type) { return WhichDataType(data_type).isInt32(); } -template inline bool isInt64(const T & data_type) { return WhichDataType(data_type).isInt64(); } -template inline bool isNativeInt(const T & data_type) { return WhichDataType(data_type).isNativeInt(); } -template inline bool isInt(const T & data_type) { return WhichDataType(data_type).isInt(); } +#define DISPATCH(TYPE) \ +bool isUInt8(TYPE data_type); \ +bool isUInt16(TYPE data_type); \ +bool isUInt32(TYPE data_type); \ +bool isUInt64(TYPE data_type); \ +bool isNativeUInt(TYPE data_type); \ +bool isUInt(TYPE data_type); \ +\ +bool isInt8(TYPE data_type); \ +bool isInt16(TYPE data_type); \ +bool isInt32(TYPE data_type); \ +bool isInt64(TYPE data_type); \ +bool isNativeInt(TYPE data_type); \ +bool isInt(TYPE data_type); \ +\ +bool isInteger(TYPE data_type); \ +bool isNativeInteger(TYPE data_type); \ +\ +bool isDecimal(TYPE data_type); \ +\ +bool isFloat(TYPE data_type); \ +\ +bool isNativeNumber(TYPE data_type); \ +bool isNumber(TYPE data_type); \ +\ +bool isEnum8(TYPE data_type); \ +bool isEnum16(TYPE data_type); \ +bool isEnum(TYPE data_type); \ +\ +bool isDate(TYPE data_type); \ +bool isDate32(TYPE data_type); \ +bool isDateOrDate32(TYPE data_type); \ +bool isDateTime(TYPE data_type); \ +bool isDateTime64(TYPE data_type); \ +bool isDateTimeOrDateTime64(TYPE data_type); \ +bool isDateOrDate32OrDateTimeOrDateTime64(TYPE data_type); \ +\ +bool isString(TYPE data_type); \ +bool isFixedString(TYPE data_type); \ +bool isStringOrFixedString(TYPE data_type); \ +\ +bool isUUID(TYPE data_type); \ +bool isIPv4(TYPE data_type); \ +bool isIPv6(TYPE data_type); \ +bool isArray(TYPE data_type); \ +bool isTuple(TYPE data_type); \ +bool isMap(TYPE data_type); \ +bool isInterval(TYPE data_type); \ +bool isObject(TYPE data_type); \ +bool isVariant(TYPE data_type); \ +bool isNothing(TYPE data_type); \ +\ +bool isColumnedAsNumber(TYPE data_type); \ +\ +bool isColumnedAsDecimal(TYPE data_type); \ +\ +bool isNotCreatable(TYPE data_type); \ +\ +bool isNotDecimalButComparableToDecimal(TYPE data_type); \ -template inline bool isInteger(const T & data_type) { return WhichDataType(data_type).isInteger(); } -template inline bool isNativeInteger(const T & data_type) { return WhichDataType(data_type).isNativeInteger(); } +FOR_TYPES_OF_TYPE(DISPATCH) -template inline bool isDecimal(const T & data_type) { return WhichDataType(data_type).isDecimal(); } - -template inline bool isFloat(const T & data_type) { return WhichDataType(data_type).isFloat(); } - -template inline bool isNativeNumber(const T & data_type) { return WhichDataType(data_type).isNativeNumber(); } -template inline bool isNumber(const T & data_type) { return WhichDataType(data_type).isNumber(); } - -template inline bool isEnum8(const T & data_type) { return WhichDataType(data_type).isEnum8(); } -template inline bool isEnum16(const T & data_type) { return WhichDataType(data_type).isEnum16(); } -template inline bool isEnum(const T & data_type) { return WhichDataType(data_type).isEnum(); } - -template inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); } -template inline bool isDate32(const T & data_type) { return WhichDataType(data_type).isDate32(); } -template inline bool isDateOrDate32(const T & data_type) { return WhichDataType(data_type).isDateOrDate32(); } -template inline bool isDateTime(const T & data_type) { return WhichDataType(data_type).isDateTime(); } -template inline bool isDateTime64(const T & data_type) { return WhichDataType(data_type).isDateTime64(); } -template inline bool isDateTimeOrDateTime64(const T & data_type) { return WhichDataType(data_type).isDateTimeOrDateTime64(); } -template inline bool isDateOrDate32OrDateTimeOrDateTime64(const T & data_type) { return WhichDataType(data_type).isDateOrDate32OrDateTimeOrDateTime64(); } - -template inline bool isString(const T & data_type) { return WhichDataType(data_type).isString(); } -template inline bool isFixedString(const T & data_type) { return WhichDataType(data_type).isFixedString(); } -template inline bool isStringOrFixedString(const T & data_type) { return WhichDataType(data_type).isStringOrFixedString(); } - -template inline bool isUUID(const T & data_type) { return WhichDataType(data_type).isUUID(); } -template inline bool isIPv4(const T & data_type) { return WhichDataType(data_type).isIPv4(); } -template inline bool isIPv6(const T & data_type) { return WhichDataType(data_type).isIPv6(); } -template inline bool isArray(const T & data_type) { return WhichDataType(data_type).isArray(); } -template inline bool isTuple(const T & data_type) { return WhichDataType(data_type).isTuple(); } -template inline bool isMap(const T & data_type) {return WhichDataType(data_type).isMap(); } -template inline bool isInterval(const T & data_type) {return WhichDataType(data_type).isInterval(); } -template inline bool isObject(const T & data_type) { return WhichDataType(data_type).isObject(); } -template inline bool isVariant(const T & data_type) { return WhichDataType(data_type).isVariant(); } - -template inline bool isNothing(const T & data_type) { return WhichDataType(data_type).isNothing(); } - -template -inline bool isColumnedAsNumber(const T & data_type) -{ - WhichDataType which(data_type); - return which.isInteger() || which.isFloat() || which.isDateOrDate32OrDateTimeOrDateTime64() || which.isUUID() || which.isIPv4() || which.isIPv6(); -} - -template -inline bool isColumnedAsDecimal(const T & data_type) -{ - WhichDataType which(data_type); - return which.isDecimal() || which.isDateTime64(); -} +#undef DISPATCH +#undef FOR_TYPES_OF_TYPE // Same as isColumnedAsDecimal but also checks value type of underlyig column. template @@ -498,19 +503,6 @@ inline bool isColumnedAsDecimalT(const DataType & data_type) return (which.isDecimal() || which.isDateTime64()) && which.idx == TypeToTypeIndex; } -template -inline bool isNotCreatable(const T & data_type) -{ - WhichDataType which(data_type); - return which.isNothing() || which.isFunction() || which.isSet(); -} - -inline bool isNotDecimalButComparableToDecimal(const DataTypePtr & data_type) -{ - WhichDataType which(data_type); - return which.isInt() || which.isUInt() || which.isFloat(); -} - inline bool isBool(const DataTypePtr & data_type) { return data_type->getName() == "Bool"; diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.h b/src/DataTypes/Serializations/SerializationDecimalBase.h index 08f963cedbb..5676280d34b 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.h +++ b/src/DataTypes/Serializations/SerializationDecimalBase.h @@ -29,4 +29,10 @@ public: void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; }; +extern template class SerializationDecimalBase; +extern template class SerializationDecimalBase; +extern template class SerializationDecimalBase; +extern template class SerializationDecimalBase; +extern template class SerializationDecimalBase; + } diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index de922747ccd..979c589c64b 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -100,7 +100,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_arguments{ - {"value", &isStringOrFixedString, nullptr, "String or FixedString"} + {"value", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_arguments); diff --git a/src/Functions/FunctionHelpers.h b/src/Functions/FunctionHelpers.h index 5619ebdae49..9f44d3e95c2 100644 --- a/src/Functions/FunctionHelpers.h +++ b/src/Functions/FunctionHelpers.h @@ -108,8 +108,10 @@ struct FunctionArgumentDescriptor { const char * argument_name; - std::function type_validator_func; - std::function column_validator_func; + using TypeValidator = bool (*)(const IDataType &); + TypeValidator type_validator_func; + using ColumnValidator = bool (*)(const IColumn &); + ColumnValidator column_validator_func; const char * expected_type_description; diff --git a/src/Functions/FunctionStringReplace.h b/src/Functions/FunctionStringReplace.h index 4d723a5632c..aee04a5969a 100644 --- a/src/Functions/FunctionStringReplace.h +++ b/src/Functions/FunctionStringReplace.h @@ -35,9 +35,9 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"haystack", &isStringOrFixedString, nullptr, "String or FixedString"}, - {"pattern", &isString, nullptr, "String"}, - {"replacement", &isString, nullptr, "String"} + {"haystack", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"}, + {"pattern", static_cast(&isString), nullptr, "String"}, + {"replacement", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/FunctionTokens.h b/src/Functions/FunctionTokens.h index 5c4e582c637..ddf10b863ac 100644 --- a/src/Functions/FunctionTokens.h +++ b/src/Functions/FunctionTokens.h @@ -184,12 +184,12 @@ static inline void checkArgumentsWithSeparatorAndOptionalMaxSubstrings( const IFunction & func, const ColumnsWithTypeAndName & arguments) { FunctionArgumentDescriptors mandatory_args{ - {"separator", &isString, isColumnConst, "const String"}, - {"s", &isString, nullptr, "String"} + {"separator", static_cast(&isString), isColumnConst, "const String"}, + {"s", static_cast(&isString), nullptr, "String"} }; FunctionArgumentDescriptors optional_args{ - {"max_substrings", &isNativeInteger, isColumnConst, "const Number"}, + {"max_substrings", static_cast(&isNativeInteger), isColumnConst, "const Number"}, }; validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args); @@ -198,11 +198,11 @@ static inline void checkArgumentsWithSeparatorAndOptionalMaxSubstrings( static inline void checkArgumentsWithOptionalMaxSubstrings(const IFunction & func, const ColumnsWithTypeAndName & arguments) { FunctionArgumentDescriptors mandatory_args{ - {"s", &isString, nullptr, "String"}, + {"s", static_cast(&isString), nullptr, "String"}, }; FunctionArgumentDescriptors optional_args{ - {"max_substrings", &isNativeInteger, isColumnConst, "const Number"}, + {"max_substrings", static_cast(&isNativeInteger), isColumnConst, "const Number"}, }; validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args); diff --git a/src/Functions/FunctionUnixTimestamp64.h b/src/Functions/FunctionUnixTimestamp64.h index d74237afd77..53421a565cb 100644 --- a/src/Functions/FunctionUnixTimestamp64.h +++ b/src/Functions/FunctionUnixTimestamp64.h @@ -45,7 +45,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"value", &isDateTime64, nullptr, "DateTime64"} + {"value", static_cast(&isDateTime64), nullptr, "DateTime64"} }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index 4792c997f51..a03f0b602b9 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -154,21 +154,21 @@ private: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { auto optional_args = FunctionArgumentDescriptors{ - {"IV", &isStringOrFixedString, nullptr, "Initialization vector binary string"}, + {"IV", static_cast(&isStringOrFixedString), nullptr, "Initialization vector binary string"}, }; if constexpr (compatibility_mode == OpenSSLDetails::CompatibilityMode::OpenSSL) { optional_args.emplace_back(FunctionArgumentDescriptor{ - "AAD", &isStringOrFixedString, nullptr, "Additional authenticated data binary string for GCM mode" + "AAD", static_cast(&isStringOrFixedString), nullptr, "Additional authenticated data binary string for GCM mode" }); } validateFunctionArgumentTypes(*this, arguments, FunctionArgumentDescriptors{ - {"mode", &isStringOrFixedString, isColumnConst, "encryption mode string"}, - {"input", &isStringOrFixedString, {}, "plaintext"}, - {"key", &isStringOrFixedString, {}, "encryption key binary string"}, + {"mode", static_cast(&isStringOrFixedString), isColumnConst, "encryption mode string"}, + {"input", static_cast(&isStringOrFixedString), {}, "plaintext"}, + {"key", static_cast(&isStringOrFixedString), {}, "encryption key binary string"}, }, optional_args ); @@ -425,21 +425,21 @@ private: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { auto optional_args = FunctionArgumentDescriptors{ - {"IV", &isStringOrFixedString, nullptr, "Initialization vector binary string"}, + {"IV", static_cast(&isStringOrFixedString), nullptr, "Initialization vector binary string"}, }; if constexpr (compatibility_mode == OpenSSLDetails::CompatibilityMode::OpenSSL) { optional_args.emplace_back(FunctionArgumentDescriptor{ - "AAD", &isStringOrFixedString, nullptr, "Additional authenticated data binary string for GCM mode" + "AAD", static_cast(&isStringOrFixedString), nullptr, "Additional authenticated data binary string for GCM mode" }); } validateFunctionArgumentTypes(*this, arguments, FunctionArgumentDescriptors{ - {"mode", &isStringOrFixedString, isColumnConst, "decryption mode string"}, - {"input", &isStringOrFixedString, {}, "ciphertext"}, - {"key", &isStringOrFixedString, {}, "decryption key binary string"}, + {"mode", static_cast(&isStringOrFixedString), isColumnConst, "decryption mode string"}, + {"input", static_cast(&isStringOrFixedString), {}, "ciphertext"}, + {"key", static_cast(&isStringOrFixedString), {}, "decryption key binary string"}, }, optional_args ); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 19647e2f086..1522e76893e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2129,12 +2129,12 @@ public: if constexpr (to_decimal) { - mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); + mandatory_args.push_back({"scale", static_cast(&isNativeInteger), &isColumnConst, "const Integer"}); } if (!to_decimal && isDateTime64(arguments)) { - mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); + mandatory_args.push_back({"scale", static_cast(&isNativeInteger), &isColumnConst, "const Integer"}); } // toString(DateTime or DateTime64, [timezone: String]) @@ -2150,7 +2150,7 @@ public: // toDateTime64(value, scale : Integer[, timezone: String]) || std::is_same_v) { - optional_args.push_back({"timezone", &isString, nullptr, "String"}); + optional_args.push_back({"timezone", static_cast(&isString), nullptr, "String"}); } validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -2498,11 +2498,11 @@ public: if (isDateTime64(arguments)) { validateFunctionArgumentTypes(*this, arguments, - FunctionArgumentDescriptors{{"string", &isStringOrFixedString, nullptr, "String or FixedString"}}, + FunctionArgumentDescriptors{{"string", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"}}, // optional FunctionArgumentDescriptors{ - {"precision", &isUInt8, isColumnConst, "const UInt8"}, - {"timezone", &isStringOrFixedString, isColumnConst, "const String or FixedString"}, + {"precision", static_cast(&isUInt8), isColumnConst, "const UInt8"}, + {"timezone", static_cast(&isStringOrFixedString), isColumnConst, "const String or FixedString"}, }); UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; diff --git a/src/Functions/JSONArrayLength.cpp b/src/Functions/JSONArrayLength.cpp index a82c50360f9..84e87061398 100644 --- a/src/Functions/JSONArrayLength.cpp +++ b/src/Functions/JSONArrayLength.cpp @@ -45,7 +45,7 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { auto args = FunctionArgumentDescriptors{ - {"json", &isString, nullptr, "String"}, + {"json", static_cast(&isString), nullptr, "String"}, }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/URL/URLHierarchy.cpp b/src/Functions/URL/URLHierarchy.cpp index 25c6c9ef40b..bb39566c342 100644 --- a/src/Functions/URL/URLHierarchy.cpp +++ b/src/Functions/URL/URLHierarchy.cpp @@ -27,7 +27,7 @@ public: static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) { FunctionArgumentDescriptors mandatory_args{ - {"URL", &isString, nullptr, "String"}, + {"URL", static_cast(&isString), nullptr, "String"}, }; validateFunctionArgumentTypes(func, arguments, mandatory_args); diff --git a/src/Functions/URL/URLPathHierarchy.cpp b/src/Functions/URL/URLPathHierarchy.cpp index 9a60d4cf989..9f5b0031eeb 100644 --- a/src/Functions/URL/URLPathHierarchy.cpp +++ b/src/Functions/URL/URLPathHierarchy.cpp @@ -25,7 +25,7 @@ public: static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) { FunctionArgumentDescriptors mandatory_args{ - {"URL", &isString, nullptr, "String"}, + {"URL", static_cast(&isString), nullptr, "String"}, }; validateFunctionArgumentTypes(func, arguments, mandatory_args); diff --git a/src/Functions/URL/extractURLParameterNames.cpp b/src/Functions/URL/extractURLParameterNames.cpp index 08da148b43e..ee2eb25ae9d 100644 --- a/src/Functions/URL/extractURLParameterNames.cpp +++ b/src/Functions/URL/extractURLParameterNames.cpp @@ -25,7 +25,7 @@ public: static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) { FunctionArgumentDescriptors mandatory_args{ - {"URL", &isString, nullptr, "String"}, + {"URL", static_cast(&isString), nullptr, "String"}, }; validateFunctionArgumentTypes(func, arguments, mandatory_args); diff --git a/src/Functions/URL/extractURLParameters.cpp b/src/Functions/URL/extractURLParameters.cpp index 939622dd9d1..93f349acb06 100644 --- a/src/Functions/URL/extractURLParameters.cpp +++ b/src/Functions/URL/extractURLParameters.cpp @@ -26,7 +26,7 @@ public: static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) { FunctionArgumentDescriptors mandatory_args{ - {"URL", &isString, nullptr, "String"}, + {"URL", static_cast(&isString), nullptr, "String"}, }; validateFunctionArgumentTypes(func, arguments, mandatory_args); diff --git a/src/Functions/array/arrayJaccardIndex.cpp b/src/Functions/array/arrayJaccardIndex.cpp index c2a4fee4845..9cb74a7aa62 100644 --- a/src/Functions/array/arrayJaccardIndex.cpp +++ b/src/Functions/array/arrayJaccardIndex.cpp @@ -84,8 +84,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"array_1", &isArray, nullptr, "Array"}, - {"array_2", &isArray, nullptr, "Array"}, + {"array_1", static_cast(&isArray), nullptr, "Array"}, + {"array_2", static_cast(&isArray), nullptr, "Array"}, }; validateFunctionArgumentTypes(*this, arguments, args); return std::make_shared>(); diff --git a/src/Functions/array/arrayRandomSample.cpp b/src/Functions/array/arrayRandomSample.cpp index 40344efb077..b08a73b93f3 100644 --- a/src/Functions/array/arrayRandomSample.cpp +++ b/src/Functions/array/arrayRandomSample.cpp @@ -36,8 +36,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"array", &isArray, nullptr, "Array"}, - {"samples", &isUInt, isColumnConst, "const UInt*"}, + {"array", static_cast(&isArray), nullptr, "Array"}, + {"samples", static_cast(&isUInt), isColumnConst, "const UInt*"}, }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/array/arrayShingles.cpp b/src/Functions/array/arrayShingles.cpp index ade1cb862f7..8932482c69c 100644 --- a/src/Functions/array/arrayShingles.cpp +++ b/src/Functions/array/arrayShingles.cpp @@ -28,8 +28,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"array", &isArray, nullptr, "Array"}, - {"length", &isInteger, nullptr, "Integer"} + {"array", static_cast(&isArray), nullptr, "Array"}, + {"length", static_cast(&isInteger), nullptr, "Integer"} }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/arrayStringConcat.cpp b/src/Functions/arrayStringConcat.cpp index 0194cc4871a..c186c0ca7e6 100644 --- a/src/Functions/arrayStringConcat.cpp +++ b/src/Functions/arrayStringConcat.cpp @@ -151,12 +151,12 @@ public: { FunctionArgumentDescriptors mandatory_args { - {"arr", &isArray, nullptr, "Array"}, + {"arr", static_cast(&isArray), nullptr, "Array"}, }; FunctionArgumentDescriptors optional_args { - {"separator", &isString, isColumnConst, "const String"}, + {"separator", static_cast(&isString), isColumnConst, "const String"}, }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); diff --git a/src/Functions/castOrDefault.cpp b/src/Functions/castOrDefault.cpp index 26eaf4f5613..970e6fd6f75 100644 --- a/src/Functions/castOrDefault.cpp +++ b/src/Functions/castOrDefault.cpp @@ -210,10 +210,10 @@ private: FunctionArgumentDescriptors optional_args; if constexpr (IsDataTypeDecimal) - mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); + mandatory_args.push_back({"scale", static_cast(&isNativeInteger), &isColumnConst, "const Integer"}); if (std::is_same_v || std::is_same_v) - optional_args.push_back({"timezone", &isString, isColumnConst, "const String"}); + optional_args.push_back({"timezone", static_cast(&isString), isColumnConst, "const String"}); optional_args.push_back({"default_value", nullptr, nullptr, nullptr}); diff --git a/src/Functions/countMatches.h b/src/Functions/countMatches.h index e9880e6e93f..fbbb9d017ee 100644 --- a/src/Functions/countMatches.h +++ b/src/Functions/countMatches.h @@ -35,8 +35,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"haystack", &isStringOrFixedString, nullptr, "String or FixedString"}, - {"pattern", &isString, isColumnConst, "constant String"} + {"haystack", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"}, + {"pattern", static_cast(&isString), isColumnConst, "constant String"} }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/extractAll.cpp b/src/Functions/extractAll.cpp index ad49f32f769..94c915f8e38 100644 --- a/src/Functions/extractAll.cpp +++ b/src/Functions/extractAll.cpp @@ -53,8 +53,8 @@ public: static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) { FunctionArgumentDescriptors mandatory_args{ - {"haystack", &isString, nullptr, "String"}, - {"pattern", &isString, isColumnConst, "const String"} + {"haystack", static_cast(&isString), nullptr, "String"}, + {"pattern", static_cast(&isString), isColumnConst, "const String"} }; validateFunctionArgumentTypes(func, arguments, mandatory_args); diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index c64c9d6ccef..ac12cad1698 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -71,8 +71,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"haystack", &isStringOrFixedString, nullptr, "const String or const FixedString"}, - {"needle", &isStringOrFixedString, isColumnConst, "const String or const FixedString"}, + {"haystack", static_cast(&isStringOrFixedString), nullptr, "const String or const FixedString"}, + {"needle", static_cast(&isStringOrFixedString), isColumnConst, "const String or const FixedString"}, }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/extractGroups.cpp b/src/Functions/extractGroups.cpp index e22938f8565..f62352af0bd 100644 --- a/src/Functions/extractGroups.cpp +++ b/src/Functions/extractGroups.cpp @@ -45,8 +45,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"haystack", &isStringOrFixedString, nullptr, "const String or const FixedString"}, - {"needle", &isStringOrFixedString, isColumnConst, "const String or const FixedString"}, + {"haystack", static_cast(&isStringOrFixedString), nullptr, "const String or const FixedString"}, + {"needle", static_cast(&isStringOrFixedString), isColumnConst, "const String or const FixedString"}, }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index 2f6bc6f9903..92403d2e88e 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -54,7 +54,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"query", &isString, nullptr, "String"} + {"query", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/fromDaysSinceYearZero.cpp b/src/Functions/fromDaysSinceYearZero.cpp index a21d0cc25bf..b98c587d172 100644 --- a/src/Functions/fromDaysSinceYearZero.cpp +++ b/src/Functions/fromDaysSinceYearZero.cpp @@ -52,7 +52,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - FunctionArgumentDescriptors args{{"days", &isNativeInteger, nullptr, "Integer"}}; + FunctionArgumentDescriptors args{{"days", static_cast(&isNativeInteger), nullptr, "Integer"}}; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 987cf4eb1a9..c7f3c195578 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -82,17 +82,17 @@ public: if (is_year_month_variant) { FunctionArgumentDescriptors args{ - {mandatory_argument_names_year_month_day[0], &isNumber, nullptr, "Number"}, - {mandatory_argument_names_year_month_day[1], &isNumber, nullptr, "Number"}, - {mandatory_argument_names_year_month_day[2], &isNumber, nullptr, "Number"} + {mandatory_argument_names_year_month_day[0], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names_year_month_day[1], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names_year_month_day[2], static_cast(&isNumber), nullptr, "Number"} }; validateFunctionArgumentTypes(*this, arguments, args); } else { FunctionArgumentDescriptors args{ - {mandatory_argument_names_year_dayofyear[0], &isNumber, nullptr, "Number"}, - {mandatory_argument_names_year_dayofyear[1], &isNumber, nullptr, "Number"} + {mandatory_argument_names_year_dayofyear[0], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names_year_dayofyear[1], static_cast(&isNumber), nullptr, "Number"} }; validateFunctionArgumentTypes(*this, arguments, args); } @@ -189,7 +189,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {mandatory_argument_names[0], &isNumber, nullptr, "Number"} + {mandatory_argument_names[0], static_cast(&isNumber), nullptr, "Number"} }; validateFunctionArgumentTypes(*this, arguments, args); @@ -344,16 +344,16 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {mandatory_argument_names[0], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[1], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[2], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[3], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[4], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[5], &isNumber, nullptr, "Number"} + {mandatory_argument_names[0], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[1], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[2], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[3], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[4], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[5], static_cast(&isNumber), nullptr, "Number"} }; FunctionArgumentDescriptors optional_args{ - {optional_argument_names[0], &isString, isColumnConst, "const String"} + {optional_argument_names[0], static_cast(&isString), isColumnConst, "const String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -425,18 +425,18 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {mandatory_argument_names[0], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[1], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[2], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[3], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[4], &isNumber, nullptr, "Number"}, - {mandatory_argument_names[5], &isNumber, nullptr, "Number"} + {mandatory_argument_names[0], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[1], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[2], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[3], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[4], static_cast(&isNumber), nullptr, "Number"}, + {mandatory_argument_names[5], static_cast(&isNumber), nullptr, "Number"} }; FunctionArgumentDescriptors optional_args{ - {optional_argument_names[0], &isNumber, nullptr, "const Number"}, - {optional_argument_names[1], &isNumber, isColumnConst, "const Number"}, - {optional_argument_names[2], &isString, isColumnConst, "const String"} + {optional_argument_names[0], static_cast(&isNumber), nullptr, "const Number"}, + {optional_argument_names[1], static_cast(&isNumber), isColumnConst, "const Number"}, + {optional_argument_names[2], static_cast(&isString), isColumnConst, "const String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -564,11 +564,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {mandatory_argument_names[0], &isNumber, nullptr, "Number"} + {mandatory_argument_names[0], static_cast(&isNumber), nullptr, "Number"} }; FunctionArgumentDescriptors optional_args{ - {optional_argument_names[0], &isString, isColumnConst, "const String"} + {optional_argument_names[0], static_cast(&isString), isColumnConst, "const String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -643,12 +643,12 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {mandatory_argument_names[0], &isNumber, nullptr, "Number"} + {mandatory_argument_names[0], static_cast(&isNumber), nullptr, "Number"} }; FunctionArgumentDescriptors optional_args{ - {optional_argument_names[0], &isNumber, isColumnConst, "const Number"}, - {optional_argument_names[0], &isString, isColumnConst, "const String"} + {optional_argument_names[0], static_cast(&isNumber), isColumnConst, "const Number"}, + {optional_argument_names[0], static_cast(&isString), isColumnConst, "const String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 860603dc503..18882177c90 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -489,12 +489,12 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {"time", &isString, nullptr, "String"}, - {"format", &isString, nullptr, "String"} + {"time", static_cast(&isString), nullptr, "String"}, + {"format", static_cast(&isString), nullptr, "String"} }; FunctionArgumentDescriptors optional_args{ - {"timezone", &isString, &isColumnConst, "const String"} + {"timezone", static_cast(&isString), &isColumnConst, "const String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); diff --git a/src/Functions/regexpExtract.cpp b/src/Functions/regexpExtract.cpp index f6bbd2f96f2..cfb42580cb0 100644 --- a/src/Functions/regexpExtract.cpp +++ b/src/Functions/regexpExtract.cpp @@ -47,12 +47,12 @@ public: arguments.size()); FunctionArgumentDescriptors args{ - {"haystack", &isString, nullptr, "String"}, - {"pattern", &isString, isColumnConst, "const String"}, + {"haystack", static_cast(&isString), nullptr, "String"}, + {"pattern", static_cast(&isString), isColumnConst, "const String"}, }; if (arguments.size() == 3) - args.emplace_back(FunctionArgumentDescriptor{"index", &isInteger, nullptr, "Integer"}); + args.emplace_back(FunctionArgumentDescriptor{"index", static_cast(&isInteger), nullptr, "Integer"}); validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index c1b553ac6b3..11a2ca37a3b 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -186,8 +186,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"s", &isString, nullptr, "String"}, - {"n", &isInteger, nullptr, "Integer"}, + {"s", static_cast(&isString), nullptr, "String"}, + {"n", static_cast(&isInteger), nullptr, "Integer"}, }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index fbabc801913..618808b64ed 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -42,8 +42,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"time_series", &isArray, nullptr, "Array"}, - {"period", &isNativeUInt, nullptr, "Unsigned Integer"}, + {"time_series", static_cast(&isArray), nullptr, "Array"}, + {"period", static_cast(&isNativeUInt), nullptr, "Unsigned Integer"}, }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/seriesOutliersDetectTukey.cpp b/src/Functions/seriesOutliersDetectTukey.cpp index 5bc8edf3a54..da04d3b78d3 100644 --- a/src/Functions/seriesOutliersDetectTukey.cpp +++ b/src/Functions/seriesOutliersDetectTukey.cpp @@ -45,11 +45,11 @@ public: getName(), arguments.size()); - FunctionArgumentDescriptors mandatory_args{{"time_series", &isArray, nullptr, "Array"}}; + FunctionArgumentDescriptors mandatory_args{{"time_series", static_cast(&isArray), nullptr, "Array"}}; FunctionArgumentDescriptors optional_args{ - {"min_percentile", &isFloat, isColumnConst, "Number"}, - {"max_percentile", &isFloat, isColumnConst, "Number"}, - {"k", &isNativeNumber, isColumnConst, "Number"}}; + {"min_percentile", static_cast(&isFloat), isColumnConst, "Number"}, + {"max_percentile", static_cast(&isFloat), isColumnConst, "Number"}, + {"k", static_cast(&isNativeNumber), isColumnConst, "Number"}}; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); diff --git a/src/Functions/seriesPeriodDetectFFT.cpp b/src/Functions/seriesPeriodDetectFFT.cpp index c01f6b7f07b..fbaa2b14e64 100644 --- a/src/Functions/seriesPeriodDetectFFT.cpp +++ b/src/Functions/seriesPeriodDetectFFT.cpp @@ -52,7 +52,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - FunctionArgumentDescriptors args{{"time_series", &isArray, nullptr, "Array"}}; + FunctionArgumentDescriptors args{{"time_series", static_cast(&isArray), nullptr, "Array"}}; validateFunctionArgumentTypes(*this, arguments, args); return std::make_shared(); diff --git a/src/Functions/snowflake.cpp b/src/Functions/snowflake.cpp index 6aafa2cb5cf..f2dd1f1c51d 100644 --- a/src/Functions/snowflake.cpp +++ b/src/Functions/snowflake.cpp @@ -47,7 +47,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"value", &isDateTime, nullptr, "DateTime"} + {"value", static_cast(&isDateTime), nullptr, "DateTime"} }; validateFunctionArgumentTypes(*this, arguments, args); @@ -91,10 +91,10 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {"value", &isInt64, nullptr, "Int64"} + {"value", static_cast(&isInt64), nullptr, "Int64"} }; FunctionArgumentDescriptors optional_args{ - {"time_zone", &isString, nullptr, "String"} + {"time_zone", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -151,7 +151,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"value", &isDateTime64, nullptr, "DateTime64"} + {"value", static_cast(&isDateTime64), nullptr, "DateTime64"} }; validateFunctionArgumentTypes(*this, arguments, args); @@ -203,10 +203,10 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {"value", &isInt64, nullptr, "Int64"} + {"value", static_cast(&isInt64), nullptr, "Int64"} }; FunctionArgumentDescriptors optional_args{ - {"time_zone", &isString, nullptr, "String"} + {"time_zone", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 009bc20e065..03dc0d06719 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -45,7 +45,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"n", &isInteger, nullptr, "Integer"} + {"n", static_cast(&isInteger), nullptr, "Integer"} }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index cd3875e2607..a052f20d6fa 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -98,7 +98,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors args{ - {"sqid", &isString, nullptr, "String"} + {"sqid", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, args); diff --git a/src/Functions/timestamp.cpp b/src/Functions/timestamp.cpp index 48012c1376f..fbca08b0968 100644 --- a/src/Functions/timestamp.cpp +++ b/src/Functions/timestamp.cpp @@ -41,10 +41,10 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {"timestamp", &isStringOrFixedString, nullptr, "String or FixedString"} + {"timestamp", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"} }; FunctionArgumentDescriptors optional_args{ - {"time", &isString, nullptr, "String"} + {"time", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); diff --git a/src/Functions/toDecimalString.cpp b/src/Functions/toDecimalString.cpp index cc2de8df0d4..fc621b272de 100644 --- a/src/Functions/toDecimalString.cpp +++ b/src/Functions/toDecimalString.cpp @@ -39,8 +39,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args = { - {"Value", &isNumber, nullptr, "Number"}, - {"precision", &isNativeInteger, &isColumnConst, "const Integer"} + {"Value", static_cast(&isNumber), nullptr, "Number"}, + {"precision", static_cast(&isNativeInteger), &isColumnConst, "const Integer"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, {}); diff --git a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp index c2ee5923c01..3578401a0f8 100644 --- a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp @@ -12,6 +12,7 @@ #include #include +#include "DataTypes/IDataType.h" #include #include @@ -35,9 +36,12 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + constexpr auto FORMAT_NAME = "Prometheus"; -static bool isDataTypeMapString(const DataTypePtr & type) +bool isDataTypeMapString(const DataTypePtr & type) { if (!isMap(type)) return false; @@ -45,8 +49,8 @@ static bool isDataTypeMapString(const DataTypePtr & type) return isStringOrFixedString(type_map->getKeyType()) && isStringOrFixedString(type_map->getValueType()); } -template -static void getColumnPos(const Block & header, const String & col_name, Pred pred, ResType & res) +template +void getColumnPos(const Block & header, const String & col_name, bool (*pred)(const DataTypePtr &), ResType & res) { static_assert(std::is_same_v || std::is_same_v>, "Illegal ResType"); @@ -71,7 +75,7 @@ static void getColumnPos(const Block & header, const String & col_name, Pred pre } } -static Float64 tryParseFloat(const String & s) +Float64 tryParseFloat(const String & s) { Float64 t = 0; ReadBufferFromString buf(s); @@ -79,6 +83,8 @@ static Float64 tryParseFloat(const String & s) return t; } +} + PrometheusTextOutputFormat::PrometheusTextOutputFormat( WriteBuffer & out_, const Block & header_, @@ -89,12 +95,12 @@ PrometheusTextOutputFormat::PrometheusTextOutputFormat( { const Block & header = getPort(PortKind::Main).getHeader(); - getColumnPos(header, "name", isStringOrFixedString, pos.name); - getColumnPos(header, "value", isNumber, pos.value); + getColumnPos(header, "name", isStringOrFixedString, pos.name); + getColumnPos(header, "value", isNumber, pos.value); - getColumnPos(header, "help", isStringOrFixedString, pos.help); - getColumnPos(header, "type", isStringOrFixedString, pos.type); - getColumnPos(header, "timestamp", isNumber, pos.timestamp); + getColumnPos(header, "help", isStringOrFixedString, pos.help); + getColumnPos(header, "type", isStringOrFixedString, pos.type); + getColumnPos(header, "timestamp", isNumber, pos.timestamp); getColumnPos(header, "labels", isDataTypeMapString, pos.labels); } From 0f166baf6be838840541fa31031395dce0cdf0ea Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Mar 2024 17:44:27 +0000 Subject: [PATCH 196/197] Fast fix tests. --- .../02949_parallel_replicas_in_subquery.sql | 16 ++++++++-------- ...arallel_replicas_joins_and_analyzer.reference | 8 ++++---- ...7_parallel_replicas_joins_and_analyzer.sql.j2 | 4 ++-- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql index 9000d37c801..ab6e1532299 100644 --- a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql +++ b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql @@ -8,23 +8,23 @@ INSERT INTO merge_tree_in_subqueries VALUES(5, 'test5', 0); SET max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost', parallel_replicas_for_non_replicated_merge_tree=1; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; -- { serverError SUPPORT_IS_DISABLED } -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED } +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=1; SELECT '---'; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; -- { serverError SUPPORT_IS_DISABLED }; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=1; SELECT '---'; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; -- { serverError SUPPORT_IS_DISABLED }; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=1; -- IN with tuples is allowed SELECT '---'; -SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=0; -SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_subqueries_for_in=1; +SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; +SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; DROP TABLE IF EXISTS merge_tree_in_subqueries; diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index edd99058bd9..100e4e500cd 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -294,7 +294,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -317,7 +317,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0;-- { echoOn } +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0;-- { echoOn } Expression Sorting Expression @@ -631,7 +631,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -654,7 +654,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; Expression Sorting Expression diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 index e0de8c64950..54505b147a3 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 @@ -153,7 +153,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; explain description=0 with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -162,6 +162,6 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_subqueries_for_in=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; {%- endfor %} From 8e3fc1044f20edfbfa270ba34360411e02d669e2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 7 Mar 2024 20:39:56 +0100 Subject: [PATCH 197/197] Reject INSERT if `async_insert=1` + `deduplicate_blocks_in_dependent_materialized_views=1` (#60888) --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Interpreters/executeQuery.cpp | 16 ++++++++++++++++ src/Server/TCPHandler.cpp | 15 +++++++++++++++ ...uplication_throw_if_async_insert.reference | 0 ...mv_deduplication_throw_if_async_insert.sql | 19 +++++++++++++++++++ 6 files changed, 52 insertions(+) create mode 100644 tests/queries/0_stateless/03006_mv_deduplication_throw_if_async_insert.reference create mode 100644 tests/queries/0_stateless/03006_mv_deduplication_throw_if_async_insert.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9f22d35bb9e..c8bdb515baf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -598,6 +598,7 @@ class IColumn; M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there are constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ + M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index e7b96cee9d3..2f1da7935e6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -90,6 +90,7 @@ static std::map sett {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, + {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index f318f363eda..88021038ebb 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -104,6 +104,7 @@ namespace ErrorCodes extern const int QUERY_WAS_CANCELLED; extern const int INCORRECT_DATA; extern const int SYNTAX_ERROR; + extern const int SUPPORT_IS_DISABLED; extern const int INCORRECT_QUERY; } @@ -1023,6 +1024,21 @@ static std::tuple executeQueryImpl( if (settings.implicit_transaction && settings.throw_on_unsupported_query_inside_transaction) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Async inserts with 'implicit_transaction' are not supported"); + /// Let's agree on terminology and say that a mini-INSERT is an asynchronous INSERT + /// which typically contains not a lot of data inside and a big-INSERT in an INSERT + /// which was formed by concatenating several mini-INSERTs together. + /// In case when the client had to retry some mini-INSERTs then they will be properly deduplicated + /// by the source tables. This functionality is controlled by a setting `async_insert_deduplicate`. + /// But then they will be glued together into a block and pushed through a chain of Materialized Views if any. + /// The process of forming such blocks is not deteministic so each time we retry mini-INSERTs the resulting + /// block may be concatenated differently. + /// That's why deduplication in dependent Materialized Views doesn't make sense in presence of async INSERTs. + if (settings.throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert && + settings.deduplicate_blocks_in_dependent_materialized_views) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Deduplication is dependent materialized view cannot work together with async inserts. "\ + "Please disable eiher `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting."); + quota = context->getQuota(); if (quota) { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d883029408c..02bfd1d8359 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -933,6 +933,21 @@ void TCPHandler::processInsertQuery() if (insert_queue && async_insert_enabled && !insert_query.select) { + /// Let's agree on terminology and say that a mini-INSERT is an asynchronous INSERT + /// which typically contains not a lot of data inside and a big-INSERT in an INSERT + /// which was formed by concatenating several mini-INSERTs together. + /// In case when the client had to retry some mini-INSERTs then they will be properly deduplicated + /// by the source tables. This functionality is controlled by a setting `async_insert_deduplicate`. + /// But then they will be glued together into a block and pushed through a chain of Materialized Views if any. + /// The process of forming such blocks is not deteministic so each time we retry mini-INSERTs the resulting + /// block may be concatenated differently. + /// That's why deduplication in dependent Materialized Views doesn't make sense in presence of async INSERTs. + if (settings.throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert && + settings.deduplicate_blocks_in_dependent_materialized_views) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Deduplication is dependent materialized view cannot work together with async inserts. "\ + "Please disable eiher `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting."); + auto result = processAsyncInsertQuery(*insert_queue); if (result.status == AsynchronousInsertQueue::PushResult::OK) { diff --git a/tests/queries/0_stateless/03006_mv_deduplication_throw_if_async_insert.reference b/tests/queries/0_stateless/03006_mv_deduplication_throw_if_async_insert.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03006_mv_deduplication_throw_if_async_insert.sql b/tests/queries/0_stateless/03006_mv_deduplication_throw_if_async_insert.sql new file mode 100644 index 00000000000..808317c917e --- /dev/null +++ b/tests/queries/0_stateless/03006_mv_deduplication_throw_if_async_insert.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS 02985_test; + +SET async_insert = 1; +SET deduplicate_blocks_in_dependent_materialized_views = 1; + +CREATE TABLE 03006_test +( + d Date, + value UInt64 +) +ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO 03006_test VALUES ('2024-03-05', 1), ('2024-03-05', 2), ('2024-03-05', 1); -- { serverError SUPPORT_IS_DISABLED } +INSERT INTO 03006_test SETTINGS compatibility='24.1' VALUES ('2024-03-05', 1), ('2024-03-05', 2), ('2024-03-05', 1); +INSERT INTO 03006_test SETTINGS async_insert=0 VALUES ('2024-03-05', 1), ('2024-03-05', 2), ('2024-03-05', 1); +INSERT INTO 03006_test SETTINGS deduplicate_blocks_in_dependent_materialized_views=0 VALUES ('2024-03-05', 1), ('2024-03-05', 2), ('2024-03-05', 1); +INSERT INTO 03006_test SETTINGS throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert=0 VALUES ('2024-03-05', 1), ('2024-03-05', 2), ('2024-03-05', 1); + +DROP TABLE IF EXISTS 02985_test;