From 55f3e7e65a76feaf9c5566f416d45d4fcd0d28f6 Mon Sep 17 00:00:00 2001 From: terrylin Date: Fri, 30 Jul 2021 12:27:15 +0800 Subject: [PATCH 001/127] improvement of materilize ttl --- src/DataStreams/TTLBlockInputStream.cpp | 4 +- src/DataStreams/TTLCalcInputStream.cpp | 94 +++++++++++++++++++ src/DataStreams/TTLCalcInputStream.h | 44 +++++++++ src/DataStreams/TTLUpdateInfoAlgorithm.cpp | 41 ++++---- src/DataStreams/TTLUpdateInfoAlgorithm.h | 14 --- src/Interpreters/MutationsInterpreter.cpp | 71 ++++---------- src/Interpreters/MutationsInterpreter.h | 16 ---- .../MergeTree/MergeTreeDataMergerMutator.cpp | 47 ++++------ src/Storages/StorageInMemoryMetadata.cpp | 10 +- src/Storages/StorageInMemoryMetadata.h | 2 +- 10 files changed, 210 insertions(+), 133 deletions(-) create mode 100644 src/DataStreams/TTLCalcInputStream.cpp create mode 100644 src/DataStreams/TTLCalcInputStream.h diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 8b31da6d2f1..2cf7c121868 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -81,11 +81,11 @@ TTLBlockInputStream::TTLBlockInputStream( } for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) - algorithms.emplace_back(std::make_unique( + algorithms.emplace_back(std::make_unique( move_ttl, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) - algorithms.emplace_back(std::make_unique( + algorithms.emplace_back(std::make_unique( recompression_ttl, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } diff --git a/src/DataStreams/TTLCalcInputStream.cpp b/src/DataStreams/TTLCalcInputStream.cpp new file mode 100644 index 00000000000..ae21a592ca7 --- /dev/null +++ b/src/DataStreams/TTLCalcInputStream.cpp @@ -0,0 +1,94 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +TTLCalcInputStream::TTLCalcInputStream( + const BlockInputStreamPtr & input_, + const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const MergeTreeData::MutableDataPartPtr & data_part_, + time_t current_time_, + bool force_) + : data_part(data_part_) + , log(&Poco::Logger::get(storage_.getLogName() + " (TTLCalcInputStream)")) +{ + children.push_back(input_); + header = children.at(0)->getHeader(); + auto old_ttl_infos = data_part->ttl_infos; + + if (metadata_snapshot_->hasRowsTTL()) + { + const auto & rows_ttl = metadata_snapshot_->getRowsTTL(); + algorithms.emplace_back(std::make_unique( + rows_ttl, old_ttl_infos.table_ttl, current_time_, force_)); + } + + for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs()) + algorithms.emplace_back(std::make_unique( + where_ttl, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); + + for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) + algorithms.emplace_back(std::make_unique( + group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_)); + + if (metadata_snapshot_->hasAnyColumnTTL()) + { + for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs()) + { + algorithms.emplace_back(std::make_unique( + description, old_ttl_infos.columns_ttl[name], current_time_, force_)); + } + } + + for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) + algorithms.emplace_back(std::make_unique( + move_ttl, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); + + for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) + algorithms.emplace_back(std::make_unique( + recompression_ttl, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); +} + +Block reorderColumns(Block block, const Block & header) +{ + Block res; + for (const auto & col : header) + res.insert(block.getByName(col.name)); + + return res; +} + +Block TTLCalcInputStream::readImpl() +{ + auto block = children.at(0)->read(); + for (const auto & algorithm : algorithms) + algorithm->execute(block); + + if (!block) + return block; + + return reorderColumns(std::move(block), header); +} + +void TTLCalcInputStream::readSuffixImpl() +{ + data_part->ttl_infos = {}; + for (const auto & algorithm : algorithms) + algorithm->finalize(data_part); +} + +} diff --git a/src/DataStreams/TTLCalcInputStream.h b/src/DataStreams/TTLCalcInputStream.h new file mode 100644 index 00000000000..d1b629c2ad5 --- /dev/null +++ b/src/DataStreams/TTLCalcInputStream.h @@ -0,0 +1,44 @@ +#pragma once +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +class TTLCalcInputStream : public IBlockInputStream +{ +public: + TTLCalcInputStream( + const BlockInputStreamPtr & input_, + const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const MergeTreeData::MutableDataPartPtr & data_part_, + time_t current_time, + bool force_ + ); + + String getName() const override { return "TTL_CALC"; } + Block getHeader() const override { return header; } + +protected: + Block readImpl() override; + + /// Finalizes ttl infos and updates data part + void readSuffixImpl() override; + +private: + std::vector algorithms; + + /// ttl_infos and empty_columns are updating while reading + const MergeTreeData::MutableDataPartPtr & data_part; + Poco::Logger * log; + Block header; +}; + +} diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp index d5feb14658b..facf5a1f6d9 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp @@ -22,26 +22,31 @@ void TTLUpdateInfoAlgorithm::execute(Block & block) } } -TTLMoveAlgorithm::TTLMoveAlgorithm( - const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) - : TTLUpdateInfoAlgorithm(description_, old_ttl_info_, current_time_, force_) +void TTLUpdateInfoAlgorithm::finalize(const MutableDataPartPtr & data_part) const { -} + if (description.mode == TTLMode::RECOMPRESS) + { + data_part->ttl_infos.recompression_ttl[description.result_column] = new_ttl_info; + } + else if (description.mode == TTLMode::MOVE) + { + data_part->ttl_infos.moves_ttl[description.result_column] = new_ttl_info; + } + else if (description.mode == TTLMode::GROUP_BY) + { + data_part->ttl_infos.group_by_ttl[description.result_column] = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); + } + else if (description.mode == TTLMode::DELETE) + { + if (description.where_expression) + data_part->ttl_infos.rows_where_ttl[description.result_column] = new_ttl_info; + else + data_part->ttl_infos.table_ttl = new_ttl_info; -void TTLMoveAlgorithm::finalize(const MutableDataPartPtr & data_part) const -{ - data_part->ttl_infos.moves_ttl[description.result_column] = new_ttl_info; -} - -TTLRecompressionAlgorithm::TTLRecompressionAlgorithm( - const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) - : TTLUpdateInfoAlgorithm(description_, old_ttl_info_, current_time_, force_) -{ -} - -void TTLRecompressionAlgorithm::finalize(const MutableDataPartPtr & data_part) const -{ - data_part->ttl_infos.recompression_ttl[description.result_column] = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); + } + } } diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.h b/src/DataStreams/TTLUpdateInfoAlgorithm.h index c1ef0e1c90d..c0c4dcea755 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.h +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.h @@ -12,20 +12,6 @@ public: TTLUpdateInfoAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); void execute(Block & block) override; - void finalize(const MutableDataPartPtr & data_part) const override = 0; -}; - -class TTLMoveAlgorithm final : public TTLUpdateInfoAlgorithm -{ -public: - TTLMoveAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); - void finalize(const MutableDataPartPtr & data_part) const override; -}; - -class TTLRecompressionAlgorithm final : public TTLUpdateInfoAlgorithm -{ -public: - TTLRecompressionAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); void finalize(const MutableDataPartPtr & data_part) const override; }; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index fe0594bb58f..dcfe991b79a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -156,7 +156,7 @@ ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_ ColumnDependencies dependencies; while (!new_updated_columns.empty()) { - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true); new_updated_columns.clear(); for (const auto & dependency : new_dependencies) { @@ -424,14 +424,13 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) validateUpdateColumns(storage, metadata_snapshot, updated_columns, column_to_affected_materialized); } - dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); + dependencies = metadata_snapshot->getColumnDependencies(updated_columns, false); /// First, break a sequence of commands into stages. for (auto & command : commands) { if (command.type == MutationCommand::DELETE) { - mutation_kind.set(MutationKind::MUTATE_OTHER); if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); @@ -440,7 +439,6 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::UPDATE) { - mutation_kind.set(MutationKind::MUTATE_OTHER); if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); if (stages.size() == 1) /// First stage only supports filtering and can't update columns. @@ -531,7 +529,6 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::MATERIALIZE_INDEX) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); auto it = std::find_if( std::cbegin(indices_desc), std::end(indices_desc), [&](const IndexDescription & index) @@ -550,7 +547,6 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); for (const auto & column : projection.required_columns) dependencies.emplace(column, ColumnDependency::PROJECTION); @@ -558,60 +554,33 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::DROP_INDEX) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); materialized_indices.erase(command.index_name); } else if (command.type == MutationCommand::DROP_PROJECTION) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); materialized_projections.erase(command.projection_name); } else if (command.type == MutationCommand::MATERIALIZE_TTL) { - mutation_kind.set(MutationKind::MUTATE_OTHER); - if (metadata_snapshot->hasRowsTTL()) + // just recalculate ttl_infos without actual mutation + auto all_columns_vec = all_columns.getNames(); + auto new_dependencies = metadata_snapshot->getColumnDependencies(NameSet(all_columns_vec.begin(), all_columns_vec.end()), false); + + for (const auto & dependency : new_dependencies) { - for (const auto & column : all_columns) - dependencies.emplace(column.name, ColumnDependency::TTL_TARGET); + if (dependency.kind == ColumnDependency::TTL_EXPRESSION) + dependencies.insert(dependency); } - else + + if (dependencies.empty()) { - NameSet new_updated_columns; - auto column_ttls = metadata_snapshot->getColumns().getColumnTTLs(); - for (const auto & elem : column_ttls) - { - dependencies.emplace(elem.first, ColumnDependency::TTL_TARGET); - new_updated_columns.insert(elem.first); - } - - auto all_columns_vec = all_columns.getNames(); - auto all_dependencies = getAllColumnDependencies(metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end())); - - for (const auto & dependency : all_dependencies) - { - if (dependency.kind == ColumnDependency::TTL_EXPRESSION) - dependencies.insert(dependency); - } - - /// Recalc only skip indices and projections of columns which could be updated by TTL. - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns); - for (const auto & dependency : new_dependencies) - { - if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION) - dependencies.insert(dependency); - } - - if (dependencies.empty()) - { - /// Very rare case. It can happen if we have only one MOVE TTL with constant expression. - /// But we still have to read at least one column. - dependencies.emplace(all_columns.front().name, ColumnDependency::TTL_EXPRESSION); - } + /// Very rare case. It can happen if we have only one MOVE TTL with constant expression. + /// But we still have to read at least one column. + dependencies.emplace(all_columns.front().name, ColumnDependency::TTL_EXPRESSION); } } else if (command.type == MutationCommand::READ_COLUMN) { - mutation_kind.set(MutationKind::MUTATE_OTHER); if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); if (stages.size() == 1) /// First stage only supports filtering and can't update columns. @@ -638,6 +607,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) changed_columns.insert(dependency.column_name); } + // changed_columns is always empty because we don't delete or aggregate expired data here if (!changed_columns.empty()) { if (stages.empty() || !stages.back().column_to_updated.empty()) @@ -675,6 +645,11 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) plan.addStep(std::make_unique(Pipe(std::move(source)))); auto pipeline = addStreamsForLaterStages(stages_copy, plan); updated_header = std::make_unique(pipeline->getHeader()); + } + else + { + //no column updated in mutations. maybe just materialize(index\projection\ttl) + updated_header = std::make_unique(Block{}); } /// Special step to recalculate affected indices, projections and TTL expressions. @@ -981,10 +956,4 @@ bool MutationsInterpreter::isAffectingAllColumns() const return stages.back().isAffectingAllColumns(storage_columns); } -void MutationsInterpreter::MutationKind::set(const MutationKindEnum & kind) -{ - if (mutation_kind < kind) - mutation_kind = kind; -} - } diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index c9a589e6b6d..a2e11e527fc 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -64,20 +64,6 @@ public: NameSet grabMaterializedProjections() { return std::move(materialized_projections); } - struct MutationKind - { - enum MutationKindEnum - { - MUTATE_UNKNOWN, - MUTATE_INDEX_PROJECTION, - MUTATE_OTHER, - } mutation_kind = MUTATE_UNKNOWN; - - void set(const MutationKindEnum & kind); - }; - - MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } - private: ASTPtr prepare(bool dry_run); @@ -148,8 +134,6 @@ private: NameSet materialized_indices; NameSet materialized_projections; - MutationKind mutation_kind; /// Do we meet any index or projection mutation. - /// Columns, that we need to read for calculation of skip indices, projections or TTL expressions. ColumnDependencies dependencies; }; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a777c244426..44bdd3f9356 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -1249,8 +1250,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NamesAndTypesList storage_columns = metadata_snapshot->getColumns().getAllPhysical(); NameSet materialized_indices; NameSet materialized_projections; - MutationsInterpreter::MutationKind::MutationKindEnum mutation_kind - = MutationsInterpreter::MutationKind::MutationKindEnum::MUTATE_UNKNOWN; if (!for_interpreter.empty()) { @@ -1258,7 +1257,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true); materialized_indices = interpreter->grabMaterializedIndices(); materialized_projections = interpreter->grabMaterializedProjections(); - mutation_kind = interpreter->getMutationKind(); in = interpreter->execute(); updated_header = interpreter->getUpdatedHeader(); in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress)); @@ -1288,15 +1286,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); bool need_sync = needSyncPart(source_part->rows_count, source_part->getBytesOnDisk(), *data_settings); - bool need_remove_expired_values = false; + bool need_recalculate_ttl = false; if (in && shouldExecuteTTL(metadata_snapshot, interpreter->getColumnDependencies(), commands_for_part)) - need_remove_expired_values = true; + need_recalculate_ttl = true; /// All columns from part are changed and may be some more that were missing before in part /// TODO We can materialize compact part without copying data - if (!isWidePart(source_part) - || (mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && interpreter && interpreter->isAffectingAllColumns())) + if (!isWidePart(source_part) || (updated_header && interpreter && interpreter->isAffectingAllColumns())) { disk->createDirectories(new_part_tmp_path); @@ -1319,7 +1316,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values, + need_recalculate_ttl, need_sync, space_reservation, holder, @@ -1332,11 +1329,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { /// We will modify only some of the columns. Other columns and key values can be copied as-is. NameSet updated_columns; - if (mutation_kind != MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION) - { - for (const auto & name_type : updated_header.getNamesAndTypesList()) - updated_columns.emplace(name_type.name); - } + for (const auto & name_type : updated_header.getNamesAndTypesList()) + updated_columns.emplace(name_type.name); auto indices_to_recalc = getIndicesToRecalculate( in, updated_columns, metadata_snapshot, context, materialized_indices, source_part); @@ -1345,21 +1339,21 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameSet files_to_skip = collectFilesToSkip( source_part, - mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, + updated_header, indices_to_recalc, mrk_extension, projections_to_recalc); NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension); - if (indices_to_recalc.empty() && projections_to_recalc.empty() && mutation_kind != MutationsInterpreter::MutationKind::MUTATE_OTHER - && files_to_rename.empty()) + if (indices_to_recalc.empty() && projections_to_recalc.empty() && updated_columns.empty() + && files_to_rename.empty() && !need_recalculate_ttl) { LOG_TRACE( log, "Part {} doesn't change up to mutation version {} (optimized)", source_part->name, future_part.part_info.mutation); return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info, metadata_snapshot); } - if (need_remove_expired_values) + if (need_recalculate_ttl) files_to_skip.insert("ttl.txt"); disk->createDirectories(new_part_tmp_path); @@ -1413,14 +1407,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor metadata_snapshot, indices_to_recalc, projections_to_recalc, - // If it's an index/projection materialization, we don't write any data columns, thus empty header is used - mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, + updated_header, new_data_part, in, time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values, + need_recalculate_ttl, need_sync, space_reservation, holder, @@ -1441,7 +1434,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } } - finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values, compression_codec); + finalizeMutatedPart(source_part, new_data_part, need_recalculate_ttl, compression_codec); } return new_data_part; @@ -2167,7 +2160,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values, + bool need_recalculate_ttl, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -2180,8 +2173,8 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( mutating_stream = std::make_shared( std::make_shared(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression(metadata_snapshot))); - if (need_remove_expired_values) - mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); + if (need_recalculate_ttl) + mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); IMergeTreeDataPart::MinMaxIndex minmax_idx; @@ -2224,7 +2217,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values, + bool need_recalculate_ttl, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -2233,8 +2226,8 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); - if (need_remove_expired_values) - mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); + if (need_recalculate_ttl) + mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index dad83f64c70..f5faa5ea9d5 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -214,7 +214,7 @@ bool StorageInMemoryMetadata::hasAnyGroupByTTL() const return !table_ttl.group_by_ttl.empty(); } -ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet & updated_columns) const +ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet & updated_columns, bool include_ttl_target) const { if (updated_columns.empty()) return {}; @@ -250,26 +250,28 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet if (hasRowsTTL()) { auto rows_expression = getRowsTTL().expression; - if (add_dependent_columns(rows_expression, required_ttl_columns)) + if (add_dependent_columns(rows_expression, required_ttl_columns) && include_ttl_target) { /// Filter all columns, if rows TTL expression have to be recalculated. for (const auto & column : getColumns().getAllPhysical()) updated_ttl_columns.insert(column.name); } } - + for (const auto & entry : getRecompressionTTLs()) add_dependent_columns(entry.expression, required_ttl_columns); for (const auto & [name, entry] : getColumnTTLs()) { - if (add_dependent_columns(entry.expression, required_ttl_columns)) + if (add_dependent_columns(entry.expression, required_ttl_columns) && include_ttl_target) updated_ttl_columns.insert(name); } for (const auto & entry : getMoveTTLs()) add_dependent_columns(entry.expression, required_ttl_columns); + //TODO what about rows_where_ttl and group_by_ttl ?? + for (const auto & column : indices_columns) res.emplace(column, ColumnDependency::SKIP_INDEX); for (const auto & column : projections_columns) diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index d0d60f608d7..9accdb9b3b6 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -143,7 +143,7 @@ struct StorageInMemoryMetadata /// Returns columns, which will be needed to calculate dependencies (skip /// indices, TTL expressions) if we update @updated_columns set of columns. - ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; + ColumnDependencies getColumnDependencies(const NameSet & updated_columns, bool include_ttl_target) const; /// Block with ordinary + materialized columns. Block getSampleBlock() const; From c49c2e5e18b2561c6e993e7fd2af2fddf6012d66 Mon Sep 17 00:00:00 2001 From: terrylin Date: Fri, 30 Jul 2021 17:15:04 +0800 Subject: [PATCH 002/127] fix compile error --- src/DataStreams/TTLCalcInputStream.cpp | 18 +++++---------- src/Interpreters/MutationsInterpreter.cpp | 22 ------------------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- .../MergeTree/MergeTreeDataMergerMutator.h | 6 ++--- 4 files changed, 10 insertions(+), 40 deletions(-) diff --git a/src/DataStreams/TTLCalcInputStream.cpp b/src/DataStreams/TTLCalcInputStream.cpp index ae21a592ca7..6e94e89f7d7 100644 --- a/src/DataStreams/TTLCalcInputStream.cpp +++ b/src/DataStreams/TTLCalcInputStream.cpp @@ -1,16 +1,4 @@ #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include #include namespace DB @@ -81,7 +69,11 @@ Block TTLCalcInputStream::readImpl() if (!block) return block; - return reorderColumns(std::move(block), header); + Block res; + for (const auto & col : header) + res.insert(block.getByName(col.name)); + + return res; } void TTLCalcInputStream::readSuffixImpl() diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index dcfe991b79a..64796c4134e 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -150,28 +150,6 @@ ASTPtr prepareQueryAffectedAST(const std::vector & commands, co return select; } -ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns) -{ - NameSet new_updated_columns = updated_columns; - ColumnDependencies dependencies; - while (!new_updated_columns.empty()) - { - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true); - new_updated_columns.clear(); - for (const auto & dependency : new_dependencies) - { - if (!dependencies.count(dependency)) - { - dependencies.insert(dependency); - if (!dependency.isReadOnly()) - new_updated_columns.insert(dependency.column_name); - } - } - } - - return dependencies; -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 44bdd3f9356..284622a13da 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -2267,7 +2267,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( void MergeTreeDataMergerMutator::finalizeMutatedPart( const MergeTreeDataPartPtr & source_part, MergeTreeData::MutableDataPartPtr new_data_part, - bool need_remove_expired_values, + bool need_recalculate_ttl, const CompressionCodecPtr & codec) { auto disk = new_data_part->volume->getDisk(); @@ -2281,7 +2281,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( new_data_part->checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash(); } - if (need_remove_expired_values) + if (need_recalculate_ttl) { /// Write a file with ttl infos in json format. auto out_ttl = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "ttl.txt", 4096); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index ca7376d8f3e..e6ddc7447bc 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -242,7 +242,7 @@ private: time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values, + bool need_recalculate_ttl, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -260,7 +260,7 @@ private: time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values, + bool need_recalculate_ttl, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -271,7 +271,7 @@ private: static void finalizeMutatedPart( const MergeTreeDataPartPtr & source_part, MergeTreeData::MutableDataPartPtr new_data_part, - bool need_remove_expired_values, + bool need_recalculate_ttl, const CompressionCodecPtr & codec); public : From e740f5e6d56474ec8289f20ecc60b60b9bb9b973 Mon Sep 17 00:00:00 2001 From: terrylin Date: Fri, 30 Jul 2021 20:48:50 +0800 Subject: [PATCH 003/127] fix compile error --- src/DataStreams/TTLCalcInputStream.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/DataStreams/TTLCalcInputStream.cpp b/src/DataStreams/TTLCalcInputStream.cpp index 6e94e89f7d7..e34e85d4a67 100644 --- a/src/DataStreams/TTLCalcInputStream.cpp +++ b/src/DataStreams/TTLCalcInputStream.cpp @@ -51,15 +51,6 @@ TTLCalcInputStream::TTLCalcInputStream( recompression_ttl, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } -Block reorderColumns(Block block, const Block & header) -{ - Block res; - for (const auto & col : header) - res.insert(block.getByName(col.name)); - - return res; -} - Block TTLCalcInputStream::readImpl() { auto block = children.at(0)->read(); From b0df75626bb92b6f31a678f28af432f78879055b Mon Sep 17 00:00:00 2001 From: terrylin Date: Sat, 31 Jul 2021 19:37:13 +0800 Subject: [PATCH 004/127] modify test cases about ttl --- .../01070_materialize_ttl.reference | 27 ++++++++- .../0_stateless/01070_materialize_ttl.sql | 56 ++++++++++++++----- .../0_stateless/01070_modify_ttl.reference | 31 +++++++++- .../queries/0_stateless/01070_modify_ttl.sql | 55 +++++++++++++++--- ...1070_mutations_with_dependencies.reference | 13 ++++- .../01070_mutations_with_dependencies.sql | 32 +++++++++-- ..._alter_rename_with_ttl_zookeeper.reference | 1 + .../01378_alter_rename_with_ttl_zookeeper.sql | 7 +++ 8 files changed, 191 insertions(+), 31 deletions(-) diff --git a/tests/queries/0_stateless/01070_materialize_ttl.reference b/tests/queries/0_stateless/01070_materialize_ttl.reference index af1b3a4459b..1cbf104b969 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.reference +++ b/tests/queries/0_stateless/01070_materialize_ttl.reference @@ -2,15 +2,33 @@ 2000-10-10 2 2100-10-10 3 2100-10-10 4 +2000-10-10 1 +2000-10-10 2 +2100-10-10 3 +2100-10-10 4 2100-10-10 3 2100-10-10 4 1 a +2 b +3 c +4 d +1 a +3 c +1 a 3 c 1 a 2 b 3 c 4 d 1 a +2 b +3 c +4 d +1 a +2 +3 c +4 +1 a 2 3 c 4 @@ -20,7 +38,14 @@ 4 1 a 2 b +3 c +4 d +1 a +2 b +4 d +1 a +2 b 4 d 1 2 -4 d +4 d \ No newline at end of file diff --git a/tests/queries/0_stateless/01070_materialize_ttl.sql b/tests/queries/0_stateless/01070_materialize_ttl.sql index 2521ae35edf..1797d65d87f 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.sql +++ b/tests/queries/0_stateless/01070_materialize_ttl.sql @@ -1,62 +1,92 @@ drop table if exists ttl; +set mutations_sync = 2; +set materialize_ttl_after_modify = 0; + +create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; -create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); -set materialize_ttl_after_modify = 0; - alter table ttl materialize ttl; -- { serverError 80 } alter table ttl modify ttl d + interval 1 day; -- TTL should not be applied select * from ttl order by a; -alter table ttl materialize ttl settings mutations_sync=2; +alter table ttl materialize ttl; +select * from ttl order by a; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; select * from ttl order by a; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i; +create table ttl (i Int, s String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); -alter table ttl materialize ttl settings mutations_sync=2; +alter table ttl materialize ttl; select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; alter table ttl modify ttl toDate('2000-01-01'); -alter table ttl materialize ttl settings mutations_sync=2; +alter table ttl materialize ttl; +select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; select * from ttl order by i; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i; +create table ttl (i Int, s String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); -- TTL should not be applied select * from ttl order by i; -alter table ttl materialize ttl settings mutations_sync=2; +alter table ttl materialize ttl; select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; alter table ttl modify column s String ttl toDate('2000-01-01'); -alter table ttl materialize ttl settings mutations_sync=2; +alter table ttl materialize ttl; +select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; select * from ttl order by i; drop table if exists ttl; -create table ttl (d Date, i Int, s String) engine = MergeTree order by i; +create table ttl (d Date, i Int, s String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); -alter table ttl materialize ttl settings mutations_sync=2; +alter table ttl materialize ttl; select i, s from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select i, s from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; alter table ttl modify column s String ttl d + interval 1 month; -alter table ttl materialize ttl settings mutations_sync=2; +alter table ttl materialize ttl; +select i, s from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; select i, s from ttl order by i; drop table if exists ttl; diff --git a/tests/queries/0_stateless/01070_modify_ttl.reference b/tests/queries/0_stateless/01070_modify_ttl.reference index d64c1a4edc2..58682354f70 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.reference +++ b/tests/queries/0_stateless/01070_modify_ttl.reference @@ -1,15 +1,33 @@ +2000-10-10 1 +2000-10-10 2 +2100-10-10 3 +2100-10-10 4 2100-10-10 3 2100-10-10 4 ============= 1 a +2 b +3 c +4 d +1 a 3 c ============= +1 a +3 c ============= 1 a +2 b +3 c +4 d +1 a 2 3 c 4 ============= +1 a +2 +3 c +4 1 2 3 @@ -17,16 +35,27 @@ ============= 1 a 2 b +3 c +4 d +1 a +2 b 4 d ============= +1 a +2 b +4 d 1 2 4 d ============= +1 a aa +2 b bb +3 c cc +4 d dd 1 a 2 b bb 3 cc 4 d 1 ============= -0 +0 \ No newline at end of file diff --git a/tests/queries/0_stateless/01070_modify_ttl.sql b/tests/queries/0_stateless/01070_modify_ttl.sql index 4e842948afe..2196cdb7fb7 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.sql +++ b/tests/queries/0_stateless/01070_modify_ttl.sql @@ -1,64 +1,99 @@ drop table if exists ttl; +set mutations_sync = 2; + +create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; -create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); -set mutations_sync = 2; - alter table ttl modify ttl d + interval 1 day; select * from ttl order by a; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by a; select '============='; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i; +create table ttl (i Int, s String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; select '============='; alter table ttl modify ttl toDate('2000-01-01'); select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; select '============='; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i; +create table ttl (i Int, s String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; select '============='; alter table ttl modify column s String ttl toDate('2000-01-01'); select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; select '============='; drop table if exists ttl; -create table ttl (d Date, i Int, s String) engine = MergeTree order by i; +create table ttl (d Date, i Int, s String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); select i, s from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select i, s from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; select '============='; alter table ttl modify column s String ttl d + interval 1 month; select i, s from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select i, s from ttl order by i; select '============='; drop table if exists ttl; -create table ttl (i Int, s String, t String) engine = MergeTree order by i; +create table ttl (i Int, s String, t String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (1, 'a', 'aa') (2, 'b', 'bb') (3, 'c', 'cc') (4, 'd', 'dd'); alter table ttl modify column s String ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'), modify column t String ttl i % 3 = 1 ? today() - 10 : toDate('2100-01-01'); - +select i, s, t from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; select i, s, t from ttl order by i; -- MATERIALIZE TTL ran only once select count() from system.mutations where table = 'ttl' and is_done; @@ -67,7 +102,9 @@ select '============='; drop table if exists ttl; -- Nothing changed, don't run mutation -create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i; +create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + alter table ttl modify column s String ttl toDate('2000-01-02'); select count() from system.mutations where table = 'ttl' and is_done; diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.reference b/tests/queries/0_stateless/01070_mutations_with_dependencies.reference index eeb32eab7a5..94464431944 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.reference +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.reference @@ -3,9 +3,20 @@ 3 1 c 4 1 d 1 1 a +2 1 b +3 1 c +4 1 d +1 1 a +2 0 b +3 1 c +4 0 d +1 1 a 3 1 c =================== 4 +4 +0 2 2 -1 2 42 2000-01-01 +1 2 3 2000-01-01 +1 2 42 2000-01-01 \ No newline at end of file diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql index 10077a94c9d..f05c56c7daa 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql @@ -2,15 +2,24 @@ drop table if exists ttl; set mutations_sync = 2; -- check that ttl info was updated after mutation. -create table ttl (i Int, a Int, s String) engine = MergeTree order by i; +create table ttl (i Int, a Int, s String) engine = MergeTree order by i +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (1, 1, 'a') (2, 1, 'b') (3, 1, 'c') (4, 1, 'd'); alter table ttl modify ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; - select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; + alter table ttl update a = 0 where i % 2 = 0; select * from ttl order by i; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl order by i; drop table ttl; @@ -18,27 +27,38 @@ select '==================='; -- check that skip index is updated after column was modified by ttl. create table ttl (i Int, a Int, s String default 'b' ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'), - index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i; + index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i + SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (1, 1, 'a') (2, 1, 'a') (3, 1, 'a') (4, 1, 'a'); select count() from ttl where s = 'a'; alter table ttl update a = 0 where i % 2 = 0; - +select count() from ttl where s = 'a'; +select count() from ttl where s = 'b'; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; drop table ttl; -- check only that it doesn't throw exceptions. -create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default'; +create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default' +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + alter table ttl materialize ttl; drop table ttl; create table ttl (a Int, b Int, c Int default 42 ttl d, d Date, index ind (b * c) type minmax granularity 1) -engine = MergeTree order by a; +engine = MergeTree order by a SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + insert into ttl values (1, 2, 3, '2100-01-01'); alter table ttl update d = '2000-01-01' where 1; alter table ttl materialize ttl; select * from ttl; +alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table ttl; +select * from ttl; drop table ttl; diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference index bf8f7658af4..204c1900fd5 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference @@ -1,3 +1,4 @@ 9 9 +9 0 diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 0cd6feb9da1..37ba050373d 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -6,6 +6,8 @@ CREATE TABLE table_rename_with_ttl value1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') +SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + ORDER BY tuple(); INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); @@ -23,4 +25,9 @@ ALTER TABLE table_rename_with_ttl materialize TTL settings mutations_sync=2; SELECT count() FROM table_rename_with_ttl; +ALTER TABLE table_rename_with_ttl MODIFY setting max_number_of_merges_with_ttl_in_pool = 1; +optimize table table_rename_with_ttl; + +SELECT count() FROM table_rename_with_ttl; + DROP TABLE IF EXISTS table_rename_with_ttl; From 09a4f82d0f88b386c82b039cce52423d40d4194d Mon Sep 17 00:00:00 2001 From: terrylin Date: Sat, 31 Jul 2021 20:36:40 +0800 Subject: [PATCH 005/127] code style check --- src/DataStreams/TTLUpdateInfoAlgorithm.cpp | 4 ++-- src/Interpreters/MutationsInterpreter.cpp | 4 ++-- src/Storages/StorageInMemoryMetadata.cpp | 2 +- tests/queries/0_stateless/01070_materialize_ttl.reference | 2 +- tests/queries/0_stateless/01070_modify_ttl.reference | 2 +- .../0_stateless/01070_mutations_with_dependencies.reference | 2 +- .../0_stateless/01378_alter_rename_with_ttl_zookeeper.sql | 1 + 7 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp index facf5a1f6d9..49006be7c59 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp @@ -24,7 +24,7 @@ void TTLUpdateInfoAlgorithm::execute(Block & block) void TTLUpdateInfoAlgorithm::finalize(const MutableDataPartPtr & data_part) const { - if (description.mode == TTLMode::RECOMPRESS) + if (description.mode == TTLMode::RECOMPRESS) { data_part->ttl_infos.recompression_ttl[description.result_column] = new_ttl_info; } @@ -46,7 +46,7 @@ void TTLUpdateInfoAlgorithm::finalize(const MutableDataPartPtr & data_part) cons data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); } - + } } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 64796c4134e..6347d32a241 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -623,8 +623,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) plan.addStep(std::make_unique(Pipe(std::move(source)))); auto pipeline = addStreamsForLaterStages(stages_copy, plan); updated_header = std::make_unique(pipeline->getHeader()); - } - else + } + else { //no column updated in mutations. maybe just materialize(index\projection\ttl) updated_header = std::make_unique(Block{}); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index f5faa5ea9d5..73a4f6960b6 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -257,7 +257,7 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet updated_ttl_columns.insert(column.name); } } - + for (const auto & entry : getRecompressionTTLs()) add_dependent_columns(entry.expression, required_ttl_columns); diff --git a/tests/queries/0_stateless/01070_materialize_ttl.reference b/tests/queries/0_stateless/01070_materialize_ttl.reference index 1cbf104b969..12906a246e3 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.reference +++ b/tests/queries/0_stateless/01070_materialize_ttl.reference @@ -48,4 +48,4 @@ 4 d 1 2 -4 d \ No newline at end of file +4 d diff --git a/tests/queries/0_stateless/01070_modify_ttl.reference b/tests/queries/0_stateless/01070_modify_ttl.reference index 58682354f70..8bfefa38ae1 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.reference +++ b/tests/queries/0_stateless/01070_modify_ttl.reference @@ -58,4 +58,4 @@ 4 d 1 ============= -0 \ No newline at end of file +0 diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.reference b/tests/queries/0_stateless/01070_mutations_with_dependencies.reference index 94464431944..dc411d6b68e 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.reference +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.reference @@ -19,4 +19,4 @@ 2 2 1 2 3 2000-01-01 -1 2 42 2000-01-01 \ No newline at end of file +1 2 42 2000-01-01 diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 37ba050373d..0dbb97352a2 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -6,6 +6,7 @@ CREATE TABLE table_rename_with_ttl value1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') +order by value1 SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; ORDER BY tuple(); From 79c55866b52d12ff24a4c7cdc5ef4bbe75d95dca Mon Sep 17 00:00:00 2001 From: terrylin Date: Sat, 31 Jul 2021 21:10:06 +0800 Subject: [PATCH 006/127] modify test cases --- .../0_stateless/01378_alter_rename_with_ttl_zookeeper.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 0dbb97352a2..1bc788b77ac 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -6,11 +6,9 @@ CREATE TABLE table_rename_with_ttl value1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') -order by value1 +ORDER BY tuple() SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; -ORDER BY tuple(); - INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); SELECT count() FROM table_rename_with_ttl; From d0b44a3d5da802cd61ba42c24d739832c516ab50 Mon Sep 17 00:00:00 2001 From: terrylin Date: Sat, 31 Jul 2021 22:34:51 +0800 Subject: [PATCH 007/127] modify test cases --- .../0_stateless/01070_materialize_ttl.sql | 47 ++++++++++------ .../queries/0_stateless/01070_modify_ttl.sql | 54 +++++++++++++------ .../01070_mutations_with_dependencies.sql | 28 ++++++---- .../01378_alter_rename_with_ttl_zookeeper.sql | 6 ++- 4 files changed, 92 insertions(+), 43 deletions(-) diff --git a/tests/queries/0_stateless/01070_materialize_ttl.sql b/tests/queries/0_stateless/01070_materialize_ttl.sql index 1797d65d87f..3c8bbaf1af8 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.sql +++ b/tests/queries/0_stateless/01070_materialize_ttl.sql @@ -1,9 +1,11 @@ -drop table if exists ttl; set mutations_sync = 2; set materialize_ttl_after_modify = 0; +drop table if exists ttl; + create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); @@ -18,35 +20,43 @@ select * from ttl order by a; alter table ttl materialize ttl; select * from ttl order by a; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; select * from ttl order by a; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; + insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; alter table ttl modify ttl toDate('2000-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; + insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); @@ -55,38 +65,45 @@ select * from ttl order by i; alter table ttl materialize ttl; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; alter table ttl modify column s String ttl toDate('2000-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; drop table if exists ttl; create table ttl (d Date, i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select i, s from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select i, s from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; alter table ttl modify column s String ttl d + interval 1 month; alter table ttl materialize ttl; select i, s from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select i, s from ttl order by i; drop table if exists ttl; diff --git a/tests/queries/0_stateless/01070_modify_ttl.sql b/tests/queries/0_stateless/01070_modify_ttl.sql index 2196cdb7fb7..694955414be 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.sql +++ b/tests/queries/0_stateless/01070_modify_ttl.sql @@ -2,7 +2,8 @@ drop table if exists ttl; set mutations_sync = 2; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); @@ -11,90 +12,108 @@ insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); alter table ttl modify ttl d + interval 1 day; select * from ttl order by a; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by a; select '============='; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; select '============='; alter table ttl modify ttl toDate('2000-01-01'); select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; select '============='; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; select '============='; alter table ttl modify column s String ttl toDate('2000-01-01'); select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; select '============='; drop table if exists ttl; create table ttl (d Date, i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); select i, s from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select i, s from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; select '============='; alter table ttl modify column s String ttl d + interval 1 month; select i, s from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select i, s from ttl order by i; select '============='; drop table if exists ttl; create table ttl (i Int, s String, t String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (1, 'a', 'aa') (2, 'b', 'bb') (3, 'c', 'cc') (4, 'd', 'dd'); alter table ttl modify column s String ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'), modify column t String ttl i % 3 = 1 ? today() - 10 : toDate('2100-01-01'); select i, s, t from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select i, s, t from ttl order by i; + -- MATERIALIZE TTL ran only once select count() from system.mutations where table = 'ttl' and is_done; select '============='; @@ -103,7 +122,8 @@ drop table if exists ttl; -- Nothing changed, don't run mutation create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; alter table ttl modify column s String ttl toDate('2000-01-02'); select count() from system.mutations where table = 'ttl' and is_done; diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql index f05c56c7daa..fa142f280b4 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql @@ -3,22 +3,26 @@ set mutations_sync = 2; -- check that ttl info was updated after mutation. create table ttl (i Int, a Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (1, 1, 'a') (2, 1, 'b') (3, 1, 'c') (4, 1, 'd'); alter table ttl modify ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 0; alter table ttl update a = 0 where i % 2 = 0; select * from ttl order by i; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select * from ttl order by i; drop table ttl; @@ -28,7 +32,8 @@ select '==================='; -- check that skip index is updated after column was modified by ttl. create table ttl (i Int, a Int, s String default 'b' ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'), index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i - SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; + SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (1, 1, 'a') (2, 1, 'a') (3, 1, 'a') (4, 1, 'a'); @@ -37,8 +42,10 @@ select count() from ttl where s = 'a'; alter table ttl update a = 0 where i % 2 = 0; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; +SYSTEM STOP TTL MERGES; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; @@ -46,19 +53,22 @@ drop table ttl; -- check only that it doesn't throw exceptions. create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default' -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; alter table ttl materialize ttl; drop table ttl; create table ttl (a Int, b Int, c Int default 42 ttl d, d Date, index ind (b * c) type minmax granularity 1) -engine = MergeTree order by a SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +engine = MergeTree order by a SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES; insert into ttl values (1, 2, 3, '2100-01-01'); alter table ttl update d = '2000-01-01' where 1; alter table ttl materialize ttl; select * from ttl; -alter table ttl modify setting max_number_of_merges_with_ttl_in_pool = 1; + +SYSTEM START TTL MERGES; optimize table ttl; select * from ttl; drop table ttl; diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 1bc788b77ac..3c1a1c6fe44 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -1,4 +1,7 @@ DROP TABLE IF EXISTS table_rename_with_ttl; +SET replication_alter_partitions_sync = 2; +SET mutations_sync = 2; +SET materialize_ttl_after_modify = 0; CREATE TABLE table_rename_with_ttl ( @@ -13,14 +16,13 @@ INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toSt SELECT count() FROM table_rename_with_ttl; -SET materialize_ttl_after_modify = 0; ALTER TABLE table_rename_with_ttl MODIFY TTL date1 + INTERVAL 1 MONTH; SELECT count() FROM table_rename_with_ttl; ALTER TABLE table_rename_with_ttl RENAME COLUMN date1 TO renamed_date1; -ALTER TABLE table_rename_with_ttl materialize TTL settings mutations_sync=2; +ALTER TABLE table_rename_with_ttl materialize TTL; SELECT count() FROM table_rename_with_ttl; From c9166f1a3fdec02f57fd9694369ba52741e754c0 Mon Sep 17 00:00:00 2001 From: terrylin Date: Sat, 31 Jul 2021 23:24:30 +0800 Subject: [PATCH 008/127] modify test cases --- .../0_stateless/01070_materialize_ttl.sql | 34 +++++++------- .../queries/0_stateless/01070_modify_ttl.sql | 44 +++++++++---------- .../01070_mutations_with_dependencies.sql | 22 +++++----- 3 files changed, 50 insertions(+), 50 deletions(-) diff --git a/tests/queries/0_stateless/01070_materialize_ttl.sql b/tests/queries/0_stateless/01070_materialize_ttl.sql index 3c8bbaf1af8..49c6324e6ce 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.sql +++ b/tests/queries/0_stateless/01070_materialize_ttl.sql @@ -5,7 +5,7 @@ drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); @@ -21,7 +21,7 @@ select * from ttl order by a; alter table ttl materialize ttl; select * from ttl order by a; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; select * from ttl order by a; @@ -29,7 +29,7 @@ drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); @@ -37,25 +37,25 @@ alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; alter table ttl modify ttl toDate('2000-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); @@ -66,25 +66,25 @@ select * from ttl order by i; alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; alter table ttl modify column s String ttl toDate('2000-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; drop table if exists ttl; create table ttl (d Date, i Int, s String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); @@ -92,18 +92,18 @@ alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select i, s from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select i, s from ttl order by i; alter table ttl modify column s String ttl d + interval 1 month; alter table ttl materialize ttl; select i, s from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select i, s from ttl order by i; drop table if exists ttl; diff --git a/tests/queries/0_stateless/01070_modify_ttl.sql b/tests/queries/0_stateless/01070_modify_ttl.sql index 694955414be..2e904f64274 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.sql +++ b/tests/queries/0_stateless/01070_modify_ttl.sql @@ -3,7 +3,7 @@ set mutations_sync = 2; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); @@ -13,9 +13,9 @@ insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); alter table ttl modify ttl d + interval 1 day; select * from ttl order by a; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by a; select '============='; @@ -23,25 +23,25 @@ drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; select '============='; alter table ttl modify ttl toDate('2000-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; select '============='; @@ -49,25 +49,25 @@ drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; select '============='; alter table ttl modify column s String ttl toDate('2000-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; select '============='; @@ -75,25 +75,25 @@ drop table if exists ttl; create table ttl (d Date, i Int, s String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); select i, s from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select i, s from ttl order by i; select '============='; alter table ttl modify column s String ttl d + interval 1 month; select i, s from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select i, s from ttl order by i; select '============='; @@ -101,7 +101,7 @@ drop table if exists ttl; create table ttl (i Int, s String, t String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 'a', 'aa') (2, 'b', 'bb') (3, 'c', 'cc') (4, 'd', 'dd'); @@ -109,9 +109,9 @@ alter table ttl modify column s String ttl i % 3 = 0 ? today() - 10 : toDate('21 modify column t String ttl i % 3 = 1 ? today() - 10 : toDate('2100-01-01'); select i, s, t from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select i, s, t from ttl order by i; -- MATERIALIZE TTL ran only once @@ -123,7 +123,7 @@ drop table if exists ttl; -- Nothing changed, don't run mutation create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; alter table ttl modify column s String ttl toDate('2000-01-02'); select count() from system.mutations where table = 'ttl' and is_done; diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql index fa142f280b4..bbb79f4c5d6 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql @@ -4,7 +4,7 @@ set mutations_sync = 2; -- check that ttl info was updated after mutation. create table ttl (i Int, a Int, s String) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 1, 'a') (2, 1, 'b') (3, 1, 'c') (4, 1, 'd'); @@ -12,17 +12,17 @@ alter table ttl modify ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; alter table ttl update a = 0 where i % 2 = 0; select * from ttl order by i; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select * from ttl order by i; drop table ttl; @@ -33,7 +33,7 @@ select '==================='; create table ttl (i Int, a Int, s String default 'b' ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'), index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 1, 'a') (2, 1, 'a') (3, 1, 'a') (4, 1, 'a'); @@ -43,9 +43,9 @@ alter table ttl update a = 0 where i % 2 = 0; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; @@ -54,21 +54,21 @@ drop table ttl; -- check only that it doesn't throw exceptions. create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default' SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; alter table ttl materialize ttl; drop table ttl; create table ttl (a Int, b Int, c Int default 42 ttl d, d Date, index ind (b * c) type minmax granularity 1) engine = MergeTree order by a SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES; +SYSTEM STOP TTL MERGES ttl; insert into ttl values (1, 2, 3, '2100-01-01'); alter table ttl update d = '2000-01-01' where 1; alter table ttl materialize ttl; select * from ttl; -SYSTEM START TTL MERGES; +SYSTEM START TTL MERGES ttl; optimize table ttl; select * from ttl; drop table ttl; From fa1f069793adc0d85b0630c1622476275f165f4b Mon Sep 17 00:00:00 2001 From: terrylin Date: Sun, 1 Aug 2021 00:16:00 +0800 Subject: [PATCH 009/127] modify test cases --- .../0_stateless/01378_alter_rename_with_ttl_zookeeper.sql | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 3c1a1c6fe44..686c0181175 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -10,7 +10,8 @@ CREATE TABLE table_rename_with_ttl ) ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') ORDER BY tuple() -SETTINGS merge_with_ttl_timeout=0,max_number_of_merges_with_ttl_in_pool=0; +SETTINGS merge_with_ttl_timeout=0; +SYSTEM STOP TTL MERGES table_rename_with_ttl; INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); @@ -26,7 +27,7 @@ ALTER TABLE table_rename_with_ttl materialize TTL; SELECT count() FROM table_rename_with_ttl; -ALTER TABLE table_rename_with_ttl MODIFY setting max_number_of_merges_with_ttl_in_pool = 1; +SYSTEM START TTL MERGES table_rename_with_ttl; optimize table table_rename_with_ttl; SELECT count() FROM table_rename_with_ttl; From eb555fc719815f1535d064a5ba54a93ff781e3b7 Mon Sep 17 00:00:00 2001 From: terrylin Date: Sun, 1 Aug 2021 21:34:30 +0800 Subject: [PATCH 010/127] modify test cases --- .../0_stateless/01070_materialize_ttl.sql | 39 ++++----------- .../queries/0_stateless/01070_modify_ttl.sql | 50 ++++++------------- .../01070_mutations_with_dependencies.sql | 27 +++------- .../01378_alter_rename_with_ttl_zookeeper.sql | 2 +- 4 files changed, 34 insertions(+), 84 deletions(-) diff --git a/tests/queries/0_stateless/01070_materialize_ttl.sql b/tests/queries/0_stateless/01070_materialize_ttl.sql index 49c6324e6ce..19684f028a7 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.sql +++ b/tests/queries/0_stateless/01070_materialize_ttl.sql @@ -4,8 +4,7 @@ set materialize_ttl_after_modify = 0; drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); @@ -21,15 +20,13 @@ select * from ttl order by a; alter table ttl materialize ttl; select * from ttl order by a; -SYSTEM START TTL MERGES ttl; -optimize table ttl; +optimize table ttl final; select * from ttl order by a; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); @@ -37,25 +34,20 @@ alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; alter table ttl modify ttl toDate('2000-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); @@ -66,25 +58,20 @@ select * from ttl order by i; alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; alter table ttl modify column s String ttl toDate('2000-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; drop table if exists ttl; create table ttl (d Date, i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); @@ -92,18 +79,14 @@ alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select i, s from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select i, s from ttl order by i; alter table ttl modify column s String ttl d + interval 1 month; alter table ttl materialize ttl; select i, s from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select i, s from ttl order by i; drop table if exists ttl; diff --git a/tests/queries/0_stateless/01070_modify_ttl.sql b/tests/queries/0_stateless/01070_modify_ttl.sql index 2e904f64274..f4228b886ba 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.sql +++ b/tests/queries/0_stateless/01070_modify_ttl.sql @@ -2,8 +2,7 @@ drop table if exists ttl; set mutations_sync = 2; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); @@ -13,95 +12,77 @@ insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); alter table ttl modify ttl d + interval 1 day; select * from ttl order by a; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by a; select '============='; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; select '============='; alter table ttl modify ttl toDate('2000-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; select '============='; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; select '============='; alter table ttl modify column s String ttl toDate('2000-01-01'); select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; select '============='; drop table if exists ttl; create table ttl (d Date, i Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); select i, s from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select i, s from ttl order by i; select '============='; alter table ttl modify column s String ttl d + interval 1 month; select i, s from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select i, s from ttl order by i; select '============='; drop table if exists ttl; create table ttl (i Int, s String, t String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 'a', 'aa') (2, 'b', 'bb') (3, 'c', 'cc') (4, 'd', 'dd'); @@ -109,9 +90,7 @@ alter table ttl modify column s String ttl i % 3 = 0 ? today() - 10 : toDate('21 modify column t String ttl i % 3 = 1 ? today() - 10 : toDate('2100-01-01'); select i, s, t from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select i, s, t from ttl order by i; -- MATERIALIZE TTL ran only once @@ -122,8 +101,7 @@ drop table if exists ttl; -- Nothing changed, don't run mutation create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; alter table ttl modify column s String ttl toDate('2000-01-02'); select count() from system.mutations where table = 'ttl' and is_done; diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql index bbb79f4c5d6..f56f25b5e77 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql @@ -3,8 +3,7 @@ set mutations_sync = 2; -- check that ttl info was updated after mutation. create table ttl (i Int, a Int, s String) engine = MergeTree order by i -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 1, 'a') (2, 1, 'b') (3, 1, 'c') (4, 1, 'd'); @@ -12,17 +11,13 @@ alter table ttl modify ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; alter table ttl update a = 0 where i % 2 = 0; select * from ttl order by i; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select * from ttl order by i; drop table ttl; @@ -32,8 +27,7 @@ select '==================='; -- check that skip index is updated after column was modified by ttl. create table ttl (i Int, a Int, s String default 'b' ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'), index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i - SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; + SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 1, 'a') (2, 1, 'a') (3, 1, 'a') (4, 1, 'a'); @@ -43,9 +37,7 @@ alter table ttl update a = 0 where i % 2 = 0; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; -SYSTEM START TTL MERGES ttl; -optimize table ttl; -SYSTEM STOP TTL MERGES ttl; +optimize table ttl final; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; @@ -53,22 +45,19 @@ drop table ttl; -- check only that it doesn't throw exceptions. create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default' -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; alter table ttl materialize ttl; drop table ttl; create table ttl (a Int, b Int, c Int default 42 ttl d, d Date, index ind (b * c) type minmax granularity 1) -engine = MergeTree order by a SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES ttl; +engine = MergeTree order by a SETTINGS max_number_of_merges_with_ttl_in_pool=0; insert into ttl values (1, 2, 3, '2100-01-01'); alter table ttl update d = '2000-01-01' where 1; alter table ttl materialize ttl; select * from ttl; -SYSTEM START TTL MERGES ttl; -optimize table ttl; +optimize table ttl final; select * from ttl; drop table ttl; diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 686c0181175..02ab946c228 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -28,7 +28,7 @@ ALTER TABLE table_rename_with_ttl materialize TTL; SELECT count() FROM table_rename_with_ttl; SYSTEM START TTL MERGES table_rename_with_ttl; -optimize table table_rename_with_ttl; +optimize table table_rename_with_ttl FINAL; SELECT count() FROM table_rename_with_ttl; From d468bd7af3f4356be0cc6221a8d80f61a7ad4966 Mon Sep 17 00:00:00 2001 From: terrylin Date: Mon, 2 Aug 2021 23:18:42 +0800 Subject: [PATCH 011/127] modify test cases --- tests/integration/test_ttl_replicated/test.py | 19 +++++++++++++++---- .../01378_alter_rename_with_ttl_zookeeper.sql | 5 ++--- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index f37c28b2a80..a5a7355f912 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -161,7 +161,7 @@ def test_modify_ttl(started_cluster): ''' CREATE TABLE test_ttl(d DateTime, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_modify', '{replica}') - ORDER BY id + ORDER BY id SETTINGS merge_with_ttl_timeout=0; '''.format(replica=node.name)) node1.query( @@ -169,12 +169,15 @@ def test_modify_ttl(started_cluster): node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) node1.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 4 HOUR SETTINGS mutations_sync = 2") + time.sleep(5) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "2\n3\n" node2.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 2 HOUR SETTINGS mutations_sync = 2") + time.sleep(5) # TTL merges shall happen. assert node1.query("SELECT id FROM test_ttl") == "3\n" node1.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 30 MINUTE SETTINGS mutations_sync = 2") + time.sleep(5) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "" @@ -185,7 +188,7 @@ def test_modify_column_ttl(started_cluster): ''' CREATE TABLE test_ttl(d DateTime, id UInt32 DEFAULT 42) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_column', '{replica}') - ORDER BY d + ORDER BY d SETTINGS merge_with_ttl_timeout=0; '''.format(replica=node.name)) node1.query( @@ -193,12 +196,15 @@ def test_modify_column_ttl(started_cluster): node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 4 HOUR SETTINGS mutations_sync = 2") + time.sleep(5) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "42\n2\n3\n" node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 2 HOUR SETTINGS mutations_sync = 2") + time.sleep(5) # TTL merges shall happen. assert node1.query("SELECT id FROM test_ttl") == "42\n42\n3\n" node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 30 MINUTE SETTINGS mutations_sync = 2") + time.sleep(5) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "42\n42\n42\n" @@ -298,7 +304,9 @@ def test_ttl_empty_parts(started_cluster): SETTINGS max_bytes_to_merge_at_min_space_in_pool = 1, max_bytes_to_merge_at_max_space_in_pool = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 '''.format(replica=node.name)) + node.query("SYSTEM STOP TTL MERGES test_ttl_empty_parts") + for i in range (1, 7): node1.query("INSERT INTO test_ttl_empty_parts SELECT '2{}00-01-0{}', number FROM numbers(1000)".format(i % 2, i)) @@ -308,19 +316,22 @@ def test_ttl_empty_parts(started_cluster): node1.query("ALTER TABLE test_ttl_empty_parts MODIFY TTL date") - assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "3000\n" + assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "6000\n" time.sleep(3) # Wait for cleanup thread assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == \ - "all_0_0_0_6\nall_2_2_0_6\nall_4_4_0_6\n" + "all_0_0_0_6\nall_1_1_0_6\nall_2_2_0_6\nall_3_3_0_6\nall_4_4_0_6\nall_5_5_0_6\n" for node in [node1, node2]: node.query("ALTER TABLE test_ttl_empty_parts MODIFY SETTING max_bytes_to_merge_at_min_space_in_pool = 1000000000") node.query("ALTER TABLE test_ttl_empty_parts MODIFY SETTING max_bytes_to_merge_at_max_space_in_pool = 1000000000") + node.query("SYSTEM START TTL MERGES test_ttl_empty_parts") optimize_with_retry(node1, 'test_ttl_empty_parts') assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_4_1_6\n" + assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "3000\n" + # Check that after removing empty parts mutations and merges works node1.query("INSERT INTO test_ttl_empty_parts SELECT '2100-01-20', number FROM numbers(1000)") node1.query("ALTER TABLE test_ttl_empty_parts DELETE WHERE id % 2 = 0 SETTINGS mutations_sync = 2") diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 02ab946c228..7b7a3767b1f 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -10,8 +10,7 @@ CREATE TABLE table_rename_with_ttl ) ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') ORDER BY tuple() -SETTINGS merge_with_ttl_timeout=0; -SYSTEM STOP TTL MERGES table_rename_with_ttl; +SETTINGS max_number_of_merges_with_ttl_in_pool=0; INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); @@ -27,7 +26,7 @@ ALTER TABLE table_rename_with_ttl materialize TTL; SELECT count() FROM table_rename_with_ttl; -SYSTEM START TTL MERGES table_rename_with_ttl; +ALTER TABLE table_rename_with_ttl modify setting max_number_of_merges_with_ttl_in_pool=2; optimize table table_rename_with_ttl FINAL; SELECT count() FROM table_rename_with_ttl; From ffe9eab1d0da8f2f718fcd553e5baec59f9438be Mon Sep 17 00:00:00 2001 From: terrylin Date: Tue, 3 Aug 2021 15:37:25 +0800 Subject: [PATCH 012/127] update test cases and fix bug --- src/DataStreams/TTLBlockInputStream.cpp | 4 +- src/DataStreams/TTLCalcInputStream.cpp | 12 +++--- src/DataStreams/TTLUpdateInfoAlgorithm.cpp | 39 ++++++++++++------- src/DataStreams/TTLUpdateInfoAlgorithm.h | 23 ++++++++++- tests/integration/test_ttl_replicated/test.py | 14 +++---- .../01560_ttl_remove_empty_parts.sql | 3 ++ 6 files changed, 66 insertions(+), 29 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 2cf7c121868..fc557bccad1 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -82,11 +82,11 @@ TTLBlockInputStream::TTLBlockInputStream( for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) algorithms.emplace_back(std::make_unique( - move_ttl, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); + move_ttl, TTLUpdateType::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) algorithms.emplace_back(std::make_unique( - recompression_ttl, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); + recompression_ttl, TTLUpdateType::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } Block reorderColumns(Block block, const Block & header) diff --git a/src/DataStreams/TTLCalcInputStream.cpp b/src/DataStreams/TTLCalcInputStream.cpp index e34e85d4a67..82b17433b77 100644 --- a/src/DataStreams/TTLCalcInputStream.cpp +++ b/src/DataStreams/TTLCalcInputStream.cpp @@ -22,33 +22,33 @@ TTLCalcInputStream::TTLCalcInputStream( { const auto & rows_ttl = metadata_snapshot_->getRowsTTL(); algorithms.emplace_back(std::make_unique( - rows_ttl, old_ttl_infos.table_ttl, current_time_, force_)); + rows_ttl, TTLUpdateType::TABLE_TTL, rows_ttl.result_column, old_ttl_infos.table_ttl, current_time_, force_)); } for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs()) algorithms.emplace_back(std::make_unique( - where_ttl, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); + where_ttl, TTLUpdateType::ROWS_WHERE_TTL, where_ttl.result_column, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) algorithms.emplace_back(std::make_unique( - group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_)); + group_by_ttl, TTLUpdateType::GROUP_BY_TTL, group_by_ttl.result_column, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_)); if (metadata_snapshot_->hasAnyColumnTTL()) { for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs()) { algorithms.emplace_back(std::make_unique( - description, old_ttl_infos.columns_ttl[name], current_time_, force_)); + description, TTLUpdateType::COLUMNS_TTL, name, old_ttl_infos.columns_ttl[name], current_time_, force_)); } } for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) algorithms.emplace_back(std::make_unique( - move_ttl, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); + move_ttl, TTLUpdateType::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) algorithms.emplace_back(std::make_unique( - recompression_ttl, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); + recompression_ttl, TTLUpdateType::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } Block TTLCalcInputStream::readImpl() diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp index 49006be7c59..21e36f1361c 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp @@ -4,8 +4,15 @@ namespace DB { TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm( - const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + const TTLDescription & description_, + const TTLUpdateType ttl_update_type_, + const String ttl_update_key_, + const TTLInfo & old_ttl_info_, + time_t current_time_, + bool force_) : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) + , ttl_update_type(ttl_update_type_) + , ttl_update_key(ttl_update_key_) { } @@ -24,26 +31,32 @@ void TTLUpdateInfoAlgorithm::execute(Block & block) void TTLUpdateInfoAlgorithm::finalize(const MutableDataPartPtr & data_part) const { - if (description.mode == TTLMode::RECOMPRESS) + if (ttl_update_type == TTLUpdateType::RECOMPRESSION_TTL) { - data_part->ttl_infos.recompression_ttl[description.result_column] = new_ttl_info; + data_part->ttl_infos.recompression_ttl[ttl_update_key] = new_ttl_info; } - else if (description.mode == TTLMode::MOVE) + else if (ttl_update_type == TTLUpdateType::MOVES_TTL) { - data_part->ttl_infos.moves_ttl[description.result_column] = new_ttl_info; + data_part->ttl_infos.moves_ttl[ttl_update_key] = new_ttl_info; } - else if (description.mode == TTLMode::GROUP_BY) + else if (ttl_update_type == TTLUpdateType::GROUP_BY_TTL) { - data_part->ttl_infos.group_by_ttl[description.result_column] = new_ttl_info; + data_part->ttl_infos.group_by_ttl[ttl_update_key] = new_ttl_info; data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); } - else if (description.mode == TTLMode::DELETE) + else if (ttl_update_type == TTLUpdateType::ROWS_WHERE_TTL) { - if (description.where_expression) - data_part->ttl_infos.rows_where_ttl[description.result_column] = new_ttl_info; - else - data_part->ttl_infos.table_ttl = new_ttl_info; - + data_part->ttl_infos.rows_where_ttl[ttl_update_key] = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); + } + else if (ttl_update_type == TTLUpdateType::TABLE_TTL) + { + data_part->ttl_infos.table_ttl = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); + } + else if (ttl_update_type == TTLUpdateType::COLUMNS_TTL) + { + data_part->ttl_infos.columns_ttl[ttl_update_key] = new_ttl_info; data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); } diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.h b/src/DataStreams/TTLUpdateInfoAlgorithm.h index c0c4dcea755..5210b3c40c9 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.h +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.h @@ -5,14 +5,35 @@ namespace DB { +enum class TTLUpdateType +{ + COLUMNS_TTL, + TABLE_TTL, + ROWS_WHERE_TTL, + MOVES_TTL, + RECOMPRESSION_TTL, + GROUP_BY_TTL, +}; + /// Calculates new ttl_info and does nothing with data. class TTLUpdateInfoAlgorithm : public ITTLAlgorithm { public: - TTLUpdateInfoAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + TTLUpdateInfoAlgorithm( + const TTLDescription & description_, + const TTLUpdateType ttl_update_type_, + const String ttl_update_key_, + const TTLInfo & old_ttl_info_, + time_t current_time_, bool force_ + ); void execute(Block & block) override; void finalize(const MutableDataPartPtr & data_part) const override; + +private: + const TTLUpdateType ttl_update_type; + const String ttl_update_key; }; + } diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index a5a7355f912..9a815aaca7e 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -169,15 +169,15 @@ def test_modify_ttl(started_cluster): node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) node1.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 4 HOUR SETTINGS mutations_sync = 2") - time.sleep(5) # TTL merges shall happen. + time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "2\n3\n" node2.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 2 HOUR SETTINGS mutations_sync = 2") - time.sleep(5) # TTL merges shall happen. + time.sleep(6) # TTL merges shall happen. assert node1.query("SELECT id FROM test_ttl") == "3\n" node1.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 30 MINUTE SETTINGS mutations_sync = 2") - time.sleep(5) # TTL merges shall happen. + time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "" @@ -196,15 +196,15 @@ def test_modify_column_ttl(started_cluster): node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 4 HOUR SETTINGS mutations_sync = 2") - time.sleep(5) # TTL merges shall happen. + time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "42\n2\n3\n" node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 2 HOUR SETTINGS mutations_sync = 2") - time.sleep(5) # TTL merges shall happen. + time.sleep(6) # TTL merges shall happen. assert node1.query("SELECT id FROM test_ttl") == "42\n42\n3\n" node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 30 MINUTE SETTINGS mutations_sync = 2") - time.sleep(5) # TTL merges shall happen. + time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "42\n42\n42\n" @@ -328,7 +328,7 @@ def test_ttl_empty_parts(started_cluster): node.query("SYSTEM START TTL MERGES test_ttl_empty_parts") optimize_with_retry(node1, 'test_ttl_empty_parts') - assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_4_1_6\n" + assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_5_1_6\n" assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "3000\n" diff --git a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql index f40ed70caef..83b2175a41e 100644 --- a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql +++ b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql @@ -10,6 +10,9 @@ SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = ALTER TABLE ttl_empty_parts MODIFY TTL d; +SELECT sleep(3) format Null; +SELECT sleep(3) format Null; + -- To be sure, that task, which clears outdated parts executed. DETACH TABLE ttl_empty_parts; ATTACH TABLE ttl_empty_parts; From 8a6455116afcb435a07de7a89cbd859272ba4f76 Mon Sep 17 00:00:00 2001 From: terrylin Date: Wed, 4 Aug 2021 22:15:11 +0800 Subject: [PATCH 013/127] materialize ttl recalculate only optional --- tests/integration/test_ttl_replicated/test.py | 21 ++----- .../01070_materialize_ttl.reference | 25 --------- .../0_stateless/01070_materialize_ttl.sql | 56 +++++-------------- .../0_stateless/01070_modify_ttl.reference | 29 ---------- .../queries/0_stateless/01070_modify_ttl.sql | 51 +++-------------- ...1070_mutations_with_dependencies.reference | 11 ---- .../01070_mutations_with_dependencies.sql | 27 ++------- ..._alter_rename_with_ttl_zookeeper.reference | 1 - .../01378_alter_rename_with_ttl_zookeeper.sql | 14 +---- .../01560_ttl_remove_empty_parts.sql | 3 - 10 files changed, 33 insertions(+), 205 deletions(-) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 9a815aaca7e..f37c28b2a80 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -161,7 +161,7 @@ def test_modify_ttl(started_cluster): ''' CREATE TABLE test_ttl(d DateTime, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_modify', '{replica}') - ORDER BY id SETTINGS merge_with_ttl_timeout=0; + ORDER BY id '''.format(replica=node.name)) node1.query( @@ -169,15 +169,12 @@ def test_modify_ttl(started_cluster): node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) node1.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 4 HOUR SETTINGS mutations_sync = 2") - time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "2\n3\n" node2.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 2 HOUR SETTINGS mutations_sync = 2") - time.sleep(6) # TTL merges shall happen. assert node1.query("SELECT id FROM test_ttl") == "3\n" node1.query("ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 30 MINUTE SETTINGS mutations_sync = 2") - time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "" @@ -188,7 +185,7 @@ def test_modify_column_ttl(started_cluster): ''' CREATE TABLE test_ttl(d DateTime, id UInt32 DEFAULT 42) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_column', '{replica}') - ORDER BY d SETTINGS merge_with_ttl_timeout=0; + ORDER BY d '''.format(replica=node.name)) node1.query( @@ -196,15 +193,12 @@ def test_modify_column_ttl(started_cluster): node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 4 HOUR SETTINGS mutations_sync = 2") - time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "42\n2\n3\n" node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 2 HOUR SETTINGS mutations_sync = 2") - time.sleep(6) # TTL merges shall happen. assert node1.query("SELECT id FROM test_ttl") == "42\n42\n3\n" node1.query("ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 30 MINUTE SETTINGS mutations_sync = 2") - time.sleep(6) # TTL merges shall happen. assert node2.query("SELECT id FROM test_ttl") == "42\n42\n42\n" @@ -304,9 +298,7 @@ def test_ttl_empty_parts(started_cluster): SETTINGS max_bytes_to_merge_at_min_space_in_pool = 1, max_bytes_to_merge_at_max_space_in_pool = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 '''.format(replica=node.name)) - node.query("SYSTEM STOP TTL MERGES test_ttl_empty_parts") - for i in range (1, 7): node1.query("INSERT INTO test_ttl_empty_parts SELECT '2{}00-01-0{}', number FROM numbers(1000)".format(i % 2, i)) @@ -316,21 +308,18 @@ def test_ttl_empty_parts(started_cluster): node1.query("ALTER TABLE test_ttl_empty_parts MODIFY TTL date") - assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "6000\n" + assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "3000\n" time.sleep(3) # Wait for cleanup thread assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == \ - "all_0_0_0_6\nall_1_1_0_6\nall_2_2_0_6\nall_3_3_0_6\nall_4_4_0_6\nall_5_5_0_6\n" + "all_0_0_0_6\nall_2_2_0_6\nall_4_4_0_6\n" for node in [node1, node2]: node.query("ALTER TABLE test_ttl_empty_parts MODIFY SETTING max_bytes_to_merge_at_min_space_in_pool = 1000000000") node.query("ALTER TABLE test_ttl_empty_parts MODIFY SETTING max_bytes_to_merge_at_max_space_in_pool = 1000000000") - node.query("SYSTEM START TTL MERGES test_ttl_empty_parts") optimize_with_retry(node1, 'test_ttl_empty_parts') - assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_5_1_6\n" - - assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "3000\n" + assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_4_1_6\n" # Check that after removing empty parts mutations and merges works node1.query("INSERT INTO test_ttl_empty_parts SELECT '2100-01-20', number FROM numbers(1000)") diff --git a/tests/queries/0_stateless/01070_materialize_ttl.reference b/tests/queries/0_stateless/01070_materialize_ttl.reference index 12906a246e3..af1b3a4459b 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.reference +++ b/tests/queries/0_stateless/01070_materialize_ttl.reference @@ -2,33 +2,15 @@ 2000-10-10 2 2100-10-10 3 2100-10-10 4 -2000-10-10 1 -2000-10-10 2 -2100-10-10 3 -2100-10-10 4 2100-10-10 3 2100-10-10 4 1 a -2 b -3 c -4 d -1 a -3 c -1 a 3 c 1 a 2 b 3 c 4 d 1 a -2 b -3 c -4 d -1 a -2 -3 c -4 -1 a 2 3 c 4 @@ -38,13 +20,6 @@ 4 1 a 2 b -3 c -4 d -1 a -2 b -4 d -1 a -2 b 4 d 1 2 diff --git a/tests/queries/0_stateless/01070_materialize_ttl.sql b/tests/queries/0_stateless/01070_materialize_ttl.sql index 19684f028a7..2521ae35edf 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.sql +++ b/tests/queries/0_stateless/01070_materialize_ttl.sql @@ -1,92 +1,62 @@ -set mutations_sync = 2; -set materialize_ttl_after_modify = 0; - drop table if exists ttl; -create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); +set materialize_ttl_after_modify = 0; + alter table ttl materialize ttl; -- { serverError 80 } alter table ttl modify ttl d + interval 1 day; -- TTL should not be applied select * from ttl order by a; -alter table ttl materialize ttl; -select * from ttl order by a; - -optimize table ttl final; +alter table ttl materialize ttl settings mutations_sync=2; select * from ttl order by a; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, s String) engine = MergeTree order by i; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); -alter table ttl materialize ttl; -select * from ttl order by i; - -optimize table ttl final; +alter table ttl materialize ttl settings mutations_sync=2; select * from ttl order by i; alter table ttl modify ttl toDate('2000-01-01'); -alter table ttl materialize ttl; -select * from ttl order by i; - -optimize table ttl final; +alter table ttl materialize ttl settings mutations_sync=2; select * from ttl order by i; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, s String) engine = MergeTree order by i; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); -- TTL should not be applied select * from ttl order by i; -alter table ttl materialize ttl; -select * from ttl order by i; - -optimize table ttl final; +alter table ttl materialize ttl settings mutations_sync=2; select * from ttl order by i; alter table ttl modify column s String ttl toDate('2000-01-01'); -alter table ttl materialize ttl; -select * from ttl order by i; - -optimize table ttl final; +alter table ttl materialize ttl settings mutations_sync=2; select * from ttl order by i; drop table if exists ttl; -create table ttl (d Date, i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (d Date, i Int, s String) engine = MergeTree order by i; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); -alter table ttl materialize ttl; -select i, s from ttl order by i; - -optimize table ttl final; +alter table ttl materialize ttl settings mutations_sync=2; select i, s from ttl order by i; alter table ttl modify column s String ttl d + interval 1 month; -alter table ttl materialize ttl; -select i, s from ttl order by i; - -optimize table ttl final; +alter table ttl materialize ttl settings mutations_sync=2; select i, s from ttl order by i; drop table if exists ttl; diff --git a/tests/queries/0_stateless/01070_modify_ttl.reference b/tests/queries/0_stateless/01070_modify_ttl.reference index 8bfefa38ae1..d64c1a4edc2 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.reference +++ b/tests/queries/0_stateless/01070_modify_ttl.reference @@ -1,33 +1,15 @@ -2000-10-10 1 -2000-10-10 2 -2100-10-10 3 -2100-10-10 4 2100-10-10 3 2100-10-10 4 ============= 1 a -2 b -3 c -4 d -1 a 3 c ============= -1 a -3 c ============= 1 a -2 b -3 c -4 d -1 a 2 3 c 4 ============= -1 a -2 -3 c -4 1 2 3 @@ -35,23 +17,12 @@ ============= 1 a 2 b -3 c -4 d -1 a -2 b 4 d ============= -1 a -2 b -4 d 1 2 4 d ============= -1 a aa -2 b bb -3 c cc -4 d dd 1 a 2 b bb 3 cc diff --git a/tests/queries/0_stateless/01070_modify_ttl.sql b/tests/queries/0_stateless/01070_modify_ttl.sql index f4228b886ba..4e842948afe 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.sql +++ b/tests/queries/0_stateless/01070_modify_ttl.sql @@ -1,98 +1,65 @@ drop table if exists ttl; -set mutations_sync = 2; - -create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); -alter table ttl modify ttl d + interval 1 day; -select * from ttl order by a; +set mutations_sync = 2; -optimize table ttl final; +alter table ttl modify ttl d + interval 1 day; select * from ttl order by a; select '============='; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, s String) engine = MergeTree order by i; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; - -optimize table ttl final; -select * from ttl order by i; select '============='; alter table ttl modify ttl toDate('2000-01-01'); select * from ttl order by i; - -optimize table ttl final; -select * from ttl order by i; select '============='; drop table if exists ttl; -create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, s String) engine = MergeTree order by i; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); select * from ttl order by i; - -optimize table ttl final; -select * from ttl order by i; select '============='; alter table ttl modify column s String ttl toDate('2000-01-01'); select * from ttl order by i; - -optimize table ttl final; -select * from ttl order by i; select '============='; drop table if exists ttl; -create table ttl (d Date, i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (d Date, i Int, s String) engine = MergeTree order by i; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); select i, s from ttl order by i; - -optimize table ttl final; -select i, s from ttl order by i; select '============='; alter table ttl modify column s String ttl d + interval 1 month; select i, s from ttl order by i; - -optimize table ttl final; -select i, s from ttl order by i; select '============='; drop table if exists ttl; -create table ttl (i Int, s String, t String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, s String, t String) engine = MergeTree order by i; insert into ttl values (1, 'a', 'aa') (2, 'b', 'bb') (3, 'c', 'cc') (4, 'd', 'dd'); alter table ttl modify column s String ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'), modify column t String ttl i % 3 = 1 ? today() - 10 : toDate('2100-01-01'); -select i, s, t from ttl order by i; -optimize table ttl final; select i, s, t from ttl order by i; - -- MATERIALIZE TTL ran only once select count() from system.mutations where table = 'ttl' and is_done; select '============='; @@ -100,9 +67,7 @@ select '============='; drop table if exists ttl; -- Nothing changed, don't run mutation -create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i; alter table ttl modify column s String ttl toDate('2000-01-02'); select count() from system.mutations where table = 'ttl' and is_done; diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.reference b/tests/queries/0_stateless/01070_mutations_with_dependencies.reference index dc411d6b68e..eeb32eab7a5 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.reference +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.reference @@ -3,20 +3,9 @@ 3 1 c 4 1 d 1 1 a -2 1 b -3 1 c -4 1 d -1 1 a -2 0 b -3 1 c -4 0 d -1 1 a 3 1 c =================== 4 -4 -0 2 2 -1 2 3 2000-01-01 1 2 42 2000-01-01 diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql index f56f25b5e77..10077a94c9d 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql @@ -2,62 +2,43 @@ drop table if exists ttl; set mutations_sync = 2; -- check that ttl info was updated after mutation. -create table ttl (i Int, a Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, a Int, s String) engine = MergeTree order by i; insert into ttl values (1, 1, 'a') (2, 1, 'b') (3, 1, 'c') (4, 1, 'd'); alter table ttl modify ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'); alter table ttl materialize ttl; -select * from ttl order by i; -optimize table ttl final; select * from ttl order by i; - alter table ttl update a = 0 where i % 2 = 0; select * from ttl order by i; -optimize table ttl final; -select * from ttl order by i; - drop table ttl; select '==================='; -- check that skip index is updated after column was modified by ttl. create table ttl (i Int, a Int, s String default 'b' ttl a % 2 = 0 ? today() - 10 : toDate('2100-01-01'), - index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i - SETTINGS max_number_of_merges_with_ttl_in_pool=0; - + index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i; insert into ttl values (1, 1, 'a') (2, 1, 'a') (3, 1, 'a') (4, 1, 'a'); select count() from ttl where s = 'a'; alter table ttl update a = 0 where i % 2 = 0; -select count() from ttl where s = 'a'; -select count() from ttl where s = 'b'; -optimize table ttl final; select count() from ttl where s = 'a'; select count() from ttl where s = 'b'; drop table ttl; -- check only that it doesn't throw exceptions. -create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default' -SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default'; alter table ttl materialize ttl; drop table ttl; create table ttl (a Int, b Int, c Int default 42 ttl d, d Date, index ind (b * c) type minmax granularity 1) -engine = MergeTree order by a SETTINGS max_number_of_merges_with_ttl_in_pool=0; - +engine = MergeTree order by a; insert into ttl values (1, 2, 3, '2100-01-01'); alter table ttl update d = '2000-01-01' where 1; alter table ttl materialize ttl; select * from ttl; - -optimize table ttl final; -select * from ttl; drop table ttl; diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference index 204c1900fd5..bf8f7658af4 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference @@ -1,4 +1,3 @@ 9 9 -9 0 diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 7b7a3767b1f..0cd6feb9da1 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -1,7 +1,4 @@ DROP TABLE IF EXISTS table_rename_with_ttl; -SET replication_alter_partitions_sync = 2; -SET mutations_sync = 2; -SET materialize_ttl_after_modify = 0; CREATE TABLE table_rename_with_ttl ( @@ -9,25 +6,20 @@ CREATE TABLE table_rename_with_ttl value1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl_01378', '1') -ORDER BY tuple() -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +ORDER BY tuple(); INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); SELECT count() FROM table_rename_with_ttl; +SET materialize_ttl_after_modify = 0; ALTER TABLE table_rename_with_ttl MODIFY TTL date1 + INTERVAL 1 MONTH; SELECT count() FROM table_rename_with_ttl; ALTER TABLE table_rename_with_ttl RENAME COLUMN date1 TO renamed_date1; -ALTER TABLE table_rename_with_ttl materialize TTL; - -SELECT count() FROM table_rename_with_ttl; - -ALTER TABLE table_rename_with_ttl modify setting max_number_of_merges_with_ttl_in_pool=2; -optimize table table_rename_with_ttl FINAL; +ALTER TABLE table_rename_with_ttl materialize TTL settings mutations_sync=2; SELECT count() FROM table_rename_with_ttl; diff --git a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql index 83b2175a41e..f40ed70caef 100644 --- a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql +++ b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql @@ -10,9 +10,6 @@ SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = ALTER TABLE ttl_empty_parts MODIFY TTL d; -SELECT sleep(3) format Null; -SELECT sleep(3) format Null; - -- To be sure, that task, which clears outdated parts executed. DETACH TABLE ttl_empty_parts; ATTACH TABLE ttl_empty_parts; From 88b9e7f24f663ece5e8e82a177a3f0d86a088d57 Mon Sep 17 00:00:00 2001 From: terrylin Date: Wed, 4 Aug 2021 22:16:13 +0800 Subject: [PATCH 014/127] materialize ttl recalculate only optional --- src/Core/Settings.h | 1 + src/DataStreams/TTLBlockInputStream.cpp | 6 +- src/DataStreams/TTLCalcInputStream.cpp | 12 +- src/DataStreams/TTLColumnAlgorithm.cpp | 6 +- src/DataStreams/TTLColumnAlgorithm.h | 5 +- src/DataStreams/TTLUpdateInfoAlgorithm.cpp | 16 +-- src/DataStreams/TTLUpdateInfoAlgorithm.h | 6 +- src/Interpreters/MutationsInterpreter.cpp | 97 +++++++++++++--- src/Interpreters/MutationsInterpreter.h | 16 +++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 77 ++++++++----- .../MergeTree/MergeTreeDataMergerMutator.h | 15 ++- .../01070_modify_ttl_recalc_only.reference | 68 +++++++++++ .../01070_modify_ttl_recalc_only.sql | 108 ++++++++++++++++++ 13 files changed, 362 insertions(+), 71 deletions(-) create mode 100644 tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference create mode 100644 tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1a5cf0fd87d..4b84e944394 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -429,6 +429,7 @@ class IColumn; M(Bool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ + M(Bool, materialize_ttl_recalculate_only, false, "only recalculate ttl info when MATERIALIZE TTL", 0) \ M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index fc557bccad1..05d4ba0a395 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -76,17 +76,17 @@ TTLBlockInputStream::TTLBlockInputStream( algorithms.emplace_back(std::make_unique( description, old_ttl_infos.columns_ttl[name], current_time_, - force_, name, default_expression, default_column_name)); + force_, name, default_expression, default_column_name, isCompactPart(data_part))); } } for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) algorithms.emplace_back(std::make_unique( - move_ttl, TTLUpdateType::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); + move_ttl, TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) algorithms.emplace_back(std::make_unique( - recompression_ttl, TTLUpdateType::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); + recompression_ttl, TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } Block reorderColumns(Block block, const Block & header) diff --git a/src/DataStreams/TTLCalcInputStream.cpp b/src/DataStreams/TTLCalcInputStream.cpp index 82b17433b77..2353e9ec259 100644 --- a/src/DataStreams/TTLCalcInputStream.cpp +++ b/src/DataStreams/TTLCalcInputStream.cpp @@ -22,33 +22,33 @@ TTLCalcInputStream::TTLCalcInputStream( { const auto & rows_ttl = metadata_snapshot_->getRowsTTL(); algorithms.emplace_back(std::make_unique( - rows_ttl, TTLUpdateType::TABLE_TTL, rows_ttl.result_column, old_ttl_infos.table_ttl, current_time_, force_)); + rows_ttl, TTLUpdateField::TABLE_TTL, rows_ttl.result_column, old_ttl_infos.table_ttl, current_time_, force_)); } for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs()) algorithms.emplace_back(std::make_unique( - where_ttl, TTLUpdateType::ROWS_WHERE_TTL, where_ttl.result_column, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); + where_ttl, TTLUpdateField::ROWS_WHERE_TTL, where_ttl.result_column, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) algorithms.emplace_back(std::make_unique( - group_by_ttl, TTLUpdateType::GROUP_BY_TTL, group_by_ttl.result_column, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_)); + group_by_ttl, TTLUpdateField::GROUP_BY_TTL, group_by_ttl.result_column, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_)); if (metadata_snapshot_->hasAnyColumnTTL()) { for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs()) { algorithms.emplace_back(std::make_unique( - description, TTLUpdateType::COLUMNS_TTL, name, old_ttl_infos.columns_ttl[name], current_time_, force_)); + description, TTLUpdateField::COLUMNS_TTL, name, old_ttl_infos.columns_ttl[name], current_time_, force_)); } } for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) algorithms.emplace_back(std::make_unique( - move_ttl, TTLUpdateType::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); + move_ttl, TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) algorithms.emplace_back(std::make_unique( - recompression_ttl, TTLUpdateType::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); + recompression_ttl, TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } Block TTLCalcInputStream::readImpl() diff --git a/src/DataStreams/TTLColumnAlgorithm.cpp b/src/DataStreams/TTLColumnAlgorithm.cpp index 1318ea382db..71ad2a4e38f 100644 --- a/src/DataStreams/TTLColumnAlgorithm.cpp +++ b/src/DataStreams/TTLColumnAlgorithm.cpp @@ -10,11 +10,13 @@ TTLColumnAlgorithm::TTLColumnAlgorithm( bool force_, const String & column_name_, const ExpressionActionsPtr & default_expression_, - const String & default_column_name_) + const String & default_column_name_, + bool is_compact_part_) : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) , column_name(column_name_) , default_expression(default_expression_) , default_column_name(default_column_name_) + , is_compact_part(is_compact_part_) { if (!isMinTTLExpired()) { @@ -40,7 +42,7 @@ void TTLColumnAlgorithm::execute(Block & block) return; /// Later drop full column - if (isMaxTTLExpired()) + if (isMaxTTLExpired() && !is_compact_part) return; auto default_column = executeExpressionAndGetColumn(default_expression, block, default_column_name); diff --git a/src/DataStreams/TTLColumnAlgorithm.h b/src/DataStreams/TTLColumnAlgorithm.h index e09dd663af0..ddf963eaee2 100644 --- a/src/DataStreams/TTLColumnAlgorithm.h +++ b/src/DataStreams/TTLColumnAlgorithm.h @@ -17,7 +17,9 @@ public: bool force_, const String & column_name_, const ExpressionActionsPtr & default_expression_, - const String & default_column_name_); + const String & default_column_name_, + bool is_compact_part_ + ); void execute(Block & block) override; void finalize(const MutableDataPartPtr & data_part) const override; @@ -28,6 +30,7 @@ private: const String default_column_name; bool is_fully_empty = true; + bool is_compact_part; }; } diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp index 21e36f1361c..6a983d052c1 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp @@ -5,13 +5,13 @@ namespace DB TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm( const TTLDescription & description_, - const TTLUpdateType ttl_update_type_, + const TTLUpdateField ttl_update_field_, const String ttl_update_key_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) - , ttl_update_type(ttl_update_type_) + , ttl_update_field(ttl_update_field_) , ttl_update_key(ttl_update_key_) { } @@ -31,30 +31,30 @@ void TTLUpdateInfoAlgorithm::execute(Block & block) void TTLUpdateInfoAlgorithm::finalize(const MutableDataPartPtr & data_part) const { - if (ttl_update_type == TTLUpdateType::RECOMPRESSION_TTL) + if (ttl_update_field == TTLUpdateField::RECOMPRESSION_TTL) { data_part->ttl_infos.recompression_ttl[ttl_update_key] = new_ttl_info; } - else if (ttl_update_type == TTLUpdateType::MOVES_TTL) + else if (ttl_update_field == TTLUpdateField::MOVES_TTL) { data_part->ttl_infos.moves_ttl[ttl_update_key] = new_ttl_info; } - else if (ttl_update_type == TTLUpdateType::GROUP_BY_TTL) + else if (ttl_update_field == TTLUpdateField::GROUP_BY_TTL) { data_part->ttl_infos.group_by_ttl[ttl_update_key] = new_ttl_info; data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); } - else if (ttl_update_type == TTLUpdateType::ROWS_WHERE_TTL) + else if (ttl_update_field == TTLUpdateField::ROWS_WHERE_TTL) { data_part->ttl_infos.rows_where_ttl[ttl_update_key] = new_ttl_info; data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); } - else if (ttl_update_type == TTLUpdateType::TABLE_TTL) + else if (ttl_update_field == TTLUpdateField::TABLE_TTL) { data_part->ttl_infos.table_ttl = new_ttl_info; data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); } - else if (ttl_update_type == TTLUpdateType::COLUMNS_TTL) + else if (ttl_update_field == TTLUpdateField::COLUMNS_TTL) { data_part->ttl_infos.columns_ttl[ttl_update_key] = new_ttl_info; data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.h b/src/DataStreams/TTLUpdateInfoAlgorithm.h index 5210b3c40c9..551211fc47f 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.h +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.h @@ -5,7 +5,7 @@ namespace DB { -enum class TTLUpdateType +enum class TTLUpdateField { COLUMNS_TTL, TABLE_TTL, @@ -21,7 +21,7 @@ class TTLUpdateInfoAlgorithm : public ITTLAlgorithm public: TTLUpdateInfoAlgorithm( const TTLDescription & description_, - const TTLUpdateType ttl_update_type_, + const TTLUpdateField ttl_update_field_, const String ttl_update_key_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_ @@ -31,7 +31,7 @@ public: void finalize(const MutableDataPartPtr & data_part) const override; private: - const TTLUpdateType ttl_update_type; + const TTLUpdateField ttl_update_field; const String ttl_update_key; }; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6347d32a241..47e7b4e2caf 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -150,6 +150,28 @@ ASTPtr prepareQueryAffectedAST(const std::vector & commands, co return select; } +ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns) +{ + NameSet new_updated_columns = updated_columns; + ColumnDependencies dependencies; + while (!new_updated_columns.empty()) + { + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true); + new_updated_columns.clear(); + for (const auto & dependency : new_dependencies) + { + if (!dependencies.count(dependency)) + { + dependencies.insert(dependency); + if (!dependency.isReadOnly()) + new_updated_columns.insert(dependency.column_name); + } + } + } + + return dependencies; +} + } @@ -372,8 +394,13 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) NamesAndTypesList all_columns = columns_desc.getAllPhysical(); NameSet updated_columns; + bool materialize_ttl_recalculate_only = context->getSettingsRef().materialize_ttl_recalculate_only; for (const MutationCommand & command : commands) { + if (command.type == MutationCommand::Type::UPDATE + || command.type == MutationCommand::Type::DELETE) + materialize_ttl_recalculate_only = false; + for (const auto & kv : command.column_to_update_expression) { updated_columns.insert(kv.first); @@ -402,13 +429,14 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) validateUpdateColumns(storage, metadata_snapshot, updated_columns, column_to_affected_materialized); } - dependencies = metadata_snapshot->getColumnDependencies(updated_columns, false); + dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); /// First, break a sequence of commands into stages. for (auto & command : commands) { if (command.type == MutationCommand::DELETE) { + mutation_kind.set(MutationKind::MUTATE_OTHER); if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); @@ -417,6 +445,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::UPDATE) { + mutation_kind.set(MutationKind::MUTATE_OTHER); if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); if (stages.size() == 1) /// First stage only supports filtering and can't update columns. @@ -507,6 +536,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::MATERIALIZE_INDEX) { + mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); auto it = std::find_if( std::cbegin(indices_desc), std::end(indices_desc), [&](const IndexDescription & index) @@ -525,6 +555,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) { + mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); for (const auto & column : projection.required_columns) dependencies.emplace(column, ColumnDependency::PROJECTION); @@ -532,22 +563,59 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::DROP_INDEX) { + mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); materialized_indices.erase(command.index_name); } else if (command.type == MutationCommand::DROP_PROJECTION) { + mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); materialized_projections.erase(command.projection_name); } else if (command.type == MutationCommand::MATERIALIZE_TTL) { - // just recalculate ttl_infos without actual mutation - auto all_columns_vec = all_columns.getNames(); - auto new_dependencies = metadata_snapshot->getColumnDependencies(NameSet(all_columns_vec.begin(), all_columns_vec.end()), false); - - for (const auto & dependency : new_dependencies) + mutation_kind.set(MutationKind::MUTATE_OTHER); + if (materialize_ttl_recalculate_only) { - if (dependency.kind == ColumnDependency::TTL_EXPRESSION) - dependencies.insert(dependency); + // just recalculate ttl_infos without remove expired data + auto all_columns_vec = all_columns.getNames(); + auto new_dependencies = metadata_snapshot->getColumnDependencies(NameSet(all_columns_vec.begin(), all_columns_vec.end()), false); + for (const auto & dependency : new_dependencies) + { + if (dependency.kind == ColumnDependency::TTL_EXPRESSION) + dependencies.insert(dependency); + } + } + else if (metadata_snapshot->hasRowsTTL()) + { + for (const auto & column : all_columns) + dependencies.emplace(column.name, ColumnDependency::TTL_TARGET); + } + else + { + NameSet new_updated_columns; + auto column_ttls = metadata_snapshot->getColumns().getColumnTTLs(); + for (const auto & elem : column_ttls) + { + dependencies.emplace(elem.first, ColumnDependency::TTL_TARGET); + new_updated_columns.insert(elem.first); + } + + auto all_columns_vec = all_columns.getNames(); + auto all_dependencies = getAllColumnDependencies(metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end())); + + for (const auto & dependency : all_dependencies) + { + if (dependency.kind == ColumnDependency::TTL_EXPRESSION) + dependencies.insert(dependency); + } + + /// Recalc only skip indices and projections of columns which could be updated by TTL. + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true); + for (const auto & dependency : new_dependencies) + { + if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION) + dependencies.insert(dependency); + } } if (dependencies.empty()) @@ -559,6 +627,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::READ_COLUMN) { + mutation_kind.set(MutationKind::MUTATE_OTHER); if (stages.empty() || !stages.back().column_to_updated.empty()) stages.emplace_back(context); if (stages.size() == 1) /// First stage only supports filtering and can't update columns. @@ -585,7 +654,6 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) changed_columns.insert(dependency.column_name); } - // changed_columns is always empty because we don't delete or aggregate expired data here if (!changed_columns.empty()) { if (stages.empty() || !stages.back().column_to_updated.empty()) @@ -624,11 +692,6 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) auto pipeline = addStreamsForLaterStages(stages_copy, plan); updated_header = std::make_unique(pipeline->getHeader()); } - else - { - //no column updated in mutations. maybe just materialize(index\projection\ttl) - updated_header = std::make_unique(Block{}); - } /// Special step to recalculate affected indices, projections and TTL expressions. stages.emplace_back(context); @@ -934,4 +997,10 @@ bool MutationsInterpreter::isAffectingAllColumns() const return stages.back().isAffectingAllColumns(storage_columns); } +void MutationsInterpreter::MutationKind::set(const MutationKindEnum & kind) +{ + if (mutation_kind < kind) + mutation_kind = kind; +} + } diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index a2e11e527fc..c9a589e6b6d 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -64,6 +64,20 @@ public: NameSet grabMaterializedProjections() { return std::move(materialized_projections); } + struct MutationKind + { + enum MutationKindEnum + { + MUTATE_UNKNOWN, + MUTATE_INDEX_PROJECTION, + MUTATE_OTHER, + } mutation_kind = MUTATE_UNKNOWN; + + void set(const MutationKindEnum & kind); + }; + + MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } + private: ASTPtr prepare(bool dry_run); @@ -134,6 +148,8 @@ private: NameSet materialized_indices; NameSet materialized_projections; + MutationKind mutation_kind; /// Do we meet any index or projection mutation. + /// Columns, that we need to read for calculation of skip indices, projections or TTL expressions. ColumnDependencies dependencies; }; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 284622a13da..4ca73a427df 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1250,6 +1250,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NamesAndTypesList storage_columns = metadata_snapshot->getColumns().getAllPhysical(); NameSet materialized_indices; NameSet materialized_projections; + MutationsInterpreter::MutationKind::MutationKindEnum mutation_kind + = MutationsInterpreter::MutationKind::MutationKindEnum::MUTATE_UNKNOWN; if (!for_interpreter.empty()) { @@ -1257,6 +1259,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true); materialized_indices = interpreter->grabMaterializedIndices(); materialized_projections = interpreter->grabMaterializedProjections(); + mutation_kind = interpreter->getMutationKind(); in = interpreter->execute(); updated_header = interpreter->getUpdatedHeader(); in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress)); @@ -1286,14 +1289,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); bool need_sync = needSyncPart(source_part->rows_count, source_part->getBytesOnDisk(), *data_settings); - bool need_recalculate_ttl = false; + auto execute_ttl_type = ExecuteTTLType::NONE; - if (in && shouldExecuteTTL(metadata_snapshot, interpreter->getColumnDependencies(), commands_for_part)) - need_recalculate_ttl = true; + if (in) + execute_ttl_type = shouldExecuteTTL(metadata_snapshot, interpreter->getColumnDependencies(), commands_for_part); /// All columns from part are changed and may be some more that were missing before in part /// TODO We can materialize compact part without copying data - if (!isWidePart(source_part) || (updated_header && interpreter && interpreter->isAffectingAllColumns())) + if (!isWidePart(source_part) + || (mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && interpreter && interpreter->isAffectingAllColumns())) { disk->createDirectories(new_part_tmp_path); @@ -1316,7 +1320,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_recalculate_ttl, + execute_ttl_type, need_sync, space_reservation, holder, @@ -1329,8 +1333,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { /// We will modify only some of the columns. Other columns and key values can be copied as-is. NameSet updated_columns; - for (const auto & name_type : updated_header.getNamesAndTypesList()) - updated_columns.emplace(name_type.name); + if (mutation_kind != MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION) + { + for (const auto & name_type : updated_header.getNamesAndTypesList()) + updated_columns.emplace(name_type.name); + } auto indices_to_recalc = getIndicesToRecalculate( in, updated_columns, metadata_snapshot, context, materialized_indices, source_part); @@ -1339,21 +1346,21 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameSet files_to_skip = collectFilesToSkip( source_part, - updated_header, + mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, indices_to_recalc, mrk_extension, projections_to_recalc); NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension); - if (indices_to_recalc.empty() && projections_to_recalc.empty() && updated_columns.empty() - && files_to_rename.empty() && !need_recalculate_ttl) + if (indices_to_recalc.empty() && projections_to_recalc.empty() && mutation_kind != MutationsInterpreter::MutationKind::MUTATE_OTHER + && files_to_rename.empty()) { LOG_TRACE( log, "Part {} doesn't change up to mutation version {} (optimized)", source_part->name, future_part.part_info.mutation); return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info, metadata_snapshot); } - if (need_recalculate_ttl) + if (execute_ttl_type != ExecuteTTLType::NONE) files_to_skip.insert("ttl.txt"); disk->createDirectories(new_part_tmp_path); @@ -1407,13 +1414,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor metadata_snapshot, indices_to_recalc, projections_to_recalc, - updated_header, + // If it's an index/projection materialization, we don't write any data columns, thus empty header is used + mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header, new_data_part, in, time_of_mutation, compression_codec, merge_entry, - need_recalculate_ttl, + execute_ttl_type, need_sync, space_reservation, holder, @@ -1434,7 +1442,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } } - finalizeMutatedPart(source_part, new_data_part, need_recalculate_ttl, compression_codec); + finalizeMutatedPart(source_part, new_data_part, execute_ttl_type, compression_codec); } return new_data_part; @@ -1972,21 +1980,24 @@ std::set MergeTreeDataMergerMutator::getProjectionsToRec return projections_to_recalc; } -bool MergeTreeDataMergerMutator::shouldExecuteTTL( +ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL( const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies, const MutationCommands & commands) { if (!metadata_snapshot->hasAnyTTL()) - return false; + return ExecuteTTLType::NONE; - for (const auto & command : commands) - if (command.type == MutationCommand::MATERIALIZE_TTL) - return true; + bool has_ttl_expression; + bool has_ttl_target; for (const auto & dependency : dependencies) - if (dependency.kind == ColumnDependency::TTL_EXPRESSION || dependency.kind == ColumnDependency::TTL_TARGET) - return true; - - return false; + { + if (dependency.kind == ColumnDependency::TTL_EXPRESSION) + has_ttl_expression = true; + + if (dependency.kind == ColumnDependency::TTL_TARGET) + return ExecuteTTLType::NORMAL; + } + return has_ttl_expression ? ExecuteTTLType::RECALCULATE : ExecuteTTLType::NONE; } // 1. get projection pipeline and a sink to write parts @@ -2160,7 +2171,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_recalculate_ttl, + ExecuteTTLType execute_ttl_type, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -2173,7 +2184,10 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( mutating_stream = std::make_shared( std::make_shared(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression(metadata_snapshot))); - if (need_recalculate_ttl) + if (execute_ttl_type == ExecuteTTLType::NORMAL) + mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); + + if (execute_ttl_type == ExecuteTTLType::RECALCULATE) mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); IMergeTreeDataPart::MinMaxIndex minmax_idx; @@ -2217,7 +2231,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_recalculate_ttl, + ExecuteTTLType execute_ttl_type, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -2226,9 +2240,12 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); - if (need_recalculate_ttl) - mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); + if (execute_ttl_type == ExecuteTTLType::NORMAL) + mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); + if (execute_ttl_type == ExecuteTTLType::RECALCULATE) + mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); + IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( new_data_part, @@ -2267,7 +2284,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( void MergeTreeDataMergerMutator::finalizeMutatedPart( const MergeTreeDataPartPtr & source_part, MergeTreeData::MutableDataPartPtr new_data_part, - bool need_recalculate_ttl, + ExecuteTTLType execute_ttl_type, const CompressionCodecPtr & codec) { auto disk = new_data_part->volume->getDisk(); @@ -2281,7 +2298,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( new_data_part->checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash(); } - if (need_recalculate_ttl) + if (execute_ttl_type != ExecuteTTLType::NONE) { /// Write a file with ttl infos in json format. auto out_ttl = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "ttl.txt", 4096); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index e6ddc7447bc..5687e7dca24 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -23,6 +23,13 @@ enum class SelectPartsDecision NOTHING_TO_MERGE = 2, }; +enum class ExecuteTTLType +{ + NONE = 0, + NORMAL = 1, + RECALCULATE= 2, +}; + /// Auxiliary struct holding metainformation for the future merged or mutated part. struct FutureMergedMutatedPart { @@ -200,7 +207,7 @@ private: const ProjectionsDescription & all_projections, const MutationCommands & commands_for_removes); - static bool shouldExecuteTTL( + static ExecuteTTLType shouldExecuteTTL( const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies, const MutationCommands & commands); /// Return set of indices which should be recalculated during mutation also @@ -242,7 +249,7 @@ private: time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_recalculate_ttl, + ExecuteTTLType execute_ttl_type, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -260,7 +267,7 @@ private: time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_recalculate_ttl, + ExecuteTTLType execute_ttl_type, bool need_sync, const ReservationPtr & space_reservation, TableLockHolder & holder, @@ -271,7 +278,7 @@ private: static void finalizeMutatedPart( const MergeTreeDataPartPtr & source_part, MergeTreeData::MutableDataPartPtr new_data_part, - bool need_recalculate_ttl, + ExecuteTTLType execute_ttl_type, const CompressionCodecPtr & codec); public : diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference new file mode 100644 index 00000000000..1f3ba027e7a --- /dev/null +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference @@ -0,0 +1,68 @@ +2000-10-10 1 +2000-10-10 2 +2100-10-10 3 +2100-10-10 4 +2000-10-11 00:00:00 2000-10-11 00:00:00 +2000-10-11 00:00:00 2000-10-11 00:00:00 +2100-10-11 00:00:00 2100-10-11 00:00:00 +2100-10-11 00:00:00 2100-10-11 00:00:00 +2100-10-10 3 +2100-10-10 4 +============= +1 a +2 b +3 c +4 d +2100-01-01 00:00:00 +1 a +3 c +============= +1 a +3 c +2000-01-01 00:00:00 2000-01-01 00:00:00 +============= +1 a +2 b +3 c +4 d +1 a +2 +3 c +4 +============= +1 a +2 +3 c +4 +1 +2 +3 +4 +============= +1 a +2 b +3 c +4 d +2100-01-01 00:00:00 +1 a +2 b +4 d +============= +1 a +2 b +4 d +1 +2 +4 d +============= +1 a aa +2 b bb +3 c cc +4 d dd +1 a +2 b bb +3 cc +4 d +1 +============= +0 diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql new file mode 100644 index 00000000000..6ee7fe311bb --- /dev/null +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql @@ -0,0 +1,108 @@ +set mutations_sync = 2; +set materialize_ttl_recalculate_only = true; + +drop table if exists ttl; + +create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) +SETTINGS max_number_of_merges_with_ttl_in_pool=0; + +insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); +insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); +insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3); +insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); + + +alter table ttl modify ttl d + interval 1 day; +select * from ttl order by a; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0 order by name asc; +optimize table ttl final; +select * from ttl order by a; +select '============='; + +drop table if exists ttl; + +create table ttl (i Int, s String) engine = MergeTree order by i +SETTINGS max_number_of_merges_with_ttl_in_pool=0; + +insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); + +alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); +select * from ttl order by i; +select delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; +optimize table ttl final; +select * from ttl order by i; +select '============='; + +alter table ttl modify ttl toDate('2000-01-01'); +select * from ttl order by i; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; +optimize table ttl final; +select * from ttl order by i; +select '============='; + +drop table if exists ttl; + +create table ttl (i Int, s String) engine = MergeTree order by i +SETTINGS max_number_of_merges_with_ttl_in_pool=0; + +insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); + +alter table ttl modify column s String ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); +select * from ttl order by i; +optimize table ttl final; +select * from ttl order by i; +select '============='; + +alter table ttl modify column s String ttl toDate('2000-01-01'); +select * from ttl order by i; +optimize table ttl final; +select * from ttl order by i; +select '============='; + +drop table if exists ttl; + +create table ttl (d Date, i Int, s String) engine = MergeTree order by i +SETTINGS max_number_of_merges_with_ttl_in_pool=0; + +insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); + +alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); +select i, s from ttl order by i; +select delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; +optimize table ttl final; +select i, s from ttl order by i; +select '============='; + +alter table ttl modify column s String ttl d + interval 1 month; +select i, s from ttl order by i; +optimize table ttl final; +select i, s from ttl order by i; +select '============='; + +drop table if exists ttl; + +create table ttl (i Int, s String, t String) engine = MergeTree order by i +SETTINGS max_number_of_merges_with_ttl_in_pool=0; + +insert into ttl values (1, 'a', 'aa') (2, 'b', 'bb') (3, 'c', 'cc') (4, 'd', 'dd'); + +alter table ttl modify column s String ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'), + modify column t String ttl i % 3 = 1 ? today() - 10 : toDate('2100-01-01'); + +select i, s, t from ttl order by i; +optimize table ttl final; +select i, s, t from ttl order by i; +-- MATERIALIZE TTL ran only once +select count() from system.mutations where table = 'ttl' and is_done; +select '============='; + +drop table if exists ttl; + +-- Nothing changed, don't run mutation +create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i +SETTINGS max_number_of_merges_with_ttl_in_pool=0; + +alter table ttl modify column s String ttl toDate('2000-01-02'); +select count() from system.mutations where table = 'ttl' and is_done; + +drop table if exists ttl; From 6f42ec6b9b4401580fe14c07455f021b091feacb Mon Sep 17 00:00:00 2001 From: terrylin Date: Thu, 5 Aug 2021 00:16:21 +0800 Subject: [PATCH 015/127] fix compile error --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 +++----- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 3 +-- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4ca73a427df..e0cc6d9742c 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1292,7 +1292,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto execute_ttl_type = ExecuteTTLType::NONE; if (in) - execute_ttl_type = shouldExecuteTTL(metadata_snapshot, interpreter->getColumnDependencies(), commands_for_part); + execute_ttl_type = shouldExecuteTTL(metadata_snapshot, interpreter->getColumnDependencies()); /// All columns from part are changed and may be some more that were missing before in part /// TODO We can materialize compact part without copying data @@ -1980,14 +1980,12 @@ std::set MergeTreeDataMergerMutator::getProjectionsToRec return projections_to_recalc; } -ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL( - const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies, const MutationCommands & commands) +ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies) { if (!metadata_snapshot->hasAnyTTL()) return ExecuteTTLType::NONE; - bool has_ttl_expression; - bool has_ttl_target; + bool has_ttl_expression = false; for (const auto & dependency : dependencies) { diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 5687e7dca24..3a0041e4a37 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -207,8 +207,7 @@ private: const ProjectionsDescription & all_projections, const MutationCommands & commands_for_removes); - static ExecuteTTLType shouldExecuteTTL( - const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies, const MutationCommands & commands); + static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies); /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream From bd3d9a45185d241f975782d3993e52e0f76cc6a7 Mon Sep 17 00:00:00 2001 From: terrylin Date: Thu, 5 Aug 2021 14:17:48 +0800 Subject: [PATCH 016/127] materialize ttl recalculate only (optional) --- src/Core/Settings.h | 1 - src/Interpreters/MutationsInterpreter.cpp | 11 +++++++++- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/StorageFromMergeTreeDataPart.h | 5 +++++ .../01070_modify_ttl_recalc_only.reference | 4 ++-- .../01070_modify_ttl_recalc_only.sql | 21 +++++++++---------- 6 files changed, 28 insertions(+), 15 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4b84e944394..1a5cf0fd87d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -429,7 +429,6 @@ class IColumn; M(Bool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ - M(Bool, materialize_ttl_recalculate_only, false, "only recalculate ttl info when MATERIALIZE TTL", 0) \ M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 47e7b4e2caf..53ec3c81579 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -303,6 +303,15 @@ static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPt return key_columns; } +static bool materializeTTLRecalculateOnly(const StoragePtr & storage) +{ + auto storage_from_merge_tree_data_part = std::dynamic_pointer_cast(storage); + if (!storage_from_merge_tree_data_part) + return false; + + return storage_from_merge_tree_data_part->materializeTTLRecalculateOnly(); +} + static void validateUpdateColumns( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns, @@ -394,7 +403,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) NamesAndTypesList all_columns = columns_desc.getAllPhysical(); NameSet updated_columns; - bool materialize_ttl_recalculate_only = context->getSettingsRef().materialize_ttl_recalculate_only; + bool materialize_ttl_recalculate_only = materializeTTLRecalculateOnly(storage); for (const MutationCommand & command : commands) { if (command.type == MutationCommand::Type::UPDATE diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index d018059c248..fabcad3c390 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -117,6 +117,7 @@ struct Settings; M(Int64, merge_with_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with delete TTL can be repeated.", 0) \ M(Int64, merge_with_recompression_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \ M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ + M(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \ M(Bool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ M(Bool, enable_mixed_granularity_parts, 1, "Enable parts with adaptive and non adaptive granularity", 0) \ M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 15beb94404b..5e7d873af1c 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -72,6 +72,11 @@ public: return parts.front()->storage.getPartitionIDFromQuery(ast, context); } + bool materializeTTLRecalculateOnly() const + { + return parts.front()->storage.getSettings()->materialize_ttl_recalculate_only; + } + protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) : IStorage(getIDFromPart(part_)) diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference index 1f3ba027e7a..fe9cba71c4c 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference @@ -13,7 +13,7 @@ 2 b 3 c 4 d -2100-01-01 00:00:00 +2000-01-01 00:00:00 2100-01-01 00:00:00 1 a 3 c ============= @@ -43,7 +43,7 @@ 2 b 3 c 4 d -2100-01-01 00:00:00 +2000-01-01 00:00:00 2100-01-01 00:00:00 1 a 2 b 4 d diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql index 6ee7fe311bb..a44214e3a02 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql @@ -1,10 +1,9 @@ set mutations_sync = 2; -set materialize_ttl_recalculate_only = true; drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +SETTINGS max_number_of_merges_with_ttl_in_pool=0,materialize_ttl_recalculate_only=true; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); @@ -22,13 +21,13 @@ select '============='; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +SETTINGS max_number_of_merges_with_ttl_in_pool=0,materialize_ttl_recalculate_only=true; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); -alter table ttl modify ttl i % 2 = 0 ? today() - 10 : toDate('2100-01-01'); +alter table ttl modify ttl i % 2 = 0 ? toDate('2000-01-01') : toDate('2100-01-01'); select * from ttl order by i; -select delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; optimize table ttl final; select * from ttl order by i; select '============='; @@ -43,7 +42,7 @@ select '============='; drop table if exists ttl; create table ttl (i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +SETTINGS max_number_of_merges_with_ttl_in_pool=0,materialize_ttl_recalculate_only=true; insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); @@ -62,13 +61,13 @@ select '============='; drop table if exists ttl; create table ttl (d Date, i Int, s String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +SETTINGS max_number_of_merges_with_ttl_in_pool=0,materialize_ttl_recalculate_only=true; insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, 'b') (toDate('2080-01-01'), 3, 'c') (toDate('2080-01-03'), 4, 'd'); -alter table ttl modify ttl i % 3 = 0 ? today() - 10 : toDate('2100-01-01'); +alter table ttl modify ttl i % 3 = 0 ? toDate('2000-01-01') : toDate('2100-01-01'); select i, s from ttl order by i; -select delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; optimize table ttl final; select i, s from ttl order by i; select '============='; @@ -82,7 +81,7 @@ select '============='; drop table if exists ttl; create table ttl (i Int, s String, t String) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +SETTINGS max_number_of_merges_with_ttl_in_pool=0,materialize_ttl_recalculate_only=true; insert into ttl values (1, 'a', 'aa') (2, 'b', 'bb') (3, 'c', 'cc') (4, 'd', 'dd'); @@ -100,7 +99,7 @@ drop table if exists ttl; -- Nothing changed, don't run mutation create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree order by i -SETTINGS max_number_of_merges_with_ttl_in_pool=0; +SETTINGS max_number_of_merges_with_ttl_in_pool=0,materialize_ttl_recalculate_only=true; alter table ttl modify column s String ttl toDate('2000-01-02'); select count() from system.mutations where table = 'ttl' and is_done; From dd539f0a0d9fad49c7694a4822778c7cd6fe0751 Mon Sep 17 00:00:00 2001 From: terrylin Date: Thu, 5 Aug 2021 15:33:02 +0800 Subject: [PATCH 017/127] improvement of materilize ttl --- src/Interpreters/MutationsInterpreter.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- src/Storages/MergeTree/StorageFromMergeTreeDataPart.h | 2 +- .../0_stateless/01070_modify_ttl_recalc_only.reference | 7 ------- tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql | 4 ---- 5 files changed, 5 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 53ec3c81579..fb6946ca4ed 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -308,7 +308,7 @@ static bool materializeTTLRecalculateOnly(const StoragePtr & storage) auto storage_from_merge_tree_data_part = std::dynamic_pointer_cast(storage); if (!storage_from_merge_tree_data_part) return false; - + return storage_from_merge_tree_data_part->materializeTTLRecalculateOnly(); } @@ -409,7 +409,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) if (command.type == MutationCommand::Type::UPDATE || command.type == MutationCommand::Type::DELETE) materialize_ttl_recalculate_only = false; - + for (const auto & kv : command.column_to_update_expression) { updated_columns.insert(kv.first); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e0cc6d9742c..349413b04f7 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1991,7 +1991,7 @@ ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadat { if (dependency.kind == ColumnDependency::TTL_EXPRESSION) has_ttl_expression = true; - + if (dependency.kind == ColumnDependency::TTL_TARGET) return ExecuteTTLType::NORMAL; } @@ -2243,7 +2243,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( if (execute_ttl_type == ExecuteTTLType::RECALCULATE) mutating_stream = std::make_shared(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true); - + IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( new_data_part, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 5e7d873af1c..e214127b819 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -76,7 +76,7 @@ public: { return parts.front()->storage.getSettings()->materialize_ttl_recalculate_only; } - + protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) : IStorage(getIDFromPart(part_)) diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference index fe9cba71c4c..8bfefa38ae1 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference @@ -2,10 +2,6 @@ 2000-10-10 2 2100-10-10 3 2100-10-10 4 -2000-10-11 00:00:00 2000-10-11 00:00:00 -2000-10-11 00:00:00 2000-10-11 00:00:00 -2100-10-11 00:00:00 2100-10-11 00:00:00 -2100-10-11 00:00:00 2100-10-11 00:00:00 2100-10-10 3 2100-10-10 4 ============= @@ -13,13 +9,11 @@ 2 b 3 c 4 d -2000-01-01 00:00:00 2100-01-01 00:00:00 1 a 3 c ============= 1 a 3 c -2000-01-01 00:00:00 2000-01-01 00:00:00 ============= 1 a 2 b @@ -43,7 +37,6 @@ 2 b 3 c 4 d -2000-01-01 00:00:00 2100-01-01 00:00:00 1 a 2 b 4 d diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql index a44214e3a02..5dcea47c4d9 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql @@ -13,7 +13,6 @@ insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); alter table ttl modify ttl d + interval 1 day; select * from ttl order by a; -select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0 order by name asc; optimize table ttl final; select * from ttl order by a; select '============='; @@ -27,14 +26,12 @@ insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? toDate('2000-01-01') : toDate('2100-01-01'); select * from ttl order by i; -select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; optimize table ttl final; select * from ttl order by i; select '============='; alter table ttl modify ttl toDate('2000-01-01'); select * from ttl order by i; -select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; optimize table ttl final; select * from ttl order by i; select '============='; @@ -67,7 +64,6 @@ insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, alter table ttl modify ttl i % 3 = 0 ? toDate('2000-01-01') : toDate('2100-01-01'); select i, s from ttl order by i; -select delete_ttl_info_min, delete_ttl_info_max from system.parts where table = 'ttl' and active > 0; optimize table ttl final; select i, s from ttl order by i; select '============='; From 12b422f82cf2bf99023617e240bc1a5d3861918a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Aug 2021 16:34:40 +0300 Subject: [PATCH 018/127] check replication consistency after 993-like tests --- ...ts_race_condition_zookeeper_long.reference | 2 ++ ...tem_parts_race_condition_zookeeper_long.sh | 23 ++++++------ ...ts_race_condition_drop_zookeeper.reference | 2 ++ ...tem_parts_race_condition_drop_zookeeper.sh | 4 +++ ..._alter_add_drop_column_zookeeper.reference | 2 ++ ...arallel_alter_add_drop_column_zookeeper.sh | 4 +++ ...llel_alter_modify_zookeeper_long.reference | 2 ++ ...79_parallel_alter_modify_zookeeper_long.sh | 4 +++ .../01154_move_partition_long.reference | 4 ++- .../0_stateless/01154_move_partition_long.sh | 11 +++--- ...utations_kill_many_replicas_long.reference | 2 ++ ...alter_mutations_kill_many_replicas_long.sh | 10 +++--- ...and_normal_merges_zookeeper_long.reference | 2 ++ ...nt_ttl_and_normal_merges_zookeeper_long.sh | 7 ++-- .../0_stateless/mergetree_mutations.lib | 20 +++++++++++ tests/queries/0_stateless/replication.lib | 35 +++++++++++++++++++ 16 files changed, 108 insertions(+), 26 deletions(-) create mode 100755 tests/queries/0_stateless/replication.lib diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference index e69de29bb2d..8a6b9c4f877 100644 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference @@ -0,0 +1,2 @@ +Replication did not hang: synced all replicas of alter_table +0 1 diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 793fc8e9575..19f72120912 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -3,15 +3,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib set -e $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS alter_table; - DROP TABLE IF EXISTS alter_table2; + DROP TABLE IF EXISTS alter_table0; + DROP TABLE IF EXISTS alter_table1; - CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0; - CREATE TABLE alter_table2 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 + CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0; + CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 " function thread1() @@ -22,22 +24,22 @@ function thread1() function thread2() { - while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String DEFAULT '0'; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done + while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table0 ADD COLUMN h String DEFAULT '0'; ALTER TABLE alter_table0 MODIFY COLUMN h UInt64; ALTER TABLE alter_table0 DROP COLUMN h;"; done } function thread3() { - while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done + while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table0 SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done } function thread4() { - while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table FINAL"; done + while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table0 FINAL"; done } function thread5() { - while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288"; done + while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288"; done } # https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout @@ -74,8 +76,9 @@ timeout $TIMEOUT bash -c thread4 2> /dev/null & timeout $TIMEOUT bash -c thread5 2> /dev/null & wait +check_replication_consistency "alter_table" "count(), sum(a), sum(b), round(sum(c))" -$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table;" & -$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table2;" & +$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table0;" & +$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table1;" & wait diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference index e69de29bb2d..0d13bb62797 100644 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference @@ -0,0 +1,2 @@ +Replication did not hang: synced all replicas of alter_table_ +0 1 diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index 32fe31f68c6..bdad08fb0e1 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -3,6 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib set -e @@ -99,6 +101,8 @@ timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from Zo wait +check_replication_consistency "alter_table_" "count(), sum(a), sum(b), round(sum(c))" + for i in {0..9}; do $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table_$i" 2>&1 | grep "was not completely removed from ZooKeeper" & done diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference index af33a5bfc3f..34a89ec4d07 100644 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference @@ -1,6 +1,8 @@ Starting alters Finishing alters Equal number of columns +Replication did not hang: synced all replicas of concurrent_alter_add_drop_ +0 1 0 0 0 diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh index fd0b53cf122..4b67a03760b 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh @@ -3,6 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib REPLICAS=3 @@ -101,6 +103,8 @@ while [[ $(timeout 120 ${CLICKHOUSE_CLIENT} --query "ALTER TABLE concurrent_alte sleep 1 done +check_replication_consistency "concurrent_alter_add_drop_" "count(), sum(key), sum(cityHash64(value0))" + for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_add_drop_$i" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i'" diff --git a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference index ff9c6824f00..15223e4fd99 100644 --- a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference +++ b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference @@ -5,6 +5,8 @@ 1725 Starting alters Finishing alters +Replication did not hang: synced all replicas of concurrent_alter_mt_ +0 1 1 0 1 diff --git a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh index 37d880bdce7..acbb01a1c68 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh @@ -3,6 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib REPLICAS=5 @@ -112,6 +114,8 @@ while [[ $(timeout 120 ${CLICKHOUSE_CLIENT} --query "ALTER TABLE concurrent_alte sleep 1 done +check_replication_consistency "concurrent_alter_mt_" "count(), sum(key), sum(cityHash64(value1)), sum(cityHash64(value2))" + for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_mt_$i" $CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_alter_mt_$i" diff --git a/tests/queries/0_stateless/01154_move_partition_long.reference b/tests/queries/0_stateless/01154_move_partition_long.reference index c6d9204ed02..40aaa81456a 100644 --- a/tests/queries/0_stateless/01154_move_partition_long.reference +++ b/tests/queries/0_stateless/01154_move_partition_long.reference @@ -1 +1,3 @@ -Replication did not hang +Replication did not hang: synced all replicas of dst_ +0 1 +Replication did not hang: synced all replicas of src_ diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index 1b5985b9942..541550160f2 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -3,6 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib declare -A engines engines[0]="MergeTree" @@ -116,13 +118,8 @@ timeout $TIMEOUT bash -c optimize_thread & timeout $TIMEOUT bash -c drop_part_thread & wait -for ((i=0; i<16; i++)) do - # The size of log is big, so increase timeout. - $CLICKHOUSE_CLIENT --receive_timeout 600 -q "SYSTEM SYNC REPLICA dst_$i" & - $CLICKHOUSE_CLIENT --receive_timeout 600 -q "SYSTEM SYNC REPLICA src_$i" 2>/dev/null & -done -wait -echo "Replication did not hang" +check_replication_consistency "dst_" "count(), sum(p), sum(k), sum(v)" +try_sync_replicas "src_" for ((i=0; i<16; i++)) do $CLICKHOUSE_CLIENT -q "DROP TABLE dst_$i" 2>&1| grep -Fv "is already started to be removing" & diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference index f7c65e36be4..5a3c0201732 100644 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference @@ -14,3 +14,5 @@ CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\ Metadata version on replica 5 equal with first replica, OK CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 499999500000 +Replication did not hang: synced all replicas of concurrent_kill_ +0 1 diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh index e263750c431..bb04facba15 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh @@ -3,6 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib REPLICAS=5 @@ -59,10 +61,6 @@ timeout $TIMEOUT bash -c kill_mutation_thread 2> /dev/null & wait -for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_kill_$i" -done - # with timeout alter query can be not finished yet, so to execute new alter # we use retries counter=0 @@ -80,7 +78,7 @@ while true; do done -metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r1$i/' and name = 'metadata_version'") +metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r11/' and name = 'metadata_version'") for i in $(seq $REPLICAS); do replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r1$i/' and name = 'metadata_version'") @@ -95,6 +93,8 @@ done $CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM concurrent_kill_1" +check_replication_consistency "concurrent_kill_" "count(), sum(key), sum(cityHash64(value))" + for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_kill_$i" done diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference index d00491fd7e5..25e14257d8d 100644 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference @@ -1 +1,3 @@ +Replication did not hang: synced all replicas of ttl_table +0 1 1 diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index 13086879e0d..3daab1e9fdd 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -3,6 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib NUM_REPLICAS=5 @@ -59,13 +61,12 @@ timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & wait -for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA ttl_table$i" -done +check_replication_consistency "ttl_table" "count(), sum(toUInt64(key))" $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue where table like 'ttl_table%' and database = '${CLICKHOUSE_DATABASE}' and type='MERGE_PARTS' and last_exception != '' FORMAT Vertical" $CLICKHOUSE_CLIENT --query "SELECT COUNT() > 0 FROM system.part_log where table like 'ttl_table%' and database = '${CLICKHOUSE_DATABASE}'" + for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_table$i" & done diff --git a/tests/queries/0_stateless/mergetree_mutations.lib b/tests/queries/0_stateless/mergetree_mutations.lib index d10ac883764..ffd8dce78fd 100644 --- a/tests/queries/0_stateless/mergetree_mutations.lib +++ b/tests/queries/0_stateless/mergetree_mutations.lib @@ -20,3 +20,23 @@ function wait_for_mutation() done } + +function wait_for_all_mutations() +{ + local table=$1 + local database=$2 + database=${database:="${CLICKHOUSE_DATABASE}"} + + for i in {1..200} + do + sleep 1 + if [[ $(${CLICKHOUSE_CLIENT} --query="SELECT coalesce(minOrNull(is_done), 1) FROM system.mutations WHERE database='$database' AND table like '$table'") -eq 1 ]]; then + break + fi + + if [[ $i -eq 200 ]]; then + echo "Timed out while waiting for mutation to execute!" | tee /dev/stderr + fi + + done +} diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib new file mode 100755 index 00000000000..84224ab4a4d --- /dev/null +++ b/tests/queries/0_stateless/replication.lib @@ -0,0 +1,35 @@ +#!/usr/bin/env bash +# shellcheck source=./mergetree_mutations.lib +. "$CURDIR"/mergetree_mutations.lib + +function try_sync_replicas +{ + readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} --query="SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' AND engine like '%Replicated%'") + for t in "${tables_arr[@]}" + do + # The size of log may be big, so increase timeout. + $CLICKHOUSE_CLIENT --receive_timeout 300 -q "SYSTEM SYNC REPLICA $t" & + done + wait + echo "Replication did not hang: synced all replicas of $1" +} + +function check_replication_consistency() +{ + try_sync_replicas "$1" + + # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet + # TODO maybe just kill all mutations? + wait_for_all_mutations "$1%" + + $CLICKHOUSE_CLIENT -q \ + "SELECT + throwIf((countDistinct(data) AS c) != 1, 'Replicas have diverged'), c + FROM + ( + SELECT _table, ($2) AS data + FROM merge(currentDatabase(), '$1') GROUP BY _table + )" || $CLICKHOUSE_CLIENT -q \ + "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" | tee /dev/stderr +} + From 35f1caddcb80b8a1bcadb48dbc93670367e23510 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 11 Aug 2021 18:24:47 +0300 Subject: [PATCH 019/127] kill mutations before syncing --- ...nt_ttl_and_normal_merges_zookeeper_long.sh | 4 +++- tests/queries/0_stateless/replication.lib | 20 ++++++++++++++----- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index 3daab1e9fdd..6a0fa192321 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -60,7 +60,9 @@ timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & wait - +for i in $(seq 1 $NUM_REPLICAS); do + $CLICKHOUSE_CLIENT --query "SYSTEM STOP TTL MERGES ttl_table$i" & +done check_replication_consistency "ttl_table" "count(), sum(toUInt64(key))" $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue where table like 'ttl_table%' and database = '${CLICKHOUSE_DATABASE}' and type='MERGE_PARTS' and last_exception != '' FORMAT Vertical" diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 84224ab4a4d..7dbb988ec61 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -2,7 +2,7 @@ # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -function try_sync_replicas +function try_sync_replicas() { readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} --query="SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' AND engine like '%Replicated%'") for t in "${tables_arr[@]}" @@ -16,12 +16,14 @@ function try_sync_replicas function check_replication_consistency() { - try_sync_replicas "$1" + # Forcefully cancel mutations to avoid waiting for them to finish + ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database=currentDatabase() AND table like '$1%'" > /dev/null # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet - # TODO maybe just kill all mutations? wait_for_all_mutations "$1%" + try_sync_replicas "$1" + $CLICKHOUSE_CLIENT -q \ "SELECT throwIf((countDistinct(data) AS c) != 1, 'Replicas have diverged'), c @@ -29,7 +31,15 @@ function check_replication_consistency() ( SELECT _table, ($2) AS data FROM merge(currentDatabase(), '$1') GROUP BY _table - )" || $CLICKHOUSE_CLIENT -q \ - "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" | tee /dev/stderr + )" + res=$? + if ! [ $res -eq 0 ]; then + echo "Replicas have diverged" | tee /dev/stderr + $CLICKHOUSE_CLIENT -q "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" | tee /dev/stderr + $CLICKHOUSE_CLIENT -q "select * from system.replication_queue where database=currentDatabase() and table like '$1%'" | tee /dev/stderr + $CLICKHOUSE_CLIENT -q "select * from system.mutations where database=currentDatabase() and table like '$1%'" | tee /dev/stderr + $CLICKHOUSE_CLIENT -q "select * from system.parts where database=currentDatabase() and table like '$1%'" | tee /dev/stderr + fi + } From a2b2e8cb3fbe1767763b5c7cd60a2ae701f66815 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 11 Aug 2021 21:02:35 +0300 Subject: [PATCH 020/127] fix --- tests/queries/0_stateless/mergetree_mutations.lib | 2 +- tests/queries/0_stateless/replication.lib | 10 +++++----- tests/queries/skip_list.json | 1 + 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/mergetree_mutations.lib b/tests/queries/0_stateless/mergetree_mutations.lib index ffd8dce78fd..bcaee740c1f 100644 --- a/tests/queries/0_stateless/mergetree_mutations.lib +++ b/tests/queries/0_stateless/mergetree_mutations.lib @@ -35,7 +35,7 @@ function wait_for_all_mutations() fi if [[ $i -eq 200 ]]; then - echo "Timed out while waiting for mutation to execute!" | tee /dev/stderr + echo "Timed out while waiting for mutation to execute!" | tee >(cat >&2) fi done diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 7dbb988ec61..d3d93070663 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -34,11 +34,11 @@ function check_replication_consistency() )" res=$? if ! [ $res -eq 0 ]; then - echo "Replicas have diverged" | tee /dev/stderr - $CLICKHOUSE_CLIENT -q "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" | tee /dev/stderr - $CLICKHOUSE_CLIENT -q "select * from system.replication_queue where database=currentDatabase() and table like '$1%'" | tee /dev/stderr - $CLICKHOUSE_CLIENT -q "select * from system.mutations where database=currentDatabase() and table like '$1%'" | tee /dev/stderr - $CLICKHOUSE_CLIENT -q "select * from system.parts where database=currentDatabase() and table like '$1%'" | tee /dev/stderr + echo "Replicas have diverged" | tee >(cat >&2) + $CLICKHOUSE_CLIENT -q "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" | tee >(cat >&2) + $CLICKHOUSE_CLIENT -q "select * from system.replication_queue where database=currentDatabase() and table like '$1%'" | tee >(cat >&2) + $CLICKHOUSE_CLIENT -q "select * from system.mutations where database=currentDatabase() and table like '$1%'" | tee >(cat >&2) + $CLICKHOUSE_CLIENT -q "select * from system.parts where database=currentDatabase() and table like '$1%'" | tee >(cat >&2) fi } diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 5078dc9a256..84ec61d8281 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -321,6 +321,7 @@ "01076_parallel_alter_replicated_zookeeper", "01079_parallel_alter_add_drop_column_zookeeper", "01079_parallel_alter_detach_table_zookeeper", + "01079_parallel_alter_modify_zookeeper_long", "01080_check_for_error_incorrect_size_of_nested_column", "01083_expressions_in_engine_arguments", "01084_regexp_empty", From 0256e313b3ab4cd248086bd6dd17bdce2dd70d52 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Aug 2021 00:29:37 +0300 Subject: [PATCH 021/127] fix --- tests/clickhouse-test | 1 + tests/queries/0_stateless/mergetree_mutations.lib | 2 +- tests/queries/0_stateless/replication.lib | 12 ++++++------ 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f6833cfbd09..8ed9ac7c302 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -641,6 +641,7 @@ def run_tests_array(all_tests_with_params): status += print_test_time(total_time) status += " - having stderror:\n{}\n".format( '\n'.join(stderr.split('\n')[:100])) + status += "\nstdout:\n{}\n".format(stdout) status += 'Database: ' + testcase_args.testcase_database elif 'Exception' in stdout: failures += 1 diff --git a/tests/queries/0_stateless/mergetree_mutations.lib b/tests/queries/0_stateless/mergetree_mutations.lib index bcaee740c1f..7d02f9f1b41 100644 --- a/tests/queries/0_stateless/mergetree_mutations.lib +++ b/tests/queries/0_stateless/mergetree_mutations.lib @@ -35,7 +35,7 @@ function wait_for_all_mutations() fi if [[ $i -eq 200 ]]; then - echo "Timed out while waiting for mutation to execute!" | tee >(cat >&2) + echo "Timed out while waiting for mutation to execute!" fi done diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index d3d93070663..d67dd3721e6 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -33,12 +33,12 @@ function check_replication_consistency() FROM merge(currentDatabase(), '$1') GROUP BY _table )" res=$? - if ! [ $res -eq 0 ]; then - echo "Replicas have diverged" | tee >(cat >&2) - $CLICKHOUSE_CLIENT -q "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" | tee >(cat >&2) - $CLICKHOUSE_CLIENT -q "select * from system.replication_queue where database=currentDatabase() and table like '$1%'" | tee >(cat >&2) - $CLICKHOUSE_CLIENT -q "select * from system.mutations where database=currentDatabase() and table like '$1%'" | tee >(cat >&2) - $CLICKHOUSE_CLIENT -q "select * from system.parts where database=currentDatabase() and table like '$1%'" | tee >(cat >&2) + if [ $res -ne 0 ]; then + echo "Replicas have diverged" + $CLICKHOUSE_CLIENT -q "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" + $CLICKHOUSE_CLIENT -q "select * from system.replication_queue where database=currentDatabase() and table like '$1%'" + $CLICKHOUSE_CLIENT -q "select * from system.mutations where database=currentDatabase() and table like '$1%'" + $CLICKHOUSE_CLIENT -q "select * from system.parts where database=currentDatabase() and table like '$1%'" fi } From 9dd742cc29cc1743c55b107f37f677cc164e187f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Aug 2021 13:39:06 +0300 Subject: [PATCH 022/127] more debug info --- tests/clickhouse-test | 1 + ...ts_race_condition_zookeeper_long.reference | 2 +- ...ts_race_condition_drop_zookeeper.reference | 2 +- ..._alter_add_drop_column_zookeeper.reference | 2 +- ...llel_alter_modify_zookeeper_long.reference | 2 +- .../01154_move_partition_long.reference | 2 +- ...utations_kill_many_replicas_long.reference | 2 +- ...and_normal_merges_zookeeper_long.reference | 2 +- tests/queries/0_stateless/replication.lib | 25 +++++++++++-------- 9 files changed, 22 insertions(+), 18 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 8ed9ac7c302..dcb64a819e4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -632,6 +632,7 @@ def run_tests_array(all_tests_with_params): open(stdout_file).read().split('\n')[:100]) status += '\n' + status += "\nstdout:\n{}\n".format(stdout) status += 'Database: ' + testcase_args.testcase_database elif stderr: diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference index 8a6b9c4f877..c3165c3d6ef 100644 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.reference @@ -1,2 +1,2 @@ Replication did not hang: synced all replicas of alter_table -0 1 +Consistency: 1 diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference index 0d13bb62797..6e705f05f04 100644 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.reference @@ -1,2 +1,2 @@ Replication did not hang: synced all replicas of alter_table_ -0 1 +Consistency: 1 diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference index 34a89ec4d07..4b640354c1b 100644 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.reference @@ -2,7 +2,7 @@ Starting alters Finishing alters Equal number of columns Replication did not hang: synced all replicas of concurrent_alter_add_drop_ -0 1 +Consistency: 1 0 0 0 diff --git a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference index 15223e4fd99..435b1b1f1ae 100644 --- a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference +++ b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.reference @@ -6,7 +6,7 @@ Starting alters Finishing alters Replication did not hang: synced all replicas of concurrent_alter_mt_ -0 1 +Consistency: 1 1 0 1 diff --git a/tests/queries/0_stateless/01154_move_partition_long.reference b/tests/queries/0_stateless/01154_move_partition_long.reference index 40aaa81456a..37f0181524e 100644 --- a/tests/queries/0_stateless/01154_move_partition_long.reference +++ b/tests/queries/0_stateless/01154_move_partition_long.reference @@ -1,3 +1,3 @@ Replication did not hang: synced all replicas of dst_ -0 1 +Consistency: 1 Replication did not hang: synced all replicas of src_ diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference index 5a3c0201732..c68053e8270 100644 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference @@ -15,4 +15,4 @@ Metadata version on replica 5 equal with first replica, OK CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 499999500000 Replication did not hang: synced all replicas of concurrent_kill_ -0 1 +Consistency: 1 diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference index 25e14257d8d..e5a8ecd20b4 100644 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.reference @@ -1,3 +1,3 @@ Replication did not hang: synced all replicas of ttl_table -0 1 +Consistency: 1 1 diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index d67dd3721e6..af5375fb235 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -8,7 +8,8 @@ function try_sync_replicas() for t in "${tables_arr[@]}" do # The size of log may be big, so increase timeout. - $CLICKHOUSE_CLIENT --receive_timeout 300 -q "SYSTEM SYNC REPLICA $t" & + $CLICKHOUSE_CLIENT --receive_timeout 400 -q "SYSTEM SYNC REPLICA $t" || $CLICKHOUSE_CLIENT -q \ + "select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t'" & done wait echo "Replication did not hang: synced all replicas of $1" @@ -24,21 +25,23 @@ function check_replication_consistency() try_sync_replicas "$1" - $CLICKHOUSE_CLIENT -q \ + res=$($CLICKHOUSE_CLIENT -q \ "SELECT - throwIf((countDistinct(data) AS c) != 1, 'Replicas have diverged'), c + countDistinct(data) FROM ( SELECT _table, ($2) AS data FROM merge(currentDatabase(), '$1') GROUP BY _table - )" - res=$? - if [ $res -ne 0 ]; then - echo "Replicas have diverged" - $CLICKHOUSE_CLIENT -q "select _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" - $CLICKHOUSE_CLIENT -q "select * from system.replication_queue where database=currentDatabase() and table like '$1%'" - $CLICKHOUSE_CLIENT -q "select * from system.mutations where database=currentDatabase() and table like '$1%'" - $CLICKHOUSE_CLIENT -q "select * from system.parts where database=currentDatabase() and table like '$1%'" + )") + + echo "Consistency: $res" + if [ $res -ne 1 ]; then + echo "Replicas have diverged:" + $CLICKHOUSE_CLIENT -q "select 'data', _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" + $CLICKHOUSE_CLIENT -q "select 'queue', * from system.replication_queue where database=currentDatabase() and table like '$1%'" + $CLICKHOUSE_CLIENT -q "select 'mutations', * from system.mutations where database=currentDatabase() and table like '$1%'" + $CLICKHOUSE_CLIENT -q "select 'parts', * from system.parts where database=currentDatabase() and table like '$1%'" + echo "Good luck with debugging..." fi } From 95eeeb53d640b0bbe15bc0fda68ddaa8c6c79b61 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Aug 2021 18:58:49 +0300 Subject: [PATCH 023/127] fix --- .../ReplicatedMergeTreePartCheckThread.cpp | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 3 ++ src/Storages/StorageReplicatedMergeTree.cpp | 38 ++++++++++--------- tests/queries/0_stateless/replication.lib | 2 +- 4 files changed, 26 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 35a011a4a58..797d0570fbc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -145,6 +145,7 @@ ReplicatedMergeTreePartCheckThread::MissingPartSearchResult ReplicatedMergeTreeP if (found_part_with_the_same_min_block && found_part_with_the_same_max_block) { + /// FIXME It may never appear LOG_WARNING(log, "Found parts with the same min block and with the same max block as the missing part {}. Hoping that it will eventually appear as a result of a merge.", part_name); return MissingPartSearchResult::FoundAndDontNeedFetch; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index ea5f7cfc36a..277d887a46e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1488,6 +1488,9 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( /// to allow recovering from a mutation that cannot be executed. This way you can delete the mutation entry /// from /mutations in ZK and the replicas will simply skip the mutation. + /// NOTE: However, it's quite dangerous to skip MUTATE_PART. Replicas may diverge if one of them have executed part mutation, + /// and then mutation was killed before execution of MUTATE_PART on remaining replicas. + if (part->info.getDataVersion() > desired_mutation_version) { LOG_WARNING(log, "Data version of part {} is already greater than desired mutation version {}", part->name, desired_mutation_version); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 150a71a09e5..194d81ba553 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -141,6 +141,7 @@ namespace ErrorCodes extern const int DUPLICATE_DATA_PART; extern const int BAD_ARGUMENTS; extern const int CONCURRENT_ACCESS_NOT_SUPPORTED; + extern const int CHECKSUM_DOESNT_MATCH; } namespace ActionLocks @@ -1314,32 +1315,35 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: } ReplicatedMergeTreePartHeader replica_part_header; - if (!part_zk_str.empty()) - replica_part_header = ReplicatedMergeTreePartHeader::fromString(part_zk_str); - else + if (part_zk_str.empty()) { - Coordination::Stat columns_stat_before, columns_stat_after; String columns_str; String checksums_str; - /// Let's check that the node's version with the columns did not change while we were reading the checksums. - /// This ensures that the columns and the checksum refer to the same - if (!zookeeper->tryGet(fs::path(current_part_path) / "columns", columns_str, &columns_stat_before) || - !zookeeper->tryGet(fs::path(current_part_path) / "checksums", checksums_str) || - !zookeeper->exists(fs::path(current_part_path) / "columns", &columns_stat_after) || - columns_stat_before.version != columns_stat_after.version) + if (zookeeper->tryGet(fs::path(current_part_path) / "columns", columns_str) && + zookeeper->tryGet(fs::path(current_part_path) / "checksums", checksums_str)) { - LOG_INFO(log, "Not checking checksums of part {} with replica {} because part changed while we were reading its checksums", part_name, replica); + replica_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes(columns_str, checksums_str); + } + else + { + if (zookeeper->exists(current_part_path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} has empty header and does not have columns and checksums. " + "Looks like a bug.", current_part_path); + LOG_INFO(log, "Not checking checksums of part {} with replica {} because part was removed from ZooKeeper", part_name, replica); continue; } - - replica_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes( - columns_str, checksums_str); + } + else + { + replica_part_header = ReplicatedMergeTreePartHeader::fromString(part_zk_str); } if (replica_part_header.getColumnsHash() != local_part_header.getColumnsHash()) { - LOG_INFO(log, "Not checking checksums of part {} with replica {} because columns are different", part_name, replica); - continue; + /// Either it's a bug or ZooKeeper contains broken data. + /// TODO Fix KILL MUTATION and replace CHECKSUM_DOESNT_MATCH with LOGICAL_ERROR + /// (some replicas may skip killed mutation even if it was executed on other replicas) + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Part {} from {} has different columns hash", part_name, replica); } replica_part_header.getChecksums().checkEqual(local_part_header.getChecksums(), true); @@ -6058,7 +6062,7 @@ CancellationCode StorageReplicatedMergeTree::killMutation(const String & mutatio zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); - LOG_TRACE(log, "Killing mutation {}", mutation_id); + LOG_INFO(log, "Killing mutation {}", mutation_id); auto mutation_entry = queue.removeMutation(zookeeper, mutation_id); if (!mutation_entry) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index af5375fb235..2992094b5d3 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -27,7 +27,7 @@ function check_replication_consistency() res=$($CLICKHOUSE_CLIENT -q \ "SELECT - countDistinct(data) + if((countDistinct(data) as c) == 0, 1, c) FROM ( SELECT _table, ($2) AS data From 285a5848b51a58ac71fae56d515af6cf712ab253 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 13 Aug 2021 14:27:55 +0300 Subject: [PATCH 024/127] fix --- tests/config/config.d/merge_tree.xml | 5 +++++ tests/config/install.sh | 1 + .../01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh | 3 ++- 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/config/config.d/merge_tree.xml diff --git a/tests/config/config.d/merge_tree.xml b/tests/config/config.d/merge_tree.xml new file mode 100644 index 00000000000..35af1fa65eb --- /dev/null +++ b/tests/config/config.d/merge_tree.xml @@ -0,0 +1,5 @@ + + + 8 + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 571dff34018..e46ac62606b 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -31,6 +31,7 @@ ln -sf $SRC_PATH/config.d/max_concurrent_queries.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/keeper_port.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/merge_tree.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index 6a0fa192321..80022bd472d 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -61,7 +61,8 @@ timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT --query "SYSTEM STOP TTL MERGES ttl_table$i" & + # disable ttl merges before checking consistency + $CLICKHOUSE_CLIENT --query "ALTER TABLE ttl_table$i MODIFY SETTING max_replicated_merges_with_ttl_in_queue=0" done check_replication_consistency "ttl_table" "count(), sum(toUInt64(key))" From e824d96c3b3d9c579f2f280bdd80c482cf81264b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 16 Aug 2021 00:42:10 +0300 Subject: [PATCH 025/127] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 194d81ba553..b94908bca0a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2141,6 +2141,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) if (!parts_for_merge.empty() && replica.empty()) { LOG_INFO(log, "No active replica has part {}. Will fetch merged part instead.", entry.new_part_name); + /// We should enqueue it for check, because merged part may never appear if source part is lost + enqueuePartForCheck(entry.new_part_name); return false; } From de2341fe02418f527be2f678e026a09a19ddc17a Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 16 Aug 2021 01:57:09 +0000 Subject: [PATCH 026/127] Initial --- .../mergetree-family/mergetree.md | 22 +++++++++------ docs/en/operations/settings/settings.md | 22 +++++++++++++++ .../statements/alter/projection.md | 2 +- .../mergetree-family/mergetree.md | 28 +++++++++++-------- docs/ru/operations/settings/settings.md | 22 +++++++++++++++ .../statements/alter/projection.md | 2 +- 6 files changed, 75 insertions(+), 23 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 1b1313e625c..2c624fb8afd 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -388,20 +388,24 @@ Functions with a constant argument that is less than ngram size can’t be used - `s != 1` - `NOT startsWith(s, 'test')` -### Projections {#projections} -Projections are like materialized views but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +## Projections {#projections} +Projections are like [materialized views](../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. -#### Query {#projection-query} -A projection query is what defines a projection. It has the following grammar: +Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. -`SELECT [GROUP BY] [ORDER BY]` +### Projection Query {#projection-query} +A projection query is what defines a projection. It implicitly selects data from the parent table. It has the following syntax: -It implicitly selects data from the parent table. +```sql +SELECT [GROUP BY] [ORDER BY] +``` -#### Storage {#projection-storage} -Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous MergeTree table's part. The table is induced by the definition query of the projection. If there is a GROUP BY clause, the underlying storage engine becomes AggregatedMergeTree, and all aggregate functions are converted to AggregateFunction. If there is an ORDER BY clause, the MergeTree table will use it as its primary key expression. During the merge process, the projection part will be merged via its storage's merge routine. The checksum of the parent table's part will combine the projection's part. Other maintenance jobs are similar to skip indices. +Projections can be modified or dropped with the [ALTER](../../../sql-reference/statement/alter/projection.md) statement. -#### Query Analysis {#projection-query-analysis} +### Projection Storage {#projection-storage} +Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. + +### Query Analysis {#projection-query-analysis} 1. Check if the projection can be used to answer the given query, that is, it generates the same answer as querying the base table. 2. Select the best feasible match, which contains the least granules to read. 3. The query pipeline which uses projections will be different from the one that uses the original parts. If the projection is absent in some parts, we can add the pipeline to "project" it on the fly. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 07bfe158a0a..beb52330b13 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3420,3 +3420,25 @@ Possible values: - 1 — The table is automatically updated in the background, when schema changes are detected. Default value: `0`. + +## allow_experimental_projection_optimization {#allow-experimental-projection-optimization} + +Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md#projections) optimization when processing `SELECT` queries. + +Possible values: + +- 0 — Projection optimization disabled. +- 1 — Projection optimization enabled. + +Default value: `0`. + +## force_optimize_projection {#force-optimize-projection} + +Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow_experimental_projection_optimization) setting). + +Possible values: + +- 0 — Projection optimization is not obligatory. +- 1 — Projection optimization is obligatory. + +Default value: `0`. diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 07a13fc23c4..9d0c798a209 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -15,7 +15,7 @@ The following operations are available: - `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. -The commands ADD, DROP and CLEAR are lightweight in a sense that they only change metadata or remove files. +The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing projections metadata via ZooKeeper. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 61ed34b686c..381ba305f65 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -375,23 +375,27 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `s != 1` - `NOT startsWith(s, 'test')` -### Проекции {#projections} -Проекции похожи на материализованные представления, но определяются на уровне партов. Это обеспечивает гарантии согласованности наряду с автоматическим использованием в запросах. +## Проекции {#projections} +Проекции похожи на [материализованные представления](../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. -#### Запрос {#projection-query} -Запрос проекции — это то, что определяет проекцию. Он имеет следующую грамматику: +Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). -`SELECT [GROUP BY] [ORDER BY]` +### Запрос проекции {#projection-query} +Запрос проекции — это то, что определяет проекцию. Такой запрос неявно выбирает данные из родительской таблицы. Он имеет следующий синтаксис: -Он неявно выбирает данные из родительской таблицы. +```sql +SELECT [GROUP BY] [ORDER BY] +``` -#### Хранение {#projection-storage} -Проекции хранятся в каталоге парта. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный парт таблицы MergeTree. Таблица создается запросом определения проекции. Если есть конструкция GROUP BY, то базовый механизм хранения становится AggregatedMergeTree, а все агрегатные функции преобразуются в AggregateFunction. Если есть конструкция ORDER BY, таблица MergeTree будет использовать его в качестве выражения первичного ключа. Во время процесса слияния парт проекции будет слит с помощью процедуры слияния ее хранилища. Контрольная сумма парта родительской таблицы будет включать парт проекции. Другие процедуры аналогичны индексам пропуска данных. +Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statement/alter/projection.md). -#### Анализ запросов {#projection-query-analysis} -1. Проверить, можно ли использовать проекцию в данном запросе, то есть, что с ней выходит тот же результат, что и с запросом к базовой таблице. -2. Выбрать наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. -3. План запроса, который использует проекции, будет отличаться от того, который использует исходные парты. При отсутствии проекции в некоторых партах можно расширить план, чтобы «проецировать» на лету. +### Хранение проекции {#projection-storage} +Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. Если есть секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. Если есть секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния ее хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. + +### Анализ запросов {#projection-query-analysis} +1. Проверьте, можно ли использовать проекцию в данном запросе, то есть, что с ней выходит тот же результат, что и с запросом к базовой таблице. +2. Выберите наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. +3. План запроса, который использует проекции, отличается от того, который использует исходные куски данных. При отсутствии проекции в некоторых кусках можно расширить план, чтобы «проецировать» на лету. ## Конкурентный доступ к данным {#concurrent-data-access} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 209c2e2001d..7e38212f415 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3237,3 +3237,25 @@ SETTINGS index_granularity = 8192 │ - 1 — таблица обновляется автоматически в фоновом режиме при обнаружении изменений схемы. Значение по умолчанию: `0`. + +## allow_experimental_projection_optimization {#allow-experimental-projection-optimization} + +Включает или отключает поддержку [проекций](../../engines/table-engines/mergetree-family/mergetree.md#projections) при обработке запросов `SELECT`. + +Возможные значения: + +- 0 — Проекции не поддерживаются. +- 1 — Проекции поддерживаются. + +Значение по умолчанию: `0`. + +## force_optimize_projection {#force-optimize-projection} + +Включает или отключает обязательное использование [проекций](../../engines/table-engines/mergetree-family/mergetree.md#projections) в запросах `SELECT`, если поддержка проекций включена (см. настройку [allow_experimental_projection_optimization](#allow_experimental_projection_optimization)). + +Возможные значения: + +- 0 — Проекции обязательно используются. +- 1 — Проекции используются опционально. + +Значение по умолчанию: `0`. \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index db116963aa6..30bc6e0bbee 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -15,7 +15,7 @@ toc_title: PROJECTION - `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` — удаляет файлы проекции с диска без удаления описания. -Комманды ADD, DROP и CLEAR — легковесны, поскольку они только меняют метаданные или удаляют файлы. +Комманды `ADD`, `DROP` и `CLEAR` — легковесны, поскольку они только меняют метаданные или удаляют файлы. Также команды реплицируются, синхронизируя описания проекций в метаданных с помощью ZooKeeper. From 1da536f8e02eec2eb691686fb74289ed6c9ce2da Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 16 Aug 2021 01:58:10 +0000 Subject: [PATCH 027/127] Next --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 ++ docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 2c624fb8afd..72ab6920daf 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -393,6 +393,8 @@ Projections are like [materialized views](../../sql-reference/statements/create/ Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. +Projections are not supported in the `SELECT` statements with [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. + ### Projection Query {#projection-query} A projection query is what defines a projection. It implicitly selects data from the parent table. It has the following syntax: diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 381ba305f65..06a45048df2 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -380,6 +380,8 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). +Проекции не поддерживаются для запросов `SELECT` с модификатором [FINAL](../../../sql-reference/statements/select/from.md#select-from-final). + ### Запрос проекции {#projection-query} Запрос проекции — это то, что определяет проекцию. Такой запрос неявно выбирает данные из родительской таблицы. Он имеет следующий синтаксис: From 52cfbe2812dc1d27783ba6c3e277791e43fd9982 Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 16 Aug 2021 02:03:37 +0000 Subject: [PATCH 028/127] Links fixed --- clickhouse-fork | 1 + docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) create mode 160000 clickhouse-fork diff --git a/clickhouse-fork b/clickhouse-fork new file mode 160000 index 00000000000..157bca84f41 --- /dev/null +++ b/clickhouse-fork @@ -0,0 +1 @@ +Subproject commit 157bca84f412a0cf25497908ed19bf5a66f0aaec diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 72ab6920daf..d4647473c7e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ Functions with a constant argument that is less than ngram size can’t be used - `NOT startsWith(s, 'test')` ## Projections {#projections} -Projections are like [materialized views](../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. @@ -402,7 +402,7 @@ A projection query is what defines a projection. It implicitly selects data from SELECT [GROUP BY] [ORDER BY] ``` -Projections can be modified or dropped with the [ALTER](../../../sql-reference/statement/alter/projection.md) statement. +Projections can be modified or dropped with the [ALTER](../../../sql-reference/statements/alter/projection.md) statement. ### Projection Storage {#projection-storage} Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 06a45048df2..aa28d7b3e3d 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -376,7 +376,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `NOT startsWith(s, 'test')` ## Проекции {#projections} -Проекции похожи на [материализованные представления](../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. +Проекции похожи на [материализованные представления](../../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). @@ -389,7 +389,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT SELECT [GROUP BY] [ORDER BY] ``` -Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statement/alter/projection.md). +Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statements/alter/projection.md). ### Хранение проекции {#projection-storage} Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. Если есть секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. Если есть секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния ее хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. From 3dc8e4931e931adf2e83afd6f119f95b9bb09480 Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 16 Aug 2021 02:14:36 +0000 Subject: [PATCH 029/127] Links added --- docs/en/sql-reference/statements/alter/projection.md | 2 +- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 9d0c798a209..429241ebf13 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -5,7 +5,7 @@ toc_title: PROJECTION # Manipulating Projections {#manipulations-with-projections} -The following operations are available: +The following operations with [projections](../../../engines/table-engines/mergetree-family/mergetree.md#projections) are available: - `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index 30bc6e0bbee..967c36e21d3 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -5,7 +5,7 @@ toc_title: PROJECTION # Манипуляции с проекциями {#manipulations-with-projections} -Доступны следующие операции: +Доступны следующие операции с [проекциями](../../../engines/table-engines/mergetree-family/mergetree.md#projections): - `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` — добавляет описание проекции в метаданные. From b2336346cd9928bd3ac8a7e379ffde64d6afd474 Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 16 Aug 2021 02:53:21 +0000 Subject: [PATCH 030/127] Revert "Links fixed" This reverts commit 52cfbe2812dc1d27783ba6c3e277791e43fd9982. --- clickhouse-fork | 1 - docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 3 files changed, 4 insertions(+), 5 deletions(-) delete mode 160000 clickhouse-fork diff --git a/clickhouse-fork b/clickhouse-fork deleted file mode 160000 index 157bca84f41..00000000000 --- a/clickhouse-fork +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 157bca84f412a0cf25497908ed19bf5a66f0aaec diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d4647473c7e..72ab6920daf 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ Functions with a constant argument that is less than ngram size can’t be used - `NOT startsWith(s, 'test')` ## Projections {#projections} -Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like [materialized views](../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. @@ -402,7 +402,7 @@ A projection query is what defines a projection. It implicitly selects data from SELECT [GROUP BY] [ORDER BY] ``` -Projections can be modified or dropped with the [ALTER](../../../sql-reference/statements/alter/projection.md) statement. +Projections can be modified or dropped with the [ALTER](../../../sql-reference/statement/alter/projection.md) statement. ### Projection Storage {#projection-storage} Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index aa28d7b3e3d..06a45048df2 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -376,7 +376,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `NOT startsWith(s, 'test')` ## Проекции {#projections} -Проекции похожи на [материализованные представления](../../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. +Проекции похожи на [материализованные представления](../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). @@ -389,7 +389,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT SELECT [GROUP BY] [ORDER BY] ``` -Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statements/alter/projection.md). +Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statement/alter/projection.md). ### Хранение проекции {#projection-storage} Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. Если есть секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. Если есть секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния ее хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. From fd8042de3ee790041036fa43fe8ab8cb1d828570 Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 16 Aug 2021 02:57:00 +0000 Subject: [PATCH 031/127] Links fixed again --- clickhouse-fork | 1 + docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) create mode 160000 clickhouse-fork diff --git a/clickhouse-fork b/clickhouse-fork new file mode 160000 index 00000000000..157bca84f41 --- /dev/null +++ b/clickhouse-fork @@ -0,0 +1 @@ +Subproject commit 157bca84f412a0cf25497908ed19bf5a66f0aaec diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 72ab6920daf..d4647473c7e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ Functions with a constant argument that is less than ngram size can’t be used - `NOT startsWith(s, 'test')` ## Projections {#projections} -Projections are like [materialized views](../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. @@ -402,7 +402,7 @@ A projection query is what defines a projection. It implicitly selects data from SELECT [GROUP BY] [ORDER BY] ``` -Projections can be modified or dropped with the [ALTER](../../../sql-reference/statement/alter/projection.md) statement. +Projections can be modified or dropped with the [ALTER](../../../sql-reference/statements/alter/projection.md) statement. ### Projection Storage {#projection-storage} Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 06a45048df2..aa28d7b3e3d 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -376,7 +376,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `NOT startsWith(s, 'test')` ## Проекции {#projections} -Проекции похожи на [материализованные представления](../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. +Проекции похожи на [материализованные представления](../../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). @@ -389,7 +389,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT SELECT [GROUP BY] [ORDER BY] ``` -Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statement/alter/projection.md). +Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statements/alter/projection.md). ### Хранение проекции {#projection-storage} Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. Если есть секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. Если есть секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния ее хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. From 2428c6c78c603a48f33b604b9dd40a07d0baa4db Mon Sep 17 00:00:00 2001 From: olgarev Date: Mon, 16 Aug 2021 02:58:00 +0000 Subject: [PATCH 032/127] Revert "Links fixed again" This reverts commit fd8042de3ee790041036fa43fe8ab8cb1d828570. --- clickhouse-fork | 1 - docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 3 files changed, 4 insertions(+), 5 deletions(-) delete mode 160000 clickhouse-fork diff --git a/clickhouse-fork b/clickhouse-fork deleted file mode 160000 index 157bca84f41..00000000000 --- a/clickhouse-fork +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 157bca84f412a0cf25497908ed19bf5a66f0aaec diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d4647473c7e..72ab6920daf 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ Functions with a constant argument that is less than ngram size can’t be used - `NOT startsWith(s, 'test')` ## Projections {#projections} -Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like [materialized views](../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. @@ -402,7 +402,7 @@ A projection query is what defines a projection. It implicitly selects data from SELECT [GROUP BY] [ORDER BY] ``` -Projections can be modified or dropped with the [ALTER](../../../sql-reference/statements/alter/projection.md) statement. +Projections can be modified or dropped with the [ALTER](../../../sql-reference/statement/alter/projection.md) statement. ### Projection Storage {#projection-storage} Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index aa28d7b3e3d..06a45048df2 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -376,7 +376,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `NOT startsWith(s, 'test')` ## Проекции {#projections} -Проекции похожи на [материализованные представления](../../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. +Проекции похожи на [материализованные представления](../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). @@ -389,7 +389,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT SELECT [GROUP BY] [ORDER BY] ``` -Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statements/alter/projection.md). +Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statement/alter/projection.md). ### Хранение проекции {#projection-storage} Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. Если есть секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. Если есть секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния ее хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. From 68b31222725ffa8cee911e29047a777488999901 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 16 Aug 2021 06:03:05 +0300 Subject: [PATCH 033/127] Apply suggestions from code review --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 72ab6920daf..d4647473c7e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -389,7 +389,7 @@ Functions with a constant argument that is less than ngram size can’t be used - `NOT startsWith(s, 'test')` ## Projections {#projections} -Projections are like [materialized views](../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. @@ -402,7 +402,7 @@ A projection query is what defines a projection. It implicitly selects data from SELECT [GROUP BY] [ORDER BY] ``` -Projections can be modified or dropped with the [ALTER](../../../sql-reference/statement/alter/projection.md) statement. +Projections can be modified or dropped with the [ALTER](../../../sql-reference/statements/alter/projection.md) statement. ### Projection Storage {#projection-storage} Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 06a45048df2..aa28d7b3e3d 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -376,7 +376,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT - `NOT startsWith(s, 'test')` ## Проекции {#projections} -Проекции похожи на [материализованные представления](../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. +Проекции похожи на [материализованные представления](../../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). @@ -389,7 +389,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT SELECT [GROUP BY] [ORDER BY] ``` -Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statement/alter/projection.md). +Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statements/alter/projection.md). ### Хранение проекции {#projection-storage} Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. Если есть секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. Если есть секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния ее хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. From ff9f8ea40d0bbe0782128812ba554f982df468d1 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 16 Aug 2021 06:52:48 +0300 Subject: [PATCH 034/127] Apply suggestions from code review --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index beb52330b13..6d6acc365b9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3434,7 +3434,7 @@ Default value: `0`. ## force_optimize_projection {#force-optimize-projection} -Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow_experimental_projection_optimization) setting). +Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow-experimental-projection-optimization) setting). Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 7e38212f415..b5917783a55 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3251,7 +3251,7 @@ SETTINGS index_granularity = 8192 │ ## force_optimize_projection {#force-optimize-projection} -Включает или отключает обязательное использование [проекций](../../engines/table-engines/mergetree-family/mergetree.md#projections) в запросах `SELECT`, если поддержка проекций включена (см. настройку [allow_experimental_projection_optimization](#allow_experimental_projection_optimization)). +Включает или отключает обязательное использование [проекций](../../engines/table-engines/mergetree-family/mergetree.md#projections) в запросах `SELECT`, если поддержка проекций включена (см. настройку [allow_experimental_projection_optimization](#allow-experimental-projection-optimization)). Возможные значения: From cc924c93c835c992ddde71c01363b4d1d392f22a Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 16 Aug 2021 07:04:18 +0300 Subject: [PATCH 035/127] Apply suggestions from code review Anchors in links fixed --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d4647473c7e..de02fb465fd 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -391,7 +391,7 @@ Functions with a constant argument that is less than ngram size can’t be used ## Projections {#projections} Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. -Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection) setting. +Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force-optimize-projection) setting. Projections are not supported in the `SELECT` statements with [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index aa28d7b3e3d..0f7519f2232 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -378,7 +378,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT ## Проекции {#projections} Проекции похожи на [материализованные представления](../../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. -Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force_optimize_projection). +Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force-optimize-projection). Проекции не поддерживаются для запросов `SELECT` с модификатором [FINAL](../../../sql-reference/statements/select/from.md#select-from-final). From d9bd4675376d2d66af83fa066c96d832d370db68 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 16 Aug 2021 12:18:52 +0300 Subject: [PATCH 036/127] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 4 +-- tests/queries/0_stateless/replication.lib | 32 ++++++++++++++++----- 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b94908bca0a..4f5d635e6ea 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5466,9 +5466,9 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( const auto & stop_waiting = [&]() { - bool stop_waiting_itself = waiting_itself && (partial_shutdown_called || is_dropped); + bool stop_waiting_itself = waiting_itself && partial_shutdown_called; bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active"); - return stop_waiting_itself || stop_waiting_non_active; + return is_dropped || stop_waiting_itself || stop_waiting_non_active; }; /// Don't recheck ZooKeeper too often diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 2992094b5d3..15af1dbd6c8 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -4,12 +4,30 @@ function try_sync_replicas() { - readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} --query="SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' AND engine like '%Replicated%'") + readarray -t empty_partitions_arr < <(${CLICKHOUSE_CLIENT} -q \ + "SELECT DISTINCT substr(new_part_name, 1, position(new_part_name, '_') - 1) AS partition_id + FROM system.replication_queue + WHERE (database = currentDatabase()) AND (table LIKE '$1%') AND (last_exception LIKE '%No active replica has part%') AND (partition_id NOT IN ( + SELECT partition_id + FROM system.parts + WHERE (database = currentDatabase()) AND (table LIKE '$1%') + ))") + readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' AND engine like '%Replicated%'") + + for t in "${tables_arr[@]}" + do + for p in "${empty_partitions_arr[@]}" + do + # Avoid "Empty part ... is not created instead of lost part because there are no parts in partition" + $CLICKHOUSE_CLIENT -q "ALTER TABLE $t DROP PARTITION ID '$p'" 2>/dev/null + done + done + for t in "${tables_arr[@]}" do # The size of log may be big, so increase timeout. $CLICKHOUSE_CLIENT --receive_timeout 400 -q "SYSTEM SYNC REPLICA $t" || $CLICKHOUSE_CLIENT -q \ - "select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t'" & + "select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t' order by database, table, node_name" & done wait echo "Replication did not hang: synced all replicas of $1" @@ -18,7 +36,7 @@ function try_sync_replicas() function check_replication_consistency() { # Forcefully cancel mutations to avoid waiting for them to finish - ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database=currentDatabase() AND table like '$1%'" > /dev/null + ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$1%'" > /dev/null # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet wait_for_all_mutations "$1%" @@ -37,10 +55,10 @@ function check_replication_consistency() echo "Consistency: $res" if [ $res -ne 1 ]; then echo "Replicas have diverged:" - $CLICKHOUSE_CLIENT -q "select 'data', _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table" - $CLICKHOUSE_CLIENT -q "select 'queue', * from system.replication_queue where database=currentDatabase() and table like '$1%'" - $CLICKHOUSE_CLIENT -q "select 'mutations', * from system.mutations where database=currentDatabase() and table like '$1%'" - $CLICKHOUSE_CLIENT -q "select 'parts', * from system.parts where database=currentDatabase() and table like '$1%'" + $CLICKHOUSE_CLIENT -q "select 'data', _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table order by _table" + $CLICKHOUSE_CLIENT -q "select 'queue', * from system.replication_queue where database=currentDatabase() and table like '$1%' order by database, table, node_name" + $CLICKHOUSE_CLIENT -q "select 'mutations', * from system.mutations where database=currentDatabase() and table like '$1%' order by database, table, mutation_id" + $CLICKHOUSE_CLIENT -q "select 'parts', * from system.parts where database=currentDatabase() and table like '$1%' order by database, table, name" echo "Good luck with debugging..." fi From b8d9bc862d6b102bb05d105bea558d7fb3b89509 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 16 Aug 2021 15:36:12 +0300 Subject: [PATCH 037/127] fix --- tests/queries/0_stateless/replication.lib | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 15af1dbd6c8..54c5f3c2faf 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -35,6 +35,10 @@ function try_sync_replicas() function check_replication_consistency() { + # Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them + some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' LIMIT 1") + $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 2>/dev/null + # Forcefully cancel mutations to avoid waiting for them to finish ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$1%'" > /dev/null From 3c8611a5220fbe23a907253a9e3005e9f527eb07 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 16 Aug 2021 15:51:04 +0300 Subject: [PATCH 038/127] fix --- tests/queries/0_stateless/replication.lib | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 54c5f3c2faf..77b09dee1e0 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -37,7 +37,7 @@ function check_replication_consistency() { # Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' LIMIT 1") - $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 2>/dev/null + $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: # Forcefully cancel mutations to avoid waiting for them to finish ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$1%'" > /dev/null From d50c5e3c32a0df3185e0965bd11421b95671a579 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 17 Aug 2021 15:01:51 +0300 Subject: [PATCH 039/127] fix sync replica --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 14 +++++++++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 11 ++++++++- .../ReplicatedMergeTreeRestartingThread.cpp | 23 ++++++++++++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 9 ++++---- 4 files changed, 46 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 3f3a7f19f72..3a9f477597d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int UNEXPECTED_NODE_IN_ZOOKEEPER; extern const int ABORTED; + extern const int READONLY; } @@ -472,9 +473,18 @@ bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo & return virtual_parts.remove(part_info); } -int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback) +int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback, PullLogsReason reason) { std::lock_guard lock(pull_logs_to_queue_mutex); + if (storage.is_readonly && reason != LOAD) + { + /// Pulling logs when replica is readonly may cause obscure bugs, allow it on replica startup only + if (reason == SYNC) + throw Exception(ErrorCodes::READONLY, "Cannot SYNC REPLICA, because replica is readonly"); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Some background task ({}) tried to pull logs on readonly replica, it's a bug", reason); + } + if (pull_log_blocker.isCancelled()) throw Exception("Log pulling is cancelled", ErrorCodes::ABORTED); @@ -1834,7 +1844,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( } } - merges_version = queue_.pullLogsToQueue(zookeeper); + merges_version = queue_.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::MERGE_PREDICATE); { /// We avoid returning here a version to be used in a lightweight transaction. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index e49d80fc832..57e1e658665 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -294,13 +294,22 @@ public: bool removeFailedQuorumPart(const MergeTreePartInfo & part_info); + enum PullLogsReason + { + LOAD, + UPDATE, + MERGE_PREDICATE, + SYNC, + OTHER, + }; + /** Copy the new entries from the shared log to the queue of this replica. Set the log_pointer to the appropriate value. * If watch_callback is not empty, will call it when new entries appear in the log. * If there were new entries, notifies storage.queue_task_handle. * Additionally loads mutations (so that the set of mutations is always more recent than the queue). * Return the version of "logs" node (that is updated for every merge/mutation/... added to the log) */ - int32_t pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {}); + int32_t pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {}, PullLogsReason reason = OTHER); /// Load new mutation entries. If something new is loaded, schedule storage.merge_selecting_task. /// If watch_callback is not empty, will call it when new mutations appear in ZK. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 25f25480549..a7bb56f1955 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -25,6 +25,8 @@ namespace DB namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; + extern const int REPLICA_STATUS_CHANGED; + } namespace @@ -55,6 +57,7 @@ void ReplicatedMergeTreeRestartingThread::run() if (need_stop) return; + bool reschedule_now = false; try { if (first_time || readonly_mode_was_set || storage.getZooKeeper()->expired()) @@ -131,15 +134,29 @@ void ReplicatedMergeTreeRestartingThread::run() first_time = false; } } - catch (...) + catch (const Exception & e) { /// We couldn't activate table let's set it into readonly mode setReadonly(); + partialShutdown(); + storage.startup_event.set(); + tryLogCurrentException(log, __PRETTY_FUNCTION__); + + if (e.code() == ErrorCodes::REPLICA_STATUS_CHANGED) + reschedule_now = true; + } + catch (...) + { + setReadonly(); + partialShutdown(); storage.startup_event.set(); tryLogCurrentException(log, __PRETTY_FUNCTION__); } - task->scheduleAfter(check_period_ms); + if (reschedule_now) + task->schedule(); + else + task->scheduleAfter(check_period_ms); } @@ -159,7 +176,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); /// because cleanup_thread doesn't delete log_pointer of active replicas. - storage.queue.pullLogsToQueue(zookeeper); + storage.queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::LOAD); storage.queue.removeCurrentPartsFromMutations(); storage.last_queue_update_finish_time.store(time(nullptr)); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4f5d635e6ea..bdec69095ce 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3089,7 +3089,7 @@ void StorageReplicatedMergeTree::queueUpdatingTask() } try { - queue.pullLogsToQueue(getZooKeeper(), queue_updating_task->getWatchCallback()); + queue.pullLogsToQueue(getZooKeeper(), queue_updating_task->getWatchCallback(), ReplicatedMergeTreeQueue::UPDATE); last_queue_update_finish_time.store(time(nullptr)); queue_update_in_progress = false; } @@ -4325,11 +4325,9 @@ void StorageReplicatedMergeTree::startup() restarting_thread.start(); /// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attempt to do it + /// TODO Do we still need startup_event? startup_event.wait(); - /// If we don't separate create/start steps, race condition will happen - /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. - background_executor.start(); startBackgroundMovesIfNeeded(); part_moves_between_shards_orchestrator.start(); @@ -6970,7 +6968,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI Stopwatch watch; /// Let's fetch new log entries firstly - queue.pullLogsToQueue(getZooKeeper()); + queue.pullLogsToQueue(getZooKeeper(), {}, ReplicatedMergeTreeQueue::SYNC); /// This is significant, because the execution of this task could be delayed at BackgroundPool. /// And we force it to be executed. @@ -7208,6 +7206,7 @@ MutationCommands StorageReplicatedMergeTree::getFirstAlterMutationCommandsForPar void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { + /// FIXME is it related to replication somehow? If it is we should start it from RestartingThread only if (areBackgroundMovesNeeded()) background_moves_executor.start(); } From 1f283aeb1f02d24af58692ecc486c09bba41c5e9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 17 Aug 2021 21:10:08 +0300 Subject: [PATCH 040/127] fix --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 22 ++++++++++++------- tests/queries/0_stateless/replication.lib | 4 +++- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 3a9f477597d..c71a79d2009 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -476,13 +476,10 @@ bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo & int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback, PullLogsReason reason) { std::lock_guard lock(pull_logs_to_queue_mutex); - if (storage.is_readonly && reason != LOAD) + if (storage.is_readonly && reason == SYNC) { - /// Pulling logs when replica is readonly may cause obscure bugs, allow it on replica startup only - if (reason == SYNC) - throw Exception(ErrorCodes::READONLY, "Cannot SYNC REPLICA, because replica is readonly"); - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Some background task ({}) tried to pull logs on readonly replica, it's a bug", reason); + throw Exception(ErrorCodes::READONLY, "Cannot SYNC REPLICA, because replica is readonly"); + /// TODO throw logical error for other reasons (except LOAD) } if (pull_log_blocker.isCancelled()) @@ -724,13 +721,22 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C std::vector> futures; for (const String & entry : entries_to_load) - futures.emplace_back(zookeeper->asyncGet(fs::path(zookeeper_path) / "mutations" / entry)); + futures.emplace_back(zookeeper->asyncTryGet(fs::path(zookeeper_path) / "mutations" / entry)); std::vector new_mutations; for (size_t i = 0; i < entries_to_load.size(); ++i) { + auto maybe_response = futures[i].get(); + if (maybe_response.error != Coordination::Error::ZOK) + { + assert(maybe_response.error == Coordination::Error::ZNONODE); + /// It's ok if it happened on server startup or table creation and replica loads all mutation entries. + /// It's also ok if mutation was killed. + LOG_WARNING(log, "Cannot get mutation node {} ({}), probably it was concurrently removed", entries_to_load[i], maybe_response.error); + continue; + } new_mutations.push_back(std::make_shared( - ReplicatedMergeTreeMutationEntry::parse(futures[i].get().data, entries_to_load[i]))); + ReplicatedMergeTreeMutationEntry::parse(maybe_response.data, entries_to_load[i]))); } bool some_mutations_are_probably_done = false; diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 77b09dee1e0..053e512747f 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -36,7 +36,9 @@ function try_sync_replicas() function check_replication_consistency() { # Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them - some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' LIMIT 1") + some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' ORDER BY rand() LIMIT 1") + $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: + some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' ORDER BY rand() LIMIT 1") $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: # Forcefully cancel mutations to avoid waiting for them to finish From 29b18d6bdc93267e71720850815e254ef92e4b39 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Fri, 20 Aug 2021 15:36:25 +0300 Subject: [PATCH 041/127] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../table-engines/mergetree-family/mergetree.md | 3 ++- .../table-engines/mergetree-family/mergetree.md | 10 +++++++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index de02fb465fd..402fc596388 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -396,7 +396,8 @@ Projections are an experimental feature. To enable them you must set the [allow_ Projections are not supported in the `SELECT` statements with [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. ### Projection Query {#projection-query} -A projection query is what defines a projection. It implicitly selects data from the parent table. It has the following syntax: +A projection query is what defines a projection. It implicitly selects data from the parent table. +**Syntax** ```sql SELECT [GROUP BY] [ORDER BY] diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 0f7519f2232..a651fddea36 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -383,7 +383,8 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Проекции не поддерживаются для запросов `SELECT` с модификатором [FINAL](../../../sql-reference/statements/select/from.md#select-from-final). ### Запрос проекции {#projection-query} -Запрос проекции — это то, что определяет проекцию. Такой запрос неявно выбирает данные из родительской таблицы. Он имеет следующий синтаксис: +Запрос проекции — это то, что определяет проекцию. Такой запрос неявно выбирает данные из родительской таблицы. +**Синтаксис** ```sql SELECT [GROUP BY] [ORDER BY] @@ -392,10 +393,13 @@ SELECT [GROUP BY] [ORDER BY] Проекции можно изменить или удалить с помощью запроса [ALTER](../../../sql-reference/statements/alter/projection.md). ### Хранение проекции {#projection-storage} -Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. Если есть секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. Если есть секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния ее хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. +Проекции хранятся в каталоге куска данных. Это похоже на хранение индексов, но используется подкаталог, в котором хранится анонимный кусок таблицы `MergeTree`. Таблица создается запросом определения проекции. +Если присутствует секция `GROUP BY`, то используется движок [AggregatingMergeTree](aggregatingmergetree.md), а все агрегатные функции преобразуются в `AggregateFunction`. +Если присутствует секция `ORDER BY`, таблица `MergeTree` использует ее в качестве выражения для первичного ключа. +Во время процесса слияния кусок данных проекции объединяется с помощью процедуры слияния хранилища. Контрольная сумма куска данных родительской таблицы включает кусок данных проекции. Другие процедуры аналогичны индексам пропуска данных. ### Анализ запросов {#projection-query-analysis} -1. Проверьте, можно ли использовать проекцию в данном запросе, то есть, что с ней выходит тот же результат, что и с запросом к базовой таблице. +1. Проверьте, можно ли использовать проекцию в данном запросе, то есть, что с ней получается тот же результат, что и с запросом к базовой таблице. 2. Выберите наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. 3. План запроса, который использует проекции, отличается от того, который использует исходные куски данных. При отсутствии проекции в некоторых кусках можно расширить план, чтобы «проецировать» на лету. From 7887f3892b1319a5828717c9d68efba9e93fe9e9 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 21 Aug 2021 13:47:06 +0300 Subject: [PATCH 042/127] Document queryID and initialQueryID functions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал две функции. --- .../functions/other-functions.md | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 577fdd668a2..33513745d3c 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2236,3 +2236,34 @@ defaultRoles() Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). +## queryID {#query-id} + +Returns the ID of the current query, which can be used instantly in other queries. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. + +**Syntax** + +``` sql +queryID() +``` + +**Returned value** + +- The ID of the current query. + +Type: [String](../../sql-reference/data-types/string.md) + +## initialQueryID {#initial-query-id} + +Returns the ID of the initial current query, which can be used instantly in other queries. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `initial_query_id`. + +**Syntax** + +``` sql +initialQueryID() +``` + +**Returned value** + +- The ID of the initial current query. + +Type: [String](../../sql-reference/data-types/string.md) From eeec8004e83ea41c0bc03b95179356a9c400b2b5 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 21 Aug 2021 23:26:27 +0300 Subject: [PATCH 043/127] Add the example MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил примеры. --- .../functions/other-functions.md | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 33513745d3c..d4c8ca59e56 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2240,6 +2240,8 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere Returns the ID of the current query, which can be used instantly in other queries. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. +In contrast to [initialQueryID](#initial-query-id) function `queryID` can return different results on shards (see example). And then the server will argue that constant column has different values. + **Syntax** ``` sql @@ -2252,10 +2254,22 @@ queryID() Type: [String](../../sql-reference/data-types/string.md) +**Example** + +In this example the result of the query will be three different values, one of which will match the `initial_query_id` for this query. + +Query: + +``` sql +SELECT queryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp'); +``` + ## initialQueryID {#initial-query-id} Returns the ID of the initial current query, which can be used instantly in other queries. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `initial_query_id`. +In contrast to [queryID](#query-id) function `initialQueryID` returns same results on shards (see example). + **Syntax** ``` sql @@ -2267,3 +2281,13 @@ initialQueryID() - The ID of the initial current query. Type: [String](../../sql-reference/data-types/string.md) + +**Example** + +In this example the result of the query will be three same values. + +Query: + +``` sql +SELECT initialQueryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp'); +``` From 80ebc6ccea5eeadbc183d4e16acc952592c9006a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 22 Aug 2021 15:09:49 +0000 Subject: [PATCH 044/127] Done --- CMakeLists.txt | 5 +++-- docker/packager/binary/build.sh | 17 ++++++++++++++++- docker/packager/packager | 2 ++ src/Compression/fuzzers/CMakeLists.txt | 8 +++++++- src/Storages/fuzzers/CMakeLists.txt | 11 ++++------- 5 files changed, 32 insertions(+), 11 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 1aef8c9fc8d..3d16d5da3f8 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -126,12 +126,13 @@ if (USE_STATIC_LIBRARIES) list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) endif () -# Implies ${WITH_COVERAGE} option (ENABLE_FUZZING "Fuzzy testing using libfuzzer" OFF) if (ENABLE_FUZZING) + # Also set WITH_COVERAGE=1 for better fuzzing process + # By default this is disabled, because fuzzers are built in CI with the clickhouse itself. + # And we don't want to enable coverage for it. message (STATUS "Fuzzing instrumentation enabled") - set (WITH_COVERAGE ON) set (FUZZER "libfuzzer") endif() diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index d6614bbb9e2..5e632e3c592 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -35,6 +35,22 @@ ninja $NINJA_FLAGS clickhouse-bundle ccache --show-config ||: ccache --show-stats ||: +# Also build fuzzers if any +FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') + +mkdir -p /output/fuzzers +for FUZZER_TARGET in $FUZZER_TARGETS +do + ninja $NINJA_FLAGS $FUZZER_TARGET + # Find this binary in build directory and strip it + FUZZER_PATH=$(find ./src -name $FUZZER_TARGET) + strip --strip-unneeded $FUZZER_PATH + mv $FUZZER_PATH /output/fuzzers +done + +tar -zcvf /output/fuzzers.tar.gz /output/fuzzers +rm -rf /output/fuzzers + mv ./programs/clickhouse* /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds find . -name '*.so' -print -exec mv '{}' /output \; @@ -95,4 +111,3 @@ then # files in place, and will fail because this directory is not writable. tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE" fi - diff --git a/docker/packager/packager b/docker/packager/packager index 95b7fcd8568..924f01dddce 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -120,6 +120,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ if sanitizer: result.append("SANITIZER={}".format(sanitizer)) + # Build fuzzers only with sanitizers + cmake_flags.append('-DENABLE_FUZZING=1') if build_type: result.append("BUILD_TYPE={}".format(build_type)) diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt index 74bf2d2649b..73b95d09fa3 100644 --- a/src/Compression/fuzzers/CMakeLists.txt +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -1,2 +1,8 @@ add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) -target_link_libraries (compressed_buffer_fuzzer PRIVATE fuzz_compression clickhouse_common_io ${LIB_FUZZING_ENGINE}) + +# Our code has strong cohesion and target associated with `Compression` also depends on `DataTypes`. +# But we can exclude some files which have dependencies in case of +# fuzzer related build (we are interested in fuzzing only particular part of our code). +# So, some symbols will be declared, but not defined. Unfortunately, this trick doesn't work with UBSan. +# If you want really small size of the resulted binary, just link with fuzz_compression and clickhouse_common_io +target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt index 93d3d2926bd..dcbd07ccd91 100644 --- a/src/Storages/fuzzers/CMakeLists.txt +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -1,11 +1,8 @@ -add_executable (mergetree_checksum_fuzzer - mergetree_checksum_fuzzer.cpp - "${ClickHouse_SOURCE_DIR}/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp" - "${ClickHouse_SOURCE_DIR}/src/Compression/CompressedReadBuffer.cpp" - "${ClickHouse_SOURCE_DIR}/src/Compression/CompressedWriteBuffer.cpp" -) -target_link_libraries (mergetree_checksum_fuzzer PRIVATE clickhouse_common_io fuzz_compression ${LIB_FUZZING_ENGINE}) +add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.cpp) + +# Look at comment around fuzz_compression target declaration +target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) target_link_libraries (columns_description_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) From ac2e9a27e69c88ff835cafd5a34fb74cde591842 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 23 Aug 2021 16:08:23 +0300 Subject: [PATCH 045/127] Fixes #26672. Lower restrictions in StorageInMemoryMetadata::check. --- src/DataTypes/DataTypeEnum.h | 4 +++- src/Storages/StorageInMemoryMetadata.cpp | 15 +++++++++++++-- .../0_stateless/02012_changed_enum_type.reference | 2 ++ .../0_stateless/02012_changed_enum_type.sql | 13 +++++++++++++ ...012_changed_enum_type_non_replicated.reference | 4 ++++ .../02012_changed_enum_type_non_replicated.sql | 8 ++++++++ 6 files changed, 43 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02012_changed_enum_type.reference create mode 100644 tests/queries/0_stateless/02012_changed_enum_type.sql create mode 100644 tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference create mode 100644 tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 57657d1d110..7d0aad5454e 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -27,6 +27,8 @@ public: bool isCategorial() const override { return true; } bool canBeInsideNullable() const override { return true; } bool isComparable() const override { return true; } + + virtual bool contains(const IDataType & rhs) const = 0; }; @@ -76,7 +78,7 @@ public: /// Example: /// Enum('a' = 1, 'b' = 2) -> Enum('c' = 1, 'b' = 2, 'd' = 3) OK /// Enum('a' = 1, 'b' = 2) -> Enum('a' = 2, 'b' = 1) NOT OK - bool contains(const IDataType & rhs) const; + virtual bool contains(const IDataType & rhs) const override; SerializationPtr doGetDefaultSerialization() const override; }; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 5183b925141..2642a06a9bd 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -493,6 +494,15 @@ namespace return res; } + + bool isEnumSubset(const IDataType* lhs, const DataTypePtr& rhs) + { + const WhichDataType & which = WhichDataType{lhs}; + if (!which.isEnum()) + return false; + IDataTypeEnum const* enum_type = dynamic_cast(lhs); + return enum_type->contains(*rhs); + } } void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const @@ -544,12 +554,13 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns) column.name, listOfColumns(available_columns)); - if (!column.type->equals(*it->getMapped())) + auto const mappedType = it->getMapped(); + if (!column.type->equals(*mappedType) && !isEnumSubset(mappedType, column.type)) throw Exception( ErrorCodes::TYPE_MISMATCH, "Type mismatch for column {}. Column has type {}, got type {}", column.name, - it->getMapped()->getName(), + mappedType->getName(), column.type->getName()); if (unique_names.end() != unique_names.find(column.name)) diff --git a/tests/queries/0_stateless/02012_changed_enum_type.reference b/tests/queries/0_stateless/02012_changed_enum_type.reference new file mode 100644 index 00000000000..814f4a42292 --- /dev/null +++ b/tests/queries/0_stateless/02012_changed_enum_type.reference @@ -0,0 +1,2 @@ +one +two diff --git a/tests/queries/0_stateless/02012_changed_enum_type.sql b/tests/queries/0_stateless/02012_changed_enum_type.sql new file mode 100644 index 00000000000..28325dc55fb --- /dev/null +++ b/tests/queries/0_stateless/02012_changed_enum_type.sql @@ -0,0 +1,13 @@ +create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) +engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r1') +ORDER BY a; + +insert into enum_alter_issue values ('one'), ('two'); +alter table enum_alter_issue modify column a Enum8('one' = 1, 'two' = 2, 'three' = 3); +insert into enum_alter_issue values ('one'), ('two'); + +alter table enum_alter_issue detach partition id 'all'; +alter table enum_alter_issue attach partition id 'all'; +select * from enum_alter_issue; + +drop table enum_alter_issue; diff --git a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference new file mode 100644 index 00000000000..22e5ee9f412 --- /dev/null +++ b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference @@ -0,0 +1,4 @@ +one +two +one +two diff --git a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql new file mode 100644 index 00000000000..86a6caba805 --- /dev/null +++ b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql @@ -0,0 +1,8 @@ +create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) engine = MergeTree() ORDER BY a; +insert into enum_alter_issue values ('one'), ('two'); +alter table enum_alter_issue modify column a Enum8('one' = 1, 'two' = 2, 'three' = 3); +insert into enum_alter_issue values ('one'), ('two'); +alter table enum_alter_issue detach partition id 'all'; +alter table enum_alter_issue attach partition id 'all'; +select * from enum_alter_issue; +drop table enum_alter_issue; From 1c956481a14577cefb82097e870738ee193d5468 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 23 Aug 2021 13:21:39 +0000 Subject: [PATCH 046/127] Updated deb/build.sh and fixed shellcheck --- docker/packager/binary/build.sh | 7 ++++--- docker/packager/deb/build.sh | 14 ++++++++++++++ 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 5e632e3c592..70139a04852 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -41,11 +41,12 @@ FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' mkdir -p /output/fuzzers for FUZZER_TARGET in $FUZZER_TARGETS do + # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. ninja $NINJA_FLAGS $FUZZER_TARGET # Find this binary in build directory and strip it - FUZZER_PATH=$(find ./src -name $FUZZER_TARGET) - strip --strip-unneeded $FUZZER_PATH - mv $FUZZER_PATH /output/fuzzers + FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") + strip --strip-unneeded "$FUZZER_PATH" + mv "$FUZZER_PATH" /output/fuzzers done tar -zcvf /output/fuzzers.tar.gz /output/fuzzers diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index 4e14574b738..7bcd79a919f 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -23,6 +23,20 @@ then echo "Place $BINARY_OUTPUT to output" mkdir /output/binary ||: # if exists mv /build/obj-*/programs/clickhouse* /output/binary + + # Copy all fuzzers + FUZZER_TARGETS=$(find /build/obj-*/src -name '*_fuzzer' -execdir basename {} ';' | tr '\n' ' ') + mkdir -p /output/fuzzers ||: # if exists + for FUZZER_TARGET in $FUZZER_TARGETS + do + FUZZER_PATH=$(find /build/obj-*/src -name "$FUZZER_TARGET") + strip --strip-unneeded "$FUZZER_PATH" + mv "$FUZZER_PATH" /output/fuzzers ||: # if exists + done + + tar -zcvf /output/fuzzers.tar.gz /output/fuzzers + rm -rf /output/fuzzers + if [ "$BINARY_OUTPUT" = "tests" ] then mv /build/obj-*/src/unit_tests_dbms /output/binary From 77430a1baa470f07cc18cc823664a69bcafe48b2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 23 Aug 2021 14:36:26 +0000 Subject: [PATCH 047/127] better --- docker/packager/binary/build.sh | 30 ++++++++++++++++-------------- docker/packager/deb/build.sh | 26 ++++++++++++++++---------- 2 files changed, 32 insertions(+), 24 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 70139a04852..0c88cbe57aa 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -35,22 +35,24 @@ ninja $NINJA_FLAGS clickhouse-bundle ccache --show-config ||: ccache --show-stats ||: -# Also build fuzzers if any -FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') + # Also build fuzzers if any +if [[ "${CMAKE_FLAGS[@]}" == "*ENABLE_FUZZING*" ]]; then + FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') -mkdir -p /output/fuzzers -for FUZZER_TARGET in $FUZZER_TARGETS -do - # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. - ninja $NINJA_FLAGS $FUZZER_TARGET - # Find this binary in build directory and strip it - FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") - strip --strip-unneeded "$FUZZER_PATH" - mv "$FUZZER_PATH" /output/fuzzers -done + mkdir -p /output/fuzzers + for FUZZER_TARGET in $FUZZER_TARGETS + do + # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. + ninja $NINJA_FLAGS $FUZZER_TARGET + # Find this binary in build directory and strip it + FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") + strip --strip-unneeded "$FUZZER_PATH" + mv "$FUZZER_PATH" /output/fuzzers + done -tar -zcvf /output/fuzzers.tar.gz /output/fuzzers -rm -rf /output/fuzzers + tar -zcvf /output/fuzzers.tar.gz /output/fuzzers + rm -rf /output/fuzzers +fi mv ./programs/clickhouse* /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index 7bcd79a919f..d343287eee9 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -24,18 +24,24 @@ then mkdir /output/binary ||: # if exists mv /build/obj-*/programs/clickhouse* /output/binary - # Copy all fuzzers + # Copy all fuzzers if any FUZZER_TARGETS=$(find /build/obj-*/src -name '*_fuzzer' -execdir basename {} ';' | tr '\n' ' ') - mkdir -p /output/fuzzers ||: # if exists - for FUZZER_TARGET in $FUZZER_TARGETS - do - FUZZER_PATH=$(find /build/obj-*/src -name "$FUZZER_TARGET") - strip --strip-unneeded "$FUZZER_PATH" - mv "$FUZZER_PATH" /output/fuzzers ||: # if exists - done - tar -zcvf /output/fuzzers.tar.gz /output/fuzzers - rm -rf /output/fuzzers + if [[ $FUZZER_TARGETS != "" ]]; then + + mkdir -p /output/fuzzers ||: # if exists + for FUZZER_TARGET in $FUZZER_TARGETS + do + FUZZER_PATH=$(find /build/obj-*/src -name "$FUZZER_TARGET") + strip --strip-unneeded "$FUZZER_PATH" + mv "$FUZZER_PATH" /output/fuzzers ||: # if exists + done + + + tar -zcvf /output/fuzzers.tar.gz /output/fuzzers + rm -rf /output/fuzzers + + fi if [ "$BINARY_OUTPUT" = "tests" ] then From 6dad70a1fb83367d98d4debde2511a736cef0e8d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 23 Aug 2021 17:55:21 +0300 Subject: [PATCH 048/127] Add check whether enums have the same underlying type or not --- src/Storages/StorageInMemoryMetadata.cpp | 4 +++- .../02012_changed_enum_type_incompatible.reference | 0 .../02012_changed_enum_type_incompatible.sql | 12 ++++++++++++ 3 files changed, 15 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02012_changed_enum_type_incompatible.reference create mode 100644 tests/queries/0_stateless/02012_changed_enum_type_incompatible.sql diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 2642a06a9bd..bda4ca88a69 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -501,7 +501,9 @@ namespace if (!which.isEnum()) return false; IDataTypeEnum const* enum_type = dynamic_cast(lhs); - return enum_type->contains(*rhs); + if (!enum_type->contains(*rhs)) + return false; + return enum_type->getMaximumSizeOfValueInMemory() == rhs->getMaximumSizeOfValueInMemory(); } } diff --git a/tests/queries/0_stateless/02012_changed_enum_type_incompatible.reference b/tests/queries/0_stateless/02012_changed_enum_type_incompatible.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02012_changed_enum_type_incompatible.sql b/tests/queries/0_stateless/02012_changed_enum_type_incompatible.sql new file mode 100644 index 00000000000..e86023c96fa --- /dev/null +++ b/tests/queries/0_stateless/02012_changed_enum_type_incompatible.sql @@ -0,0 +1,12 @@ +drop table if exists enum_alter_issue; +create table enum_alter_issue (a Enum16('one' = 1, 'two' = 2), b Int) +engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r2') +ORDER BY b; + +insert into enum_alter_issue values ('one', 1), ('two', 1); +alter table enum_alter_issue detach partition id 'all'; +alter table enum_alter_issue modify column a Enum8('one' = 1, 'two' = 2, 'three' = 3); +insert into enum_alter_issue values ('one', 1), ('two', 1); + +alter table enum_alter_issue attach partition id 'all'; -- {serverError TYPE_MISMATCH} +drop table enum_alter_issue; From 7b7959816ece355736511c6632eb757ddce9d70f Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 23 Aug 2021 20:41:48 +0300 Subject: [PATCH 049/127] Apply suggestions from code review Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 901f7d0a8bb..519e9e60ff5 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -378,7 +378,7 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT ## Проекции {#projections} Проекции похожи на [материализованные представления](../../../sql-reference/statements/create/view.md#materialized), но определяются на уровне кусков данных. Это обеспечивает гарантии согласованности данных наряду с автоматическим использованием в запросах. -Проекции это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force-optimize-projection). +Проекции — это экспериментальная возможность. Чтобы включить поддержку проекций, установите настройку [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) в значение `1`. См. также настройку [force_optimize_projection ](../../../operations/settings/settings.md#force-optimize-projection). Проекции не поддерживаются для запросов `SELECT` с модификатором [FINAL](../../../sql-reference/statements/select/from.md#select-from-final). @@ -401,7 +401,7 @@ SELECT [GROUP BY] [ORDER BY] ### Анализ запросов {#projection-query-analysis} 1. Проверьте, можно ли использовать проекцию в данном запросе, то есть, что с ней получается тот же результат, что и с запросом к базовой таблице. 2. Выберите наиболее подходящее совпадение, содержащее наименьшее количество гранул для чтения. -3. План запроса, который использует проекции, отличается от того, который использует исходные куски данных. При отсутствии проекции в некоторых кусках можно расширить план, чтобы «проецировать» на лету. +3. План запроса, который использует проекции, отличается от того, который использует исходные куски данных. Если в некоторых кусках проекции отсутствуют, можно расширить план, чтобы «проецировать» на лету. ## Конкурентный доступ к данным {#concurrent-data-access} From afae867060d1904638e301d7eb83ed34d3ba4161 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 23 Aug 2021 18:31:17 +0000 Subject: [PATCH 050/127] Fix build --- docker/packager/binary/build.sh | 3 ++- docker/packager/packager | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 0c88cbe57aa..99655450946 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -36,7 +36,8 @@ ccache --show-config ||: ccache --show-stats ||: # Also build fuzzers if any -if [[ "${CMAKE_FLAGS[@]}" == "*ENABLE_FUZZING*" ]]; then +if [ -n "$FUZZER" ] +then FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') mkdir -p /output/fuzzers diff --git a/docker/packager/packager b/docker/packager/packager index 924f01dddce..50bb4d8e23b 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -121,6 +121,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ if sanitizer: result.append("SANITIZER={}".format(sanitizer)) # Build fuzzers only with sanitizers + # This is needed to check necessity particularly in build script + result.append("FUZZER=TRUE") cmake_flags.append('-DENABLE_FUZZING=1') if build_type: result.append("BUILD_TYPE={}".format(build_type)) From 3d9668cf40e5d677ed780a17753f5390361935a6 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 23 Aug 2021 22:44:01 +0300 Subject: [PATCH 051/127] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Перевел на русский язык. --- .../functions/other-functions.md | 27 ++++++-- .../functions/other-functions.md | 69 +++++++++++++++++++ 2 files changed, 89 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index d4c8ca59e56..5d9e02b8200 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2240,7 +2240,8 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere Returns the ID of the current query, which can be used instantly in other queries. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. -In contrast to [initialQueryID](#initial-query-id) function `queryID` can return different results on shards (see example). And then the server will argue that constant column has different values. +!!! warning "Warning" + In contrast to [initialQueryID](#initial-query-id) function `queryID` can return different results on shards (see example). And then the server will argue that constant column has different values. **Syntax** @@ -2256,12 +2257,18 @@ Type: [String](../../sql-reference/data-types/string.md) **Example** -In this example the result of the query will be three different values, one of which will match the `initial_query_id` for this query. - Query: ``` sql -SELECT queryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp'); +SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); +``` + +Result: + +``` text +┌─count()─┐ +│ 3 │ +└─────────┘ ``` ## initialQueryID {#initial-query-id} @@ -2284,10 +2291,16 @@ Type: [String](../../sql-reference/data-types/string.md) **Example** -In this example the result of the query will be three same values. - Query: ``` sql -SELECT initialQueryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp'); +SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); +``` + +Result: + +``` text +┌─count()─┐ +│ 1 │ +└─────────┘ ``` diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index c023786b788..5088bbbaceb 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2185,3 +2185,72 @@ defaultRoles() - Список ролей по умолчанию. Тип: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). + +## queryID {#query-id} + +Возвращает идентификатор текущего запроса, который может быть мгновенно использован в других запросах. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `query_id`. + +!!! warning "Предупреждение" + В отличие от [initialQueryID](#initial-query-id) функция `queryID` может возвращать различные значения на шардах (см. пример), что неверно для столбца констант. + +**Синтаксис** + +``` sql +queryID() +``` + +**Возвращаемое значение** + +- Идентификатор текущего запроса. + +Тип: [String](../../sql-reference/data-types/string.md) + +**Пример** + +Запрос: + +``` sql +SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); +``` + +Результат: + +``` text +┌─count()─┐ +│ 3 │ +└─────────┘ +``` + +## initialQueryID {#initial-query-id} + +Возвращает идентификатор родительского запроса, который может быть мгновенно использован в других запросах. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `initial_query_id`. + +В отличие от [queryID](#query-id) функция `initialQueryID` возвращает одинаковые значения на шардах (см. пример). + +**Синтаксис** + +``` sql +initialQueryID() +``` + +**Возвращаемое значение** + +- Идентификатор родительского запроса. + +Тип: [String](../../sql-reference/data-types/string.md) + +**Пример** + +Запрос: + +``` sql +SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); +``` + +Результат: + +``` text +┌─count()─┐ +│ 1 │ +└─────────┘ +``` From e78f9b3d4b3596f092f05172190bd84561112c2b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 24 Aug 2021 00:42:03 +0300 Subject: [PATCH 052/127] code cleanup --- src/DataTypes/DataTypeEnum.h | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 16 +++++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 7d0aad5454e..92c72b87afa 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -78,7 +78,7 @@ public: /// Example: /// Enum('a' = 1, 'b' = 2) -> Enum('c' = 1, 'b' = 2, 'd' = 3) OK /// Enum('a' = 1, 'b' = 2) -> Enum('a' = 2, 'b' = 1) NOT OK - virtual bool contains(const IDataType & rhs) const override; + bool contains(const IDataType & rhs) const override; SerializationPtr doGetDefaultSerialization() const override; }; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index bda4ca88a69..7f69017e752 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -495,9 +495,15 @@ namespace return res; } - bool isEnumSubset(const IDataType* lhs, const DataTypePtr& rhs) + /* + * This function checks compatibility of enums. It returns true if: + * 1. Both types are enums. + * 2. The first type can represent all possible values of the second one. + * 3. Both types require the same amount of memory. + */ + bool isCompatibleEnumTypes(const IDataType* lhs, const DataTypePtr& rhs) { - const WhichDataType & which = WhichDataType{lhs}; + WhichDataType which{lhs}; if (!which.isEnum()) return false; IDataTypeEnum const* enum_type = dynamic_cast(lhs); @@ -556,13 +562,13 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns) column.name, listOfColumns(available_columns)); - auto const mappedType = it->getMapped(); - if (!column.type->equals(*mappedType) && !isEnumSubset(mappedType, column.type)) + auto const mapped_type = it->getMapped(); + if (!column.type->equals(*mapped_type) && !isCompatibleEnumTypes(mapped_type, column.type)) throw Exception( ErrorCodes::TYPE_MISMATCH, "Type mismatch for column {}. Column has type {}, got type {}", column.name, - mappedType->getName(), + mapped_type->getName(), column.type->getName()); if (unique_names.end() != unique_names.find(column.name)) From ac4a51615219e0f91799a1580d8da1e1f059bcd0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 23 Aug 2021 21:45:48 +0000 Subject: [PATCH 053/127] Fix build --- docker/packager/binary/build.sh | 47 +++++++++++++++++++-------------- docker/packager/packager | 4 --- 2 files changed, 27 insertions(+), 24 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 99655450946..9a77d3b2fe3 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -35,26 +35,6 @@ ninja $NINJA_FLAGS clickhouse-bundle ccache --show-config ||: ccache --show-stats ||: - # Also build fuzzers if any -if [ -n "$FUZZER" ] -then - FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') - - mkdir -p /output/fuzzers - for FUZZER_TARGET in $FUZZER_TARGETS - do - # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. - ninja $NINJA_FLAGS $FUZZER_TARGET - # Find this binary in build directory and strip it - FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") - strip --strip-unneeded "$FUZZER_PATH" - mv "$FUZZER_PATH" /output/fuzzers - done - - tar -zcvf /output/fuzzers.tar.gz /output/fuzzers - rm -rf /output/fuzzers -fi - mv ./programs/clickhouse* /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds find . -name '*.so' -print -exec mv '{}' /output \; @@ -115,3 +95,30 @@ then # files in place, and will fail because this directory is not writable. tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE" fi + +# Also build fuzzers if any sanitizer specified +if [ -n "$SANITIZER" ] +then + # Delete previous cache, because we add a new flag -DENABLE_FUZZING=1 + rm -rf CMakeCache.txt CMakeFiles/ + + # Hope, that the most part of files will be in cache, so we just link new executables + cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" \ + "-DSANITIZE=$SANITIZER" -DENABLE_FUZZING=1 -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. + + FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') + + mkdir -p /output/fuzzers + for FUZZER_TARGET in $FUZZER_TARGETS + do + # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. + ninja $NINJA_FLAGS $FUZZER_TARGET + # Find this binary in build directory and strip it + FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") + strip --strip-unneeded "$FUZZER_PATH" + mv "$FUZZER_PATH" /output/fuzzers + done + + tar -zcvf /output/fuzzers.tar.gz /output/fuzzers + rm -rf /output/fuzzers +fi diff --git a/docker/packager/packager b/docker/packager/packager index 50bb4d8e23b..95b7fcd8568 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -120,10 +120,6 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ if sanitizer: result.append("SANITIZER={}".format(sanitizer)) - # Build fuzzers only with sanitizers - # This is needed to check necessity particularly in build script - result.append("FUZZER=TRUE") - cmake_flags.append('-DENABLE_FUZZING=1') if build_type: result.append("BUILD_TYPE={}".format(build_type)) From d5b116b2da8226f7f1a87263ce443480a516b53c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 24 Aug 2021 01:00:41 +0300 Subject: [PATCH 054/127] update tests --- .../queries/0_stateless/02012_changed_enum_type.reference | 2 -- .../02012_changed_enum_type_non_replicated.reference | 2 +- .../02012_changed_enum_type_non_replicated.sql | 2 +- .../02012_zookeeper_changed_enum_type.reference | 4 ++++ ...num_type.sql => 02012_zookeeper_changed_enum_type.sql} | 8 ++++---- ...12_zookeeper_changed_enum_type_incompatible.reference} | 0 ...=> 02012_zookeeper_changed_enum_type_incompatible.sql} | 0 7 files changed, 10 insertions(+), 8 deletions(-) delete mode 100644 tests/queries/0_stateless/02012_changed_enum_type.reference create mode 100644 tests/queries/0_stateless/02012_zookeeper_changed_enum_type.reference rename tests/queries/0_stateless/{02012_changed_enum_type.sql => 02012_zookeeper_changed_enum_type.sql} (59%) rename tests/queries/0_stateless/{02012_changed_enum_type_incompatible.reference => 02012_zookeeper_changed_enum_type_incompatible.reference} (100%) rename tests/queries/0_stateless/{02012_changed_enum_type_incompatible.sql => 02012_zookeeper_changed_enum_type_incompatible.sql} (100%) diff --git a/tests/queries/0_stateless/02012_changed_enum_type.reference b/tests/queries/0_stateless/02012_changed_enum_type.reference deleted file mode 100644 index 814f4a42292..00000000000 --- a/tests/queries/0_stateless/02012_changed_enum_type.reference +++ /dev/null @@ -1,2 +0,0 @@ -one -two diff --git a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference index 22e5ee9f412..abb96cefb59 100644 --- a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference +++ b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.reference @@ -1,4 +1,4 @@ one +one two -one two diff --git a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql index 86a6caba805..9e25df0f41a 100644 --- a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql +++ b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql @@ -4,5 +4,5 @@ alter table enum_alter_issue modify column a Enum8('one' = 1, 'two' = 2, 'three' insert into enum_alter_issue values ('one'), ('two'); alter table enum_alter_issue detach partition id 'all'; alter table enum_alter_issue attach partition id 'all'; -select * from enum_alter_issue; +select * from enum_alter_issue order by a; drop table enum_alter_issue; diff --git a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.reference b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.reference new file mode 100644 index 00000000000..a83c714a5cf --- /dev/null +++ b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.reference @@ -0,0 +1,4 @@ +one 1 +two 2 +one 3 +two 4 diff --git a/tests/queries/0_stateless/02012_changed_enum_type.sql b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql similarity index 59% rename from tests/queries/0_stateless/02012_changed_enum_type.sql rename to tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql index 28325dc55fb..0c95c7ff403 100644 --- a/tests/queries/0_stateless/02012_changed_enum_type.sql +++ b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql @@ -1,13 +1,13 @@ -create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) +create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2), b Int) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r1') ORDER BY a; -insert into enum_alter_issue values ('one'), ('two'); +insert into enum_alter_issue values ('one', 1), ('two', 2); alter table enum_alter_issue modify column a Enum8('one' = 1, 'two' = 2, 'three' = 3); -insert into enum_alter_issue values ('one'), ('two'); +insert into enum_alter_issue values ('one', 3), ('two', 4); alter table enum_alter_issue detach partition id 'all'; alter table enum_alter_issue attach partition id 'all'; -select * from enum_alter_issue; +select * from enum_alter_issue order by b; drop table enum_alter_issue; diff --git a/tests/queries/0_stateless/02012_changed_enum_type_incompatible.reference b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.reference similarity index 100% rename from tests/queries/0_stateless/02012_changed_enum_type_incompatible.reference rename to tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.reference diff --git a/tests/queries/0_stateless/02012_changed_enum_type_incompatible.sql b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql similarity index 100% rename from tests/queries/0_stateless/02012_changed_enum_type_incompatible.sql rename to tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql From 242ef3dcb5d5f2d9f7f3d5c40e265b93146cd435 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 24 Aug 2021 00:09:19 +0000 Subject: [PATCH 055/127] Proper build --- docker/packager/binary/build.sh | 37 +++++++++------------------------ docker/packager/deb/build.sh | 25 ++++++---------------- docker/packager/other/fuzzer.sh | 30 ++++++++++++++++++++++++++ release | 3 --- 4 files changed, 46 insertions(+), 49 deletions(-) create mode 100644 docker/packager/other/fuzzer.sh diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 9a77d3b2fe3..b9900e34bf1 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -83,6 +83,16 @@ then mv "$COMBINED_OUTPUT.tgz" /output fi +# Also build fuzzers if any sanitizer specified +if [ -n "$SANITIZER" ] +then + # Currently we are in build/build_docker directory + ../docker/packager/other/fuzzer.sh +fi + +ccache --show-config ||: +ccache --show-stats ||: + if [ "${CCACHE_DEBUG:-}" == "1" ] then find . -name '*.ccache-*' -print0 \ @@ -95,30 +105,3 @@ then # files in place, and will fail because this directory is not writable. tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE" fi - -# Also build fuzzers if any sanitizer specified -if [ -n "$SANITIZER" ] -then - # Delete previous cache, because we add a new flag -DENABLE_FUZZING=1 - rm -rf CMakeCache.txt CMakeFiles/ - - # Hope, that the most part of files will be in cache, so we just link new executables - cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" \ - "-DSANITIZE=$SANITIZER" -DENABLE_FUZZING=1 -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. - - FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') - - mkdir -p /output/fuzzers - for FUZZER_TARGET in $FUZZER_TARGETS - do - # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. - ninja $NINJA_FLAGS $FUZZER_TARGET - # Find this binary in build directory and strip it - FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") - strip --strip-unneeded "$FUZZER_PATH" - mv "$FUZZER_PATH" /output/fuzzers - done - - tar -zcvf /output/fuzzers.tar.gz /output/fuzzers - rm -rf /output/fuzzers -fi diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index d343287eee9..5016e40dd2e 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -24,31 +24,18 @@ then mkdir /output/binary ||: # if exists mv /build/obj-*/programs/clickhouse* /output/binary - # Copy all fuzzers if any - FUZZER_TARGETS=$(find /build/obj-*/src -name '*_fuzzer' -execdir basename {} ';' | tr '\n' ' ') - - if [[ $FUZZER_TARGETS != "" ]]; then - - mkdir -p /output/fuzzers ||: # if exists - for FUZZER_TARGET in $FUZZER_TARGETS - do - FUZZER_PATH=$(find /build/obj-*/src -name "$FUZZER_TARGET") - strip --strip-unneeded "$FUZZER_PATH" - mv "$FUZZER_PATH" /output/fuzzers ||: # if exists - done - - - tar -zcvf /output/fuzzers.tar.gz /output/fuzzers - rm -rf /output/fuzzers - - fi - if [ "$BINARY_OUTPUT" = "tests" ] then mv /build/obj-*/src/unit_tests_dbms /output/binary fi fi +# Also build fuzzers if any sanitizer specified +if [ -n "$SANITIZER" ] +then + build/docker/packager/other/fuzzer.sh +fi + ccache --show-config ||: ccache --show-stats ||: diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh new file mode 100644 index 00000000000..d6609d3c632 --- /dev/null +++ b/docker/packager/other/fuzzer.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +set -x -e + +# This script is responsible for building all fuzzers, and copy them to output directory +# as an archive. + +# Delete previous cache, because we add a new flags -DENABLE_FUZZING=1 and -DFUZZER=libfuzzer +rm -f CMakeCache.txt +read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" +# Hope, that the most part of files will be in cache, so we just link new executables +cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" \ + "-DSANITIZE=$SANITIZER" -DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0 \ + -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. + +FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') + +mkdir -p /output/fuzzers +for FUZZER_TARGET in $FUZZER_TARGETS +do + # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. + ninja $NINJA_FLAGS $FUZZER_TARGET + # Find this binary in build directory and strip it + FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") + strip --strip-unneeded "$FUZZER_PATH" + mv "$FUZZER_PATH" /output/fuzzers +done + +tar -zcvf /output/fuzzers.tar.gz /output/fuzzers +rm -rf /output/fuzzers diff --git a/release b/release index de549595d43..6e6970d7b00 100755 --- a/release +++ b/release @@ -60,9 +60,6 @@ then elif [[ "$SANITIZER" == "thread" ]]; then VERSION_POSTFIX+="+tsan" elif [[ "$SANITIZER" == "memory" ]]; then VERSION_POSTFIX+="+msan" elif [[ "$SANITIZER" == "undefined" ]]; then VERSION_POSTFIX+="+ubsan" - elif [[ "$SANITIZER" == "libfuzzer" ]]; then - VERSION_POSTFIX+="+libfuzzer" - MALLOC_OPTS="-DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0" else echo "Unknown value of SANITIZER variable: $SANITIZER" exit 3 From 697f2bcedbb0fe62fffe11eca83dc39fb2f79f53 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 23 Aug 2021 22:28:30 +0300 Subject: [PATCH 056/127] fix --- tests/queries/0_stateless/replication.lib | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 053e512747f..62417822c6b 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -35,6 +35,16 @@ function try_sync_replicas() function check_replication_consistency() { + # Do not check anything if all replicas are readonly, + # because is this case all replicas are probably lost (it may happen and it's not a bug) + res=$($CLICKHOUSE_CLIENT -q "SELECT count() - sum(is_readonly) FROM system.replicas WHERE database=currentDatabase() AND table LIKE '$1%'") + if [ $res -eq 0 ]; then + # Print dummy lines + echo "Replication did not hang: synced all replicas of $1" + echo "Consistency: 1" + return 0 + fi + # Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' ORDER BY rand() LIMIT 1") $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: From 19fc7233287bfd7007c297ac56265983c2266c8d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 24 Aug 2021 11:14:37 +0000 Subject: [PATCH 057/127] chmod +x --- docker/packager/other/fuzzer.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 docker/packager/other/fuzzer.sh diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh old mode 100644 new mode 100755 From 987421862a7dcc0fb0d60209f42a5eb773e9e005 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 24 Aug 2021 14:19:34 +0300 Subject: [PATCH 058/127] exclude tests --- tests/queries/skip_list.json | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index c2b5782e766..9d8c3ca979e 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -161,6 +161,9 @@ "00980_zookeeper_merge_tree_alter_settings", "00980_merge_alter_settings", "02009_array_join_partition", + "02012_changed_enum_type_non_replicated", + "02012_zookeeper_changed_enum_type", + "02012_zookeeper_changed_enum_type_incompatible", /// Old syntax is not allowed "01062_alter_on_mutataion_zookeeper", "00925_zookeeper_empty_replicated_merge_tree_optimize_final", From 1984940eb635d12e62c4af6b0eb362a7248a149c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 24 Aug 2021 14:59:35 +0300 Subject: [PATCH 059/127] Update all StorageInMemoryMetadata::check overloads --- src/Storages/StorageInMemoryMetadata.cpp | 25 ++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 7f69017e752..27ea8fcbb06 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -501,12 +501,12 @@ namespace * 2. The first type can represent all possible values of the second one. * 3. Both types require the same amount of memory. */ - bool isCompatibleEnumTypes(const IDataType* lhs, const DataTypePtr& rhs) + bool isCompatibleEnumTypes(const IDataType * lhs, const IDataType * rhs) { WhichDataType which{lhs}; if (!which.isEnum()) return false; - IDataTypeEnum const* enum_type = dynamic_cast(lhs); + IDataTypeEnum const * enum_type = dynamic_cast(lhs); if (!enum_type->contains(*rhs)) return false; return enum_type->getMaximumSizeOfValueInMemory() == rhs->getMaximumSizeOfValueInMemory(); @@ -562,13 +562,13 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns) column.name, listOfColumns(available_columns)); - auto const mapped_type = it->getMapped(); - if (!column.type->equals(*mapped_type) && !isCompatibleEnumTypes(mapped_type, column.type)) + const auto * available_type = it->getMapped(); + if (!column.type->equals(*available_type) && !isCompatibleEnumTypes(available_type, column.type.get())) throw Exception( ErrorCodes::TYPE_MISMATCH, "Type mismatch for column {}. Column has type {}, got type {}", column.name, - mapped_type->getName(), + available_type->getName(), column.type->getName()); if (unique_names.end() != unique_names.find(column.name)) @@ -607,16 +607,16 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns, name, listOfColumns(available_columns)); - const auto & provided_column_type = *it->getMapped(); - const auto & available_column_type = *jt->getMapped(); + const auto * provided_column_type = it->getMapped(); + const auto * available_column_type = jt->getMapped(); - if (!provided_column_type.equals(available_column_type)) + if (!provided_column_type->equals(*available_column_type) && !isCompatibleEnumTypes(available_column_type, provided_column_type)) throw Exception( ErrorCodes::TYPE_MISMATCH, "Type mismatch for column {}. Column has type {}, got type {}", name, - provided_column_type.getName(), - available_column_type.getName()); + available_column_type->getName(), + provided_column_type->getName()); if (unique_names.end() != unique_names.find(name)) throw Exception(ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE, @@ -651,12 +651,13 @@ void StorageInMemoryMetadata::check(const Block & block, bool need_all) const column.name, listOfColumns(available_columns)); - if (!column.type->equals(*it->getMapped())) + const auto * available_type = it->getMapped(); + if (!column.type->equals(*available_type) && !isCompatibleEnumTypes(available_type, column.type.get())) throw Exception( ErrorCodes::TYPE_MISMATCH, "Type mismatch for column {}. Column has type {}, got type {}", column.name, - it->getMapped()->getName(), + available_type->getName(), column.type->getName()); } From 95cf0634dcd7e7cf2e2c701ded44ce8bc85a97dd Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Aug 2021 15:30:31 +0300 Subject: [PATCH 060/127] Clickhouse-keeper: renames and comments --- programs/keeper/Keeper.cpp | 4 +- programs/server/Server.cpp | 4 +- src/Coordination/Changelog.cpp | 47 ++++-- src/Coordination/Changelog.h | 4 +- ...ageDispatcher.cpp => KeeperDispatcher.cpp} | 68 ++++++--- ...StorageDispatcher.h => KeeperDispatcher.h} | 28 +++- src/Coordination/KeeperLogStore.h | 14 ++ src/Coordination/KeeperServer.h | 9 ++ src/Coordination/KeeperStateMachine.cpp | 59 +++++--- src/Coordination/KeeperStateMachine.h | 19 +++ src/Coordination/KeeperStorage.cpp | 143 +++++++++--------- src/Coordination/KeeperStorage.h | 31 +++- src/Interpreters/Context.cpp | 12 +- src/Interpreters/Context.h | 8 +- src/Server/KeeperTCPHandler.cpp | 2 +- src/Server/KeeperTCPHandler.h | 4 +- 16 files changed, 308 insertions(+), 148 deletions(-) rename src/Coordination/{KeeperStorageDispatcher.cpp => KeeperDispatcher.cpp} (86%) rename src/Coordination/{KeeperStorageDispatcher.h => KeeperDispatcher.h} (80%) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 4d01a523853..fdeb61e3e01 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -359,7 +359,7 @@ int Keeper::main(const std::vector & /*args*/) auto servers = std::make_shared>(); /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. - global_context->initializeKeeperStorageDispatcher(); + global_context->initializeKeeperDispatcher(); for (const auto & listen_host : listen_hosts) { /// TCP Keeper @@ -428,7 +428,7 @@ int Keeper::main(const std::vector & /*args*/) else LOG_INFO(log, "Closed connections to Keeper."); - global_context->shutdownKeeperStorageDispatcher(); + global_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers server_pool.joinAll(); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fe7f011d4d0..14f5a97eec4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -994,7 +994,7 @@ if (ThreadFuzzer::instance().isEffective()) { #if USE_NURAFT /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. - global_context->initializeKeeperStorageDispatcher(); + global_context->initializeKeeperDispatcher(); for (const auto & listen_host : listen_hosts) { /// TCP Keeper @@ -1077,7 +1077,7 @@ if (ThreadFuzzer::instance().isEffective()) else LOG_INFO(log, "Closed connections to servers for tables."); - global_context->shutdownKeeperStorageDispatcher(); + global_context->shutdownKeeperDispatcher(); } /// Wait server pool to avoid use-after-free of destroyed context in the handlers diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 6ec9b17d0a7..1bc33a5f386 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -165,10 +165,11 @@ public: while (!read_buf.eof()) { result.last_position = read_buf.count(); + /// Read checksum Checksum record_checksum; readIntBinary(record_checksum, read_buf); - /// Initialization is required, otherwise checksums may fail + /// Read header ChangelogRecord record; readIntBinary(record.header.version, read_buf); readIntBinary(record.header.index, read_buf); @@ -179,6 +180,7 @@ public: if (record.header.version > CURRENT_CHANGELOG_VERSION) throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported changelog version {} on path {}", record.header.version, filepath); + /// Read data if (record.header.blob_size != 0) { auto buffer = nuraft::buffer::alloc(record.header.blob_size); @@ -189,11 +191,13 @@ public: else record.blob = nullptr; + /// Check changelog integrity if (previous_index != 0 && previous_index + 1 != record.header.index) throw Exception(ErrorCodes::CORRUPTED_DATA, "Previous log entry {}, next log entry {}, seems like some entries skipped", previous_index, record.header.index); previous_index = record.header.index; + /// Compare checksums Checksum checksum = computeRecordChecksum(record); if (checksum != record_checksum) { @@ -202,22 +206,25 @@ public: filepath, record.header.version, record.header.index, record.header.blob_size); } + /// Check for duplicated changelog ids if (logs.count(record.header.index) != 0) throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filepath); result.entries_read += 1; + /// Read but skip this entry because our state is already more fresh if (record.header.index < start_log_index) - { continue; - } + /// Create log entry for readed data auto log_entry = nuraft::cs_new(record.header.term, record.blob, record.header.value_type); if (result.first_read_index == 0) result.first_read_index = record.header.index; + /// Put it into in memory structure logs.emplace(record.header.index, log_entry); index_to_offset[record.header.index] = result.last_position; + if (result.entries_read % 50000 == 0) LOG_TRACE(log, "Reading changelog from path {}, entries {}", filepath, result.entries_read); } @@ -235,6 +242,7 @@ public: result.error = true; tryLogCurrentException(log); } + LOG_TRACE(log, "Totally read from changelog {} {} entries", filepath, result.entries_read); return result; @@ -255,6 +263,7 @@ Changelog::Changelog( , force_sync(force_sync_) , log(log_) { + /// Load all files in changelog directory namespace fs = std::filesystem; if (!fs::exists(changelogs_dir)) fs::create_directories(changelogs_dir); @@ -264,23 +273,35 @@ Changelog::Changelog( auto file_description = getChangelogFileDescription(p.path()); existing_changelogs[file_description.from_log_index] = file_description; } + + if (existing_changelogs.empty()) + LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) { uint64_t total_read = 0; + + /// Amount of entries in last log index uint64_t entries_in_last = 0; - uint64_t incomplete_log_index = 0; + + /// Log idx of the first incomplete log (key in existing_changelogs) + uint64_t first_incomplete_log_start_index = 0; + ChangelogReadResult result{}; + /// First log index which was read from all changelogs uint64_t first_read_index = 0; uint64_t start_to_read_from = last_commited_log_index; + if (start_to_read_from > logs_to_keep) start_to_read_from -= logs_to_keep; else start_to_read_from = 1; + /// At least we read something bool started = false; + for (const auto & [changelog_start_index, changelog_description] : existing_changelogs) { entries_in_last = changelog_description.to_log_index - changelog_description.from_log_index + 1; @@ -292,7 +313,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin if (changelog_description.from_log_index > last_commited_log_index && (changelog_description.from_log_index - last_commited_log_index) > 1) { LOG_ERROR(log, "Some records was lost, last committed log index {}, smallest available log index on disk {}. Hopefully will receive missing records from leader.", last_commited_log_index, changelog_description.from_log_index); - incomplete_log_index = changelog_start_index; + first_incomplete_log_start_index = changelog_start_index; break; } else if (changelog_description.from_log_index > start_to_read_from) @@ -311,7 +332,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin /// May happen after truncate, crash or simply unfinished log if (result.entries_read < entries_in_last) { - incomplete_log_index = changelog_start_index; + first_incomplete_log_start_index = changelog_start_index; break; } } @@ -322,11 +343,13 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin else start_index = last_commited_log_index; - if (incomplete_log_index != 0) + /// Found some broken or non finished logs + /// We have to remove broken data and continue to write into incomplete log. + if (first_incomplete_log_start_index != 0) { auto start_remove_from = existing_changelogs.begin(); if (started) - start_remove_from = existing_changelogs.upper_bound(incomplete_log_index); + start_remove_from = existing_changelogs.upper_bound(first_incomplete_log_start_index); /// All subsequent logs shouldn't exist. But they may exist if we crashed after writeAt started. Remove them. for (auto itr = start_remove_from; itr != existing_changelogs.end();) @@ -363,6 +386,7 @@ void Changelog::rotate(uint64_t new_start_log_index) /// Flush previous log flush(); + /// Start new one ChangelogFileDescription new_description; new_description.prefix = DEFAULT_PREFIX; new_description.from_log_index = new_start_log_index; @@ -378,7 +402,7 @@ void Changelog::rotate(uint64_t new_start_log_index) ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_entry) { ChangelogRecord record; - record.header.version = ChangelogVersion::V0; + record.header.version = ChangelogVersion::V1; record.header.index = index; record.header.term = log_entry->get_term(); record.header.value_type = log_entry->get_val_type(); @@ -416,7 +440,9 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) if (index_to_start_pos.count(index) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index); + /// Complex case when we need to override data from already rotated log bool go_to_previous_file = index < current_writer->getStartIndex(); + if (go_to_previous_file) { auto index_changelog = existing_changelogs.lower_bound(index); @@ -450,6 +476,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) auto log_itr = logs.find(i); if (log_itr == logs.end()) break; + logs.erase(log_itr); index_to_start_pos.erase(i); entries_written--; @@ -467,7 +494,6 @@ void Changelog::compact(uint64_t up_to_log_index) /// Remove all completely outdated changelog files if (itr->second.to_log_index <= up_to_log_index) { - LOG_INFO(log, "Removing changelog {} because of compaction", itr->second.path); std::erase_if(index_to_start_pos, [right_index = itr->second.to_log_index] (const auto & item) { return item.first <= right_index; }); std::filesystem::remove(itr->second.path); @@ -482,6 +508,7 @@ void Changelog::compact(uint64_t up_to_log_index) LogEntryPtr Changelog::getLastEntry() const { + /// This entry treaded in special way by NuRaft static LogEntryPtr fake_entry = nuraft::cs_new(0, nuraft::buffer::alloc(sizeof(uint64_t))); uint64_t next_index = getNextEntryIndex() - 1; diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 893fe16abdf..88198ec398c 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -58,8 +58,8 @@ struct ChangelogFileDescription class ChangelogWriter; /// Simplest changelog with files rotation. -/// No compression, no metadata, just entries with headers one by one -/// Able to read broken files/entries and discard them. +/// No compression, no metadata, just entries with headers one by one. +/// Able to read broken files/entries and discard them. Not thread safe. class Changelog { diff --git a/src/Coordination/KeeperStorageDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp similarity index 86% rename from src/Coordination/KeeperStorageDispatcher.cpp rename to src/Coordination/KeeperDispatcher.cpp index 7c416b38d8b..26db925b4c5 100644 --- a/src/Coordination/KeeperStorageDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -9,19 +9,18 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int TIMEOUT_EXCEEDED; } -KeeperStorageDispatcher::KeeperStorageDispatcher() +KeeperDispatcher::KeeperDispatcher() : coordination_settings(std::make_shared()) , log(&Poco::Logger::get("KeeperDispatcher")) { } -void KeeperStorageDispatcher::requestThread() +void KeeperDispatcher::requestThread() { setThreadName("KeeperReqT"); @@ -133,7 +132,7 @@ void KeeperStorageDispatcher::requestThread() } } -void KeeperStorageDispatcher::responseThread() +void KeeperDispatcher::responseThread() { setThreadName("KeeperRspT"); while (!shutdown_called) @@ -159,7 +158,7 @@ void KeeperStorageDispatcher::responseThread() } } -void KeeperStorageDispatcher::snapshotThread() +void KeeperDispatcher::snapshotThread() { setThreadName("KeeperSnpT"); while (!shutdown_called) @@ -181,9 +180,11 @@ void KeeperStorageDispatcher::snapshotThread() } } -void KeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) +void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) { std::lock_guard lock(session_to_response_callback_mutex); + + /// Special new session response. if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::SessionID) { const Coordination::ZooKeeperSessionIDResponse & session_id_resp = dynamic_cast(*response); @@ -196,25 +197,28 @@ void KeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination callback(response); new_session_id_response_callback.erase(session_id_resp.internal_id); } - else + else /// Normal response, just write to client { - auto session_writer = session_to_response_callback.find(session_id); - if (session_writer == session_to_response_callback.end()) + auto session_response_callback = session_to_response_callback.find(session_id); + + /// Session was disconnected, just skip this response + if (session_response_callback == session_to_response_callback.end()) return; - session_writer->second(response); + session_response_callback->second(response); /// Session closed, no more writes if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::Close) { - session_to_response_callback.erase(session_writer); + session_to_response_callback.erase(session_response_callback); } } } -bool KeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) +bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) { { + /// If session was already disconnected than we will ignore requests std::lock_guard lock(session_to_response_callback_mutex); if (session_to_response_callback.count(session_id) == 0) return false; @@ -237,7 +241,7 @@ bool KeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr return true; } -void KeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper) +void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper) { LOG_DEBUG(log, "Initializing storage dispatcher"); int myid = config.getInt("keeper_server.server_id"); @@ -251,6 +255,7 @@ void KeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration server = std::make_unique( myid, coordination_settings, config, responses_queue, snapshots_queue, standalone_keeper); + try { LOG_DEBUG(log, "Waiting server to initialize"); @@ -266,13 +271,13 @@ void KeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration throw; } - + /// Start it after keeper server start session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); }); LOG_DEBUG(log, "Dispatcher initialized"); } -void KeeperStorageDispatcher::shutdown() +void KeeperDispatcher::shutdown() { try { @@ -306,6 +311,8 @@ void KeeperStorageDispatcher::shutdown() server->shutdown(); KeeperStorage::RequestForSession request_for_session; + + /// Set session expired for all pending requests while (requests_queue->tryPop(request_for_session)) { if (request_for_session.request) @@ -320,6 +327,7 @@ void KeeperStorageDispatcher::shutdown() } } + /// Clear all registered sessions std::lock_guard lock(session_to_response_callback_mutex); session_to_response_callback.clear(); } @@ -331,19 +339,19 @@ void KeeperStorageDispatcher::shutdown() LOG_DEBUG(log, "Dispatcher shut down"); } -KeeperStorageDispatcher::~KeeperStorageDispatcher() +KeeperDispatcher::~KeeperDispatcher() { shutdown(); } -void KeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback) +void KeeperDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback) { std::lock_guard lock(session_to_response_callback_mutex); if (!session_to_response_callback.try_emplace(session_id, callback).second) throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id); } -void KeeperStorageDispatcher::sessionCleanerTask() +void KeeperDispatcher::sessionCleanerTask() { while (true) { @@ -352,12 +360,16 @@ void KeeperStorageDispatcher::sessionCleanerTask() try { + /// Only leader node must check dead sessions if (isLeader()) { auto dead_sessions = server->getDeadSessions(); + for (int64_t dead_session : dead_sessions) { LOG_INFO(log, "Found dead session {}, will try to close it", dead_session); + + /// Close session == send close request to raft server Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); request->xid = Coordination::CLOSE_XID; KeeperStorage::RequestForSession request_info; @@ -367,6 +379,8 @@ void KeeperStorageDispatcher::sessionCleanerTask() std::lock_guard lock(push_request_mutex); requests_queue->push(std::move(request_info)); } + + /// Remove session from registered sessions finishSession(dead_session); LOG_INFO(log, "Dead session close request pushed"); } @@ -381,7 +395,7 @@ void KeeperStorageDispatcher::sessionCleanerTask() } } -void KeeperStorageDispatcher::finishSession(int64_t session_id) +void KeeperDispatcher::finishSession(int64_t session_id) { std::lock_guard lock(session_to_response_callback_mutex); auto session_it = session_to_response_callback.find(session_id); @@ -389,7 +403,7 @@ void KeeperStorageDispatcher::finishSession(int64_t session_id) session_to_response_callback.erase(session_it); } -void KeeperStorageDispatcher::addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error) +void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error) { for (const auto & [session_id, request] : requests_for_sessions) { @@ -402,7 +416,7 @@ void KeeperStorageDispatcher::addErrorResponses(const KeeperStorage::RequestsFor } } -void KeeperStorageDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions) +void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions) { if (!result->has_result()) result->get(); @@ -417,10 +431,14 @@ void KeeperStorageDispatcher::forceWaitAndProcessResult(RaftAppendResult & resul requests_for_sessions.clear(); } -int64_t KeeperStorageDispatcher::getSessionID(int64_t session_timeout_ms) +int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) { + /// New session id allocation is a special request, because we cannot process it in normal + /// way: get request -> put to raft -> set response for registered callback. KeeperStorage::RequestForSession request_info; std::shared_ptr request = std::make_shared(); + /// Internal session id. It's a temporary number which is unique for each client on this server + /// but can be same on different servers. request->internal_id = internal_session_id_counter.fetch_add(1); request->session_timeout_ms = session_timeout_ms; request->server_id = server->getServerID(); @@ -430,6 +448,7 @@ int64_t KeeperStorageDispatcher::getSessionID(int64_t session_timeout_ms) auto promise = std::make_shared>(); auto future = promise->get_future(); + { std::lock_guard lock(session_to_response_callback_mutex); new_session_id_response_callback[request->internal_id] = [promise, internal_id = request->internal_id] (const Coordination::ZooKeeperResponsePtr & response) @@ -452,6 +471,7 @@ int64_t KeeperStorageDispatcher::getSessionID(int64_t session_timeout_ms) }; } + /// Push new session request to queue { std::lock_guard lock(push_request_mutex); if (!requests_queue->tryPush(std::move(request_info), session_timeout_ms)) @@ -461,6 +481,8 @@ int64_t KeeperStorageDispatcher::getSessionID(int64_t session_timeout_ms) if (future.wait_for(std::chrono::milliseconds(session_timeout_ms)) != std::future_status::ready) throw Exception("Cannot receive session id within session timeout", ErrorCodes::TIMEOUT_EXCEEDED); + /// Forcefully wait for request execution because we cannot process any other + /// requests for this client until it get new session id. return future.get(); } diff --git a/src/Coordination/KeeperStorageDispatcher.h b/src/Coordination/KeeperDispatcher.h similarity index 80% rename from src/Coordination/KeeperStorageDispatcher.h rename to src/Coordination/KeeperDispatcher.h index cc95de04ce9..a20603f12c5 100644 --- a/src/Coordination/KeeperStorageDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -22,7 +22,9 @@ namespace DB using ZooKeeperResponseCallback = std::function; -class KeeperStorageDispatcher +/// Highlevel wrapper for ClickHouse Keeper. +/// Process user requests via consensus and return responses. +class KeeperDispatcher { private: @@ -45,6 +47,7 @@ private: /// (get, set, list, etc.). Dispatcher determines callback for each response /// using session id from this map. SessionToResponseCallback session_to_response_callback; + /// But when client connects to the server for the first time it doesn't /// have session_id. It request it from server. We give temporary /// internal id for such requests just to much client with its response. @@ -60,7 +63,7 @@ private: /// Dumping new snapshots to disk ThreadFromGlobalPool snapshot_thread; - /// RAFT wrapper. Most important class. + /// RAFT wrapper. std::unique_ptr server; Poco::Logger * log; @@ -69,10 +72,15 @@ private: std::atomic internal_session_id_counter{0}; private: + /// Thread put requests to raft void requestThread(); + /// Thread put responses for subscribed sessions void responseThread(); + /// Thread clean disconnected sessions from memory void sessionCleanerTask(); + /// Thread create snapshots in the background void snapshotThread(); + void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); /// Add error responses for requests to responses queue. @@ -84,16 +92,23 @@ private: void forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); public: - KeeperStorageDispatcher(); + /// Just allocate some objects, real initialization is done by `intialize method` + KeeperDispatcher(); + /// Call shutdown + ~KeeperDispatcher(); + + /// Initialization from config. + /// standalone_keeper -- we are standalone keeper application (not inside clickhouse server) void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper); + /// Shutdown internal keeper parts (server, state machine, log storage, etc) void shutdown(); - ~KeeperStorageDispatcher(); - + /// Put request to ClickHouse Keeper bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); + /// Are we leader bool isLeader() const { return server->isLeader(); @@ -104,9 +119,12 @@ public: return server->isLeaderAlive(); } + /// Get new session ID int64_t getSessionID(int64_t session_timeout_ms); + /// Register session and subscribe for responses with callback void registerSession(int64_t session_id, ZooKeeperResponseCallback callback); + /// Call if we don't need any responses for this session no more (session was expired) void finishSession(int64_t session_id); }; diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 01315e6e879..d8ac8330c05 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -9,39 +9,53 @@ namespace DB { +/// Wrapper around Changelog class. Implements RAFT log storage. class KeeperLogStore : public nuraft::log_store { public: KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_); + /// Read log storage from filesystem starting from last_commited_log_index void init(uint64_t last_commited_log_index, uint64_t logs_to_keep); uint64_t start_index() const override; uint64_t next_slot() const override; + /// return last entry from log nuraft::ptr last_entry() const override; + /// Append new entry to log uint64_t append(nuraft::ptr & entry) override; + /// Remove all entries starting from index and write entry into index position void write_at(uint64_t index, nuraft::ptr & entry) override; + /// Return entries between [start, end) nuraft::ptr>> log_entries(uint64_t start, uint64_t end) override; + /// Return entry at index nuraft::ptr entry_at(uint64_t index) override; + /// Term if the index uint64_t term_at(uint64_t index) override; + /// Serialize entries in interval [index, index + cnt) nuraft::ptr pack(uint64_t index, int32_t cnt) override; + /// Apply serialized entries starting from index void apply_pack(uint64_t index, nuraft::buffer & pack) override; + /// Entries from last_log_index can be removed from memory and from disk bool compact(uint64_t last_log_index) override; + /// Call fsync to the stored data bool flush() override; + /// Current log storage size uint64_t size() const; + /// Flush batch of appended entries void end_of_append_batch(uint64_t start_index, uint64_t count) override; private: diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 282a7b48dfb..7c40047ce88 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -38,6 +38,8 @@ private: Poco::Logger * log; + /// Callback func which is called by NuRaft on all internal events. + /// Used to determine the moment when raft is ready to server new requests nuraft::cb_func::ReturnCode callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param); /// Almost copy-paste from nuraft::launcher, but with separated server init and start @@ -57,18 +59,25 @@ public: SnapshotsQueue & snapshots_queue_, bool standalone_keeper); + /// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings. void startup(); + /// Put local read request and execute in state machine directly and response into + /// responses queue void putLocalReadRequest(const KeeperStorage::RequestForSession & request); + /// Put batch of requests into Raft and get result of put. Responses will be set separately into + /// responses_queue. RaftAppendResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests); + /// Return set of the non-active sessions std::unordered_set getDeadSessions(); bool isLeader() const; bool isLeaderAlive() const; + /// Wait server intitialization (see callbackFunc) void waitInit(); void shutdown(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a76b86a8171..2e5e7214e3e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -14,29 +14,32 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) +namespace { - ReadBufferFromNuraftBuffer buffer(data); - KeeperStorage::RequestForSession request_for_session; - readIntBinary(request_for_session.session_id, buffer); + KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) + { + ReadBufferFromNuraftBuffer buffer(data); + KeeperStorage::RequestForSession request_for_session; + readIntBinary(request_for_session.session_id, buffer); - int32_t length; - Coordination::read(length, buffer); + int32_t length; + Coordination::read(length, buffer); - int32_t xid; - Coordination::read(xid, buffer); + int32_t xid; + Coordination::read(xid, buffer); - Coordination::OpNum opnum; + Coordination::OpNum opnum; - Coordination::read(opnum, buffer); + Coordination::read(opnum, buffer); - request_for_session.request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); - request_for_session.request->xid = xid; - request_for_session.request->readImpl(buffer); - return request_for_session; + request_for_session.request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); + request_for_session.request->xid = xid; + request_for_session.request->readImpl(buffer); + return request_for_session; + } } - KeeperStateMachine::KeeperStateMachine( +KeeperStateMachine::KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, @@ -58,6 +61,7 @@ void KeeperStateMachine::init() LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots()); bool loaded = false; bool has_snapshots = snapshot_manager.totalSnapshots() != 0; + /// Deserialize latest snapshot from disk while (snapshot_manager.totalSnapshots() != 0) { uint64_t latest_log_index = snapshot_manager.getLatestSnapshotIndex(); @@ -97,6 +101,7 @@ void KeeperStateMachine::init() nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data); + /// Special processing of session_id request if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) { const Coordination::ZooKeeperSessionIDRequest & session_id_request = dynamic_cast(*request_for_session.request); @@ -136,7 +141,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx()); nuraft::ptr latest_snapshot_ptr; - { + { /// save snapshot into memory std::lock_guard lock(snapshots_lock); if (s.get_last_log_idx() != latest_snapshot_meta->get_last_log_idx()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Required to apply snapshot with last log index {}, but our last log index is {}", @@ -144,10 +149,11 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) latest_snapshot_ptr = latest_snapshot_buf; } - { + { /// deserialize and apply snapshot to storage std::lock_guard lock(storage_lock); std::tie(latest_snapshot_meta, storage) = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); } + last_committed_idx = s.get_last_log_idx(); return true; } @@ -168,18 +174,19 @@ void KeeperStateMachine::create_snapshot( nuraft::ptr snp_buf = s.serialize(); auto snapshot_meta_copy = nuraft::snapshot::deserialize(*snp_buf); CreateSnapshotTask snapshot_task; - { + { /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking. std::lock_guard lock(storage_lock); snapshot_task.snapshot = std::make_shared(storage.get(), snapshot_meta_copy); } + /// create snapshot task for background execution (in snapshot thread) snapshot_task.create_snapshot = [this, when_done] (KeeperStorageSnapshotPtr && snapshot) { nuraft::ptr exception(nullptr); bool ret = true; try { - { + { /// Read storage data without locks and create snapshot std::lock_guard lock(snapshots_lock); auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot); auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); @@ -192,6 +199,7 @@ void KeeperStateMachine::create_snapshot( { /// Must do it with lock (clearing elements from list) std::lock_guard lock(storage_lock); + /// Turn off "snapshot mode" and clear outdate part of storage state storage->clearGarbageAfterSnapshot(); /// Destroy snapshot with lock snapshot.reset(); @@ -209,7 +217,9 @@ void KeeperStateMachine::create_snapshot( when_done(ret, exception); }; + LOG_DEBUG(log, "In memory snapshot {} created, queueing task to flash to disk", s.get_last_log_idx()); + /// Flush snapshot to disk in a separate thread. snapshots_queue.push(std::move(snapshot_task)); } @@ -224,7 +234,7 @@ void KeeperStateMachine::save_logical_snp_obj( nuraft::ptr cloned_buffer; nuraft::ptr cloned_meta; - if (obj_id == 0) + if (obj_id == 0) /// Fake snapshot required by NuRaft at startup { std::lock_guard lock(storage_lock); KeeperStorageSnapshot snapshot(storage.get(), s.get_last_log_idx()); @@ -232,15 +242,18 @@ void KeeperStateMachine::save_logical_snp_obj( } else { + /// copy snapshot into memory cloned_buffer = nuraft::buffer::clone(data); } + /// copy snapshot meta into memory nuraft::ptr snp_buf = s.serialize(); cloned_meta = nuraft::snapshot::deserialize(*snp_buf); try { std::lock_guard lock(snapshots_lock); + /// Serialize snapshot to disk and switch in memory pointers. auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(*cloned_buffer, s.get_last_log_idx()); latest_snapshot_buf = cloned_buffer; latest_snapshot_meta = cloned_meta; @@ -262,7 +275,7 @@ int KeeperStateMachine::read_logical_snp_obj( { LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); - if (obj_id == 0) + if (obj_id == 0) /// Fake snapshot required by NuRaft at startup { data_out = nuraft::buffer::alloc(sizeof(int32_t)); nuraft::buffer_serializer bs(data_out); @@ -272,6 +285,8 @@ int KeeperStateMachine::read_logical_snp_obj( else { std::lock_guard lock(snapshots_lock); + /// Our snapshot is not equal to required. Maybe we still creating it in the background. + /// Let's wait and NuRaft will retry this call. if (s.get_last_log_idx() != latest_snapshot_meta->get_last_log_idx()) { LOG_WARNING(log, "Required to apply snapshot with last log index {}, but our last log index is {}. Will ignore this one and retry", @@ -281,11 +296,13 @@ int KeeperStateMachine::read_logical_snp_obj( data_out = nuraft::buffer::clone(*latest_snapshot_buf); is_last_obj = true; } + return 1; } void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session) { + /// Pure local request, just process it with storage KeeperStorage::ResponsesForSessions responses; { std::lock_guard lock(storage_lock); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index fb46f507baf..06be270b66e 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -13,6 +13,8 @@ namespace DB using ResponsesQueue = ThreadSafeQueue; using SnapshotsQueue = ConcurrentBoundedQueue; +/// ClickHouse Keeper state machine. Wrapper for KeeperStorage. +/// Responsible for entries commit, snapshots creation and so on. class KeeperStateMachine : public nuraft::state_machine { public: @@ -21,24 +23,30 @@ public: const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_, const std::string & superdigest_ = ""); + /// Read state from the latest snapshot void init(); + /// Currently not supported nuraft::ptr pre_commit(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; + /// Currently not supported void rollback(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override {} uint64_t last_commit_index() override { return last_committed_idx; } + /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. bool apply_snapshot(nuraft::snapshot & s) override; nuraft::ptr last_snapshot() override; + /// Create new snapshot from current state. void create_snapshot( nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) override; + /// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot. void save_logical_snp_obj( nuraft::snapshot & s, uint64_t & obj_id, @@ -46,6 +54,8 @@ public: bool is_first_obj, bool is_last_obj) override; + /// Better name is `serialize snapshot` -- save existing snapshot (created by create_snapshot) into + /// in-memory buffer data_out. int read_logical_snp_obj( nuraft::snapshot & s, void* & user_snp_ctx, @@ -58,6 +68,7 @@ public: return *storage; } + /// Process local read request void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); std::unordered_set getDeadSessions(); @@ -66,18 +77,25 @@ public: private: + /// In our state machine we always have a single snapshot which is stored + /// in memory in compressed (serialized) format. SnapshotMetadataPtr latest_snapshot_meta = nullptr; nuraft::ptr latest_snapshot_buf = nullptr; CoordinationSettingsPtr coordination_settings; + /// Main state machine logic KeeperStoragePtr storage; + /// Save/Load and Serialize/Deserialize logic for snapshots. KeeperSnapshotManager snapshot_manager; + /// Put processed responses into this queue ResponsesQueue & responses_queue; + /// Snapshots to create by snapshot thread SnapshotsQueue & snapshots_queue; + /// Mutex for snapshots std::mutex snapshots_lock; @@ -88,6 +106,7 @@ private: std::atomic last_committed_idx; Poco::Logger * log; + /// Special part of ACL system -- superdigest specified in server config. const std::string superdigest; }; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 320754c7d31..9df8329c5a9 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -177,32 +177,32 @@ KeeperStorage::KeeperStorage(int64_t tick_time_ms, const String & superdigest_) using Undo = std::function; -struct KeeperStorageRequest +struct KeeperStorageRequestProcessor { Coordination::ZooKeeperRequestPtr zk_request; - explicit KeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) + explicit KeeperStorageRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) {} virtual std::pair process(KeeperStorage & storage, int64_t zxid, int64_t session_id) const = 0; virtual KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & /*watches*/, KeeperStorage::Watches & /*list_watches*/) const { return {}; } virtual bool checkAuth(KeeperStorage & /*storage*/, int64_t /*session_id*/) const { return true; } - virtual ~KeeperStorageRequest() = default; + virtual ~KeeperStorageRequestProcessor() = default; }; -struct KeeperStorageHeartbeatRequest final : public KeeperStorageRequest +struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & /* storage */, int64_t /* zxid */, int64_t /* session_id */) const override { return {zk_request->makeResponse(), {}}; } }; -struct KeeperStorageSyncRequest final : public KeeperStorageRequest +struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & /* storage */, int64_t /* zxid */, int64_t /* session_id */) const override { auto response = zk_request->makeResponse(); @@ -212,9 +212,9 @@ struct KeeperStorageSyncRequest final : public KeeperStorageRequest } }; -struct KeeperStorageCreateRequest final : public KeeperStorageRequest +struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override { @@ -363,7 +363,7 @@ struct KeeperStorageCreateRequest final : public KeeperStorageRequest } }; -struct KeeperStorageGetRequest final : public KeeperStorageRequest +struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override @@ -381,7 +381,7 @@ struct KeeperStorageGetRequest final : public KeeperStorageRequest return checkACL(Coordination::ACL::Read, node_acls, session_auths); } - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t /* zxid */, int64_t /* session_id */) const override { auto & container = storage.container; @@ -423,7 +423,7 @@ namespace } } -struct KeeperStorageRemoveRequest final : public KeeperStorageRequest +struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override { @@ -440,7 +440,7 @@ struct KeeperStorageRemoveRequest final : public KeeperStorageRequest return checkACL(Coordination::ACL::Delete, node_acls, session_auths); } - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/) const override { auto & container = storage.container; @@ -520,9 +520,9 @@ struct KeeperStorageRemoveRequest final : public KeeperStorageRequest } }; -struct KeeperStorageExistsRequest final : public KeeperStorageRequest +struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t /* session_id */) const override { auto & container = storage.container; @@ -546,7 +546,7 @@ struct KeeperStorageExistsRequest final : public KeeperStorageRequest } }; -struct KeeperStorageSetRequest final : public KeeperStorageRequest +struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override { @@ -563,7 +563,7 @@ struct KeeperStorageSetRequest final : public KeeperStorageRequest return checkACL(Coordination::ACL::Write, node_acls, session_auths); } - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t zxid, int64_t /* session_id */) const override { auto & container = storage.container; @@ -624,7 +624,7 @@ struct KeeperStorageSetRequest final : public KeeperStorageRequest } }; -struct KeeperStorageListRequest final : public KeeperStorageRequest +struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override { @@ -641,7 +641,7 @@ struct KeeperStorageListRequest final : public KeeperStorageRequest return checkACL(Coordination::ACL::Read, node_acls, session_auths); } - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t /*session_id*/) const override { auto & container = storage.container; @@ -669,7 +669,7 @@ struct KeeperStorageListRequest final : public KeeperStorageRequest } }; -struct KeeperStorageCheckRequest final : public KeeperStorageRequest +struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override { @@ -686,7 +686,7 @@ struct KeeperStorageCheckRequest final : public KeeperStorageRequest return checkACL(Coordination::ACL::Read, node_acls, session_auths); } - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t /*session_id*/) const override { auto & container = storage.container; @@ -713,7 +713,7 @@ struct KeeperStorageCheckRequest final : public KeeperStorageRequest }; -struct KeeperStorageSetACLRequest final : public KeeperStorageRequest +struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override { @@ -730,7 +730,7 @@ struct KeeperStorageSetACLRequest final : public KeeperStorageRequest return checkACL(Coordination::ACL::Admin, node_acls, session_auths); } - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t session_id) const override { @@ -777,7 +777,7 @@ struct KeeperStorageSetACLRequest final : public KeeperStorageRequest } }; -struct KeeperStorageGetACLRequest final : public KeeperStorageRequest +struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override { @@ -794,7 +794,7 @@ struct KeeperStorageGetACLRequest final : public KeeperStorageRequest /// LOL, GetACL require more permissions, then SetACL... return checkACL(Coordination::ACL::Admin | Coordination::ACL::Read, node_acls, session_auths); } - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t /*session_id*/) const override { @@ -817,7 +817,7 @@ struct KeeperStorageGetACLRequest final : public KeeperStorageRequest } }; -struct KeeperStorageMultiRequest final : public KeeperStorageRequest +struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor { bool checkAuth(KeeperStorage & storage, int64_t session_id) const override { @@ -827,9 +827,9 @@ struct KeeperStorageMultiRequest final : public KeeperStorageRequest return true; } - std::vector concrete_requests; - explicit KeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequest(zk_request_) + std::vector concrete_requests; + explicit KeeperStorageMultiRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) + : KeeperStorageRequestProcessor(zk_request_) { Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*zk_request); concrete_requests.reserve(request.requests.size()); @@ -839,19 +839,19 @@ struct KeeperStorageMultiRequest final : public KeeperStorageRequest auto sub_zk_request = std::dynamic_pointer_cast(sub_request); if (sub_zk_request->getOpNum() == Coordination::OpNum::Create) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else if (sub_zk_request->getOpNum() == Coordination::OpNum::Remove) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else if (sub_zk_request->getOpNum() == Coordination::OpNum::Set) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else if (sub_zk_request->getOpNum() == Coordination::OpNum::Check) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", sub_zk_request->getOpNum()); @@ -923,18 +923,18 @@ struct KeeperStorageMultiRequest final : public KeeperStorageRequest } }; -struct KeeperStorageCloseRequest final : public KeeperStorageRequest +struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage &, int64_t, int64_t) const override { throw DB::Exception("Called process on close request", ErrorCodes::LOGICAL_ERROR); } }; -struct KeeperStorageAuthRequest final : public KeeperStorageRequest +struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequest::KeeperStorageRequest; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t session_id) const override { Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*zk_request); @@ -988,20 +988,20 @@ void KeeperStorage::finalize() } -class KeeperWrapperFactory final : private boost::noncopyable +class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable { public: - using Creator = std::function; + using Creator = std::function; using OpNumToRequest = std::unordered_map; - static KeeperWrapperFactory & instance() + static KeeperStorageRequestProcessorsFactory & instance() { - static KeeperWrapperFactory factory; + static KeeperStorageRequestProcessorsFactory factory; return factory; } - KeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const + KeeperStorageRequestProcessorPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const { auto it = op_num_to_request.find(zk_request->getOpNum()); if (it == op_num_to_request.end()) @@ -1018,33 +1018,33 @@ public: private: OpNumToRequest op_num_to_request; - KeeperWrapperFactory(); + KeeperStorageRequestProcessorsFactory(); }; template -void registerKeeperRequestWrapper(KeeperWrapperFactory & factory) +void registerKeeperRequestProcessor(KeeperStorageRequestProcessorsFactory & factory) { factory.registerRequest(num, [] (const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); } -KeeperWrapperFactory::KeeperWrapperFactory() +KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() { - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); - registerKeeperRequestWrapper(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); } @@ -1059,7 +1059,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina } session_expiry_queue.update(session_id, session_and_timeout[session_id]); - if (zk_request->getOpNum() == Coordination::OpNum::Close) + + if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { auto it = ephemerals.find(session_id); if (it != ephemerals.end()) @@ -1092,21 +1093,21 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina session_and_timeout.erase(session_id); results.push_back(ResponseForSession{session_id, response}); } - else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) + else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { - KeeperStorageRequestPtr storage_request = KeeperWrapperFactory::instance().get(zk_request); + KeeperStorageRequestProcessorPtr storage_request = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); auto [response, _] = storage_request->process(*this, zxid, session_id); response->xid = zk_request->xid; response->zxid = getZXID(); results.push_back(ResponseForSession{session_id, response}); } - else + else /// normal requests proccession { - KeeperStorageRequestPtr storage_request = KeeperWrapperFactory::instance().get(zk_request); + KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); Coordination::ZooKeeperResponsePtr response; - if (check_acl && !storage_request->checkAuth(*this, session_id)) + if (check_acl && !request_processor->checkAuth(*this, session_id)) { response = zk_request->makeResponse(); /// Original ZooKeeper always throws no auth, even when user provided some credentials @@ -1114,9 +1115,10 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina } else { - std::tie(response, std::ignore) = storage_request->process(*this, zxid, session_id); + std::tie(response, std::ignore) = request_processor->process(*this, zxid, session_id); } + /// Watches for this requests are added to the watches lists if (zk_request->has_watch) { if (response->error == Coordination::Error::ZOK) @@ -1135,9 +1137,10 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina } } + /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - auto watch_responses = storage_request->processWatches(watches, list_watches); + auto watch_responses = request_processor->processWatches(watches, list_watches); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -1153,11 +1156,13 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina void KeeperStorage::clearDeadWatches(int64_t session_id) { + /// Clear all wathces for this session auto watches_it = sessions_and_watchers.find(session_id); if (watches_it != sessions_and_watchers.end()) { for (const auto & watch_path : watches_it->second) { + /// Maybe it's a normal watch auto watch = watches.find(watch_path); if (watch != watches.end()) { @@ -1173,6 +1178,7 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) watches.erase(watch); } + /// Maybe it's a list watch auto list_watch = list_watches.find(watch_path); if (list_watch != list_watches.end()) { @@ -1188,6 +1194,7 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) list_watches.erase(list_watch); } } + sessions_and_watchers.erase(watches_it); } } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index e3cb0f59fdc..1e925a0634e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -15,14 +15,17 @@ namespace DB { using namespace DB; -struct KeeperStorageRequest; -using KeeperStorageRequestPtr = std::shared_ptr; +struct KeeperStorageRequestProcessor; +using KeeperStorageRequestProcessorPtr = std::shared_ptr; using ResponseCallback = std::function; using ChildrenSet = std::unordered_set; using SessionAndTimeout = std::unordered_map; struct KeeperStorageSnapshot; +/// Keeper state machine almost equal to the ZooKeeper's state machine. +/// Implements all logic of operations, data changes, sessions allocation. +/// In-memory and not thread safe. class KeeperStorage { public: @@ -77,21 +80,34 @@ public: using Watches = std::map; + /// Main hashtable with nodes. Contain all information about data. + /// All other structures expect session_and_timeout can be restored from + /// container. Container container; + + /// Mapping session_id -> set of ephemeral nodes paths Ephemerals ephemerals; + /// Mapping sessuib_id -> set of watched nodes paths SessionAndWatcher sessions_and_watchers; + /// Expiration queue for session, allows to get dead sessions at some point of time SessionExpiryQueue session_expiry_queue; + /// All active sessions with timeout SessionAndTimeout session_and_timeout; + + /// ACLMap for more compact ACLs storage inside nodes. ACLMap acl_map; + /// Global id of all requests applied to storage int64_t zxid{0}; bool finalized{false}; + /// Currently active watches (node_path -> subscribed sessions) Watches watches; Watches list_watches; /// Watches for 'list' request (watches on children). void clearDeadWatches(int64_t session_id); + /// Get current zxid int64_t getZXID() const { return zxid; @@ -102,6 +118,7 @@ public: public: KeeperStorage(int64_t tick_time_ms, const String & superdigest_); + /// Allocate new session id with the specified timeouts int64_t getSessionID(int64_t session_timeout_ms) { auto result = session_id_counter++; @@ -110,21 +127,28 @@ public: return result; } + /// Add session id. Used when restoring KeeperStorage from snapshot. void addSessionID(int64_t session_id, int64_t session_timeout_ms) { session_and_timeout.emplace(session_id, session_timeout_ms); session_expiry_queue.update(session_id, session_timeout_ms); } + /// Process user request and return response. + /// check_acl = false only when converting data from ZooKeeper. ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id, std::optional new_last_zxid, bool check_acl = true); void finalize(); + /// Set of methods for creating snapshots + + /// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. void enableSnapshotMode() { container.enableSnapshotMode(); } + /// Turn off snapshot mode. void disableSnapshotMode() { container.disableSnapshotMode(); @@ -135,16 +159,19 @@ public: return container.begin(); } + /// Clear outdated data from internal container. void clearGarbageAfterSnapshot() { container.clearOutdatedNodes(); } + /// Get all active sessions const SessionAndTimeout & getActiveSessions() const { return session_and_timeout; } + /// Get all dead sessions std::unordered_set getDeadSessions() { return session_expiry_queue.getExpiredSessions(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f59d50dbdeb..d984a350c80 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include @@ -146,7 +146,7 @@ struct ContextSharedPart #if USE_NURAFT mutable std::mutex keeper_storage_dispatcher_mutex; - mutable std::shared_ptr keeper_storage_dispatcher; + mutable std::shared_ptr keeper_storage_dispatcher; #endif mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. @@ -1649,7 +1649,7 @@ void Context::setSystemZooKeeperLogAfterInitializationIfNeeded() zk.second->setZooKeeperLog(shared->system_logs->zookeeper_log); } -void Context::initializeKeeperStorageDispatcher() const +void Context::initializeKeeperDispatcher() const { #if USE_NURAFT std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); @@ -1660,14 +1660,14 @@ void Context::initializeKeeperStorageDispatcher() const const auto & config = getConfigRef(); if (config.has("keeper_server")) { - shared->keeper_storage_dispatcher = std::make_shared(); + shared->keeper_storage_dispatcher = std::make_shared(); shared->keeper_storage_dispatcher->initialize(config, getApplicationType() == ApplicationType::KEEPER); } #endif } #if USE_NURAFT -std::shared_ptr & Context::getKeeperStorageDispatcher() const +std::shared_ptr & Context::getKeeperDispatcher() const { std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); if (!shared->keeper_storage_dispatcher) @@ -1677,7 +1677,7 @@ std::shared_ptr & Context::getKeeperStorageDispatcher() } #endif -void Context::shutdownKeeperStorageDispatcher() const +void Context::shutdownKeeperDispatcher() const { #if USE_NURAFT std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 9527b87ed39..6af2c3c4d62 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -102,7 +102,7 @@ class StoragePolicySelector; using StoragePolicySelectorPtr = std::shared_ptr; struct PartUUIDs; using PartUUIDsPtr = std::shared_ptr; -class KeeperStorageDispatcher; +class KeeperDispatcher; class Session; class IOutputFormat; @@ -647,10 +647,10 @@ public: std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; #if USE_NURAFT - std::shared_ptr & getKeeperStorageDispatcher() const; + std::shared_ptr & getKeeperDispatcher() const; #endif - void initializeKeeperStorageDispatcher() const; - void shutdownKeeperStorageDispatcher() const; + void initializeKeeperDispatcher() const; + void shutdownKeeperDispatcher() const; /// Set auxiliary zookeepers configuration at server starting or configuration reloading. void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index df40a78749b..7ead4d0d419 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -194,7 +194,7 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , server(server_) , log(&Poco::Logger::get("NuKeeperTCPHandler")) , global_context(Context::createCopy(server.context())) - , keeper_dispatcher(global_context->getKeeperStorageDispatcher()) + , keeper_dispatcher(global_context->getKeeperDispatcher()) , operation_timeout(0, global_context->getConfigRef().getUInt("keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) , session_timeout(0, global_context->getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 76371ed1a0c..7abfb72c846 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include #include @@ -38,7 +38,7 @@ private: IServer & server; Poco::Logger * log; ContextPtr global_context; - std::shared_ptr keeper_dispatcher; + std::shared_ptr keeper_dispatcher; Poco::Timespan operation_timeout; Poco::Timespan session_timeout; int64_t session_id{-1}; From 0ec716292c7d832db55c7b8428bc1257556caa35 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 24 Aug 2021 12:29:42 +0000 Subject: [PATCH 061/127] Postgres ON CONFLICT --- contrib/libpqxx-cmake/CMakeLists.txt | 3 +- .../table-engines/integrations/postgresql.md | 1 + .../odbc-bridge/ODBCBlockOutputStream.cpp | 26 +--- programs/odbc-bridge/ODBCBlockOutputStream.h | 1 + src/Common/getInsertQuery.h | 26 ++++ src/Storages/StoragePostgreSQL.cpp | 115 +++++++++++++++--- src/Storages/StoragePostgreSQL.h | 4 +- .../TableFunctionPostgreSQL.cpp | 11 +- src/TableFunctions/TableFunctionPostgreSQL.h | 2 +- .../test_storage_postgresql/test.py | 28 ++++- 10 files changed, 165 insertions(+), 52 deletions(-) create mode 100644 src/Common/getInsertQuery.h diff --git a/contrib/libpqxx-cmake/CMakeLists.txt b/contrib/libpqxx-cmake/CMakeLists.txt index ae35538ccf4..65fa94cb3fd 100644 --- a/contrib/libpqxx-cmake/CMakeLists.txt +++ b/contrib/libpqxx-cmake/CMakeLists.txt @@ -22,6 +22,7 @@ set (SRCS "${LIBRARY_DIR}/src/transaction.cxx" "${LIBRARY_DIR}/src/transaction_base.cxx" "${LIBRARY_DIR}/src/row.cxx" + "${LIBRARY_DIR}/src/params.cxx" "${LIBRARY_DIR}/src/util.cxx" "${LIBRARY_DIR}/src/version.cxx" ) @@ -31,6 +32,7 @@ set (SRCS # conflicts with all includes of . set (HDRS "${LIBRARY_DIR}/include/pqxx/array.hxx" + "${LIBRARY_DIR}/include/pqxx/params.hxx" "${LIBRARY_DIR}/include/pqxx/binarystring.hxx" "${LIBRARY_DIR}/include/pqxx/composite.hxx" "${LIBRARY_DIR}/include/pqxx/connection.hxx" @@ -75,4 +77,3 @@ set(CM_CONFIG_PQ "${LIBRARY_DIR}/include/pqxx/config-internal-libpq.h") configure_file("${CM_CONFIG_H_IN}" "${CM_CONFIG_INT}" @ONLY) configure_file("${CM_CONFIG_H_IN}" "${CM_CONFIG_PUB}" @ONLY) configure_file("${CM_CONFIG_H_IN}" "${CM_CONFIG_PQ}" @ONLY) - diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 4c763153a36..53ab3f5088c 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -34,6 +34,7 @@ The table structure can differ from the original PostgreSQL table structure: - `user` — PostgreSQL user. - `password` — User password. - `schema` — Non-default table schema. Optional. +- `on conflict ...` — example: `ON CONFLICT DO NOTHING`. Optional. Note: adding this option will make insertion less efficient. ## Implementation Details {#implementation-details} diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index b4b514d1473..5b3bb0f42b9 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -5,40 +5,16 @@ #include #include #include -#include -#include -#include #include "getIdentifierQuote.h" #include #include #include +#include namespace DB { -namespace -{ - using ValueType = ExternalResultDescription::ValueType; - - std::string getInsertQuery(const std::string & db_name, const std::string & table_name, const ColumnsWithTypeAndName & columns, IdentifierQuotingStyle quoting) - { - ASTInsertQuery query; - query.table_id.database_name = db_name; - query.table_id.table_name = table_name; - query.columns = std::make_shared(','); - query.children.push_back(query.columns); - for (const auto & column : columns) - query.columns->children.emplace_back(std::make_shared(column.name)); - - WriteBufferFromOwnString buf; - IAST::FormatSettings settings(buf, true); - settings.always_quote_identifiers = true; - settings.identifier_quoting_style = quoting; - query.IAST::format(settings); - return buf.str(); - } -} ODBCBlockOutputStream::ODBCBlockOutputStream(nanodbc::ConnectionHolderPtr connection_holder_, const std::string & remote_database_name_, diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.h b/programs/odbc-bridge/ODBCBlockOutputStream.h index 1b42119e490..16a1602d3cd 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.h +++ b/programs/odbc-bridge/ODBCBlockOutputStream.h @@ -13,6 +13,7 @@ namespace DB class ODBCBlockOutputStream : public IBlockOutputStream { +using ValueType = ExternalResultDescription::ValueType; public: ODBCBlockOutputStream( diff --git a/src/Common/getInsertQuery.h b/src/Common/getInsertQuery.h new file mode 100644 index 00000000000..c0f77064e74 --- /dev/null +++ b/src/Common/getInsertQuery.h @@ -0,0 +1,26 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ +std::string getInsertQuery(const std::string & db_name, const std::string & table_name, const ColumnsWithTypeAndName & columns, IdentifierQuotingStyle quoting) +{ + ASTInsertQuery query; + query.table_id.database_name = db_name; + query.table_id.table_name = table_name; + query.columns = std::make_shared(','); + query.children.push_back(query.columns); + for (const auto & column : columns) + query.columns->children.emplace_back(std::make_shared(column.name)); + + WriteBufferFromOwnString buf; + IAST::FormatSettings settings(buf, true); + settings.always_quote_identifiers = true; + settings.identifier_quoting_style = quoting; + query.IAST::format(settings); + return buf.str(); +} +} diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 603a52b2801..f8dd1a0d64b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -29,6 +29,8 @@ #include #include #include +#include +#include namespace DB @@ -47,10 +49,12 @@ StoragePostgreSQL::StoragePostgreSQL( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - const String & remote_table_schema_) + const String & remote_table_schema_, + const String & on_conflict_) : IStorage(table_id_) , remote_table_name(remote_table_name_) , remote_table_schema(remote_table_schema_) + , on_conflict(on_conflict_) , pool(std::move(pool_)) { StorageInMemoryMetadata storage_metadata; @@ -94,17 +98,22 @@ Pipe StoragePostgreSQL::read( class PostgreSQLSink : public SinkToStorage { + +using Row = std::vector>; + public: explicit PostgreSQLSink( const StorageMetadataPtr & metadata_snapshot_, postgres::ConnectionHolderPtr connection_holder_, const String & remote_table_name_, - const String & remote_table_schema_) + const String & remote_table_schema_, + const String & on_conflict_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , metadata_snapshot(metadata_snapshot_) , connection_holder(std::move(connection_holder_)) , remote_table_name(remote_table_name_) , remote_table_schema(remote_table_schema_) + , on_conflict(on_conflict_) { } @@ -113,11 +122,21 @@ public: void consume(Chunk chunk) override { auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + if (!inserter) - inserter = std::make_unique(connection_holder->get(), - remote_table_schema.empty() ? pqxx::table_path({remote_table_name}) - : pqxx::table_path({remote_table_schema, remote_table_name}), - block.getNames()); + { + if (on_conflict.empty()) + { + inserter = std::make_unique(connection_holder->get(), + remote_table_schema.empty() ? pqxx::table_path({remote_table_name}) + : pqxx::table_path({remote_table_schema, remote_table_name}), block.getNames()); + } + else + { + inserter = std::make_unique(connection_holder->get(), remote_table_name, + remote_table_schema, block.getColumnsWithTypeAndName(), on_conflict); + } + } const auto columns = block.getColumns(); const size_t num_rows = block.rows(), num_cols = block.columns(); @@ -151,7 +170,7 @@ public: } } - inserter->stream.write_values(row); + inserter->insert(row); } } @@ -268,37 +287,92 @@ public: } private: - struct StreamTo + struct Inserter { + pqxx::connection & connection; pqxx::work tx; + + Inserter(pqxx::connection & connection_) + : connection(connection_) + , tx(connection) {} + + virtual ~Inserter() = default; + + virtual void insert(const Row & row) = 0; + virtual void complete() = 0; + }; + + struct StreamTo : Inserter + { Names columns; pqxx::stream_to stream; - StreamTo(pqxx::connection & connection, pqxx::table_path table_, Names columns_) - : tx(connection) + StreamTo(pqxx::connection & connection_, pqxx::table_path table_, Names columns_) + : Inserter(connection_) , columns(std::move(columns_)) , stream(pqxx::stream_to::raw_table(tx, connection.quote_table(table_), connection.quote_columns(columns))) { } - void complete() + void complete() override { stream.complete(); tx.commit(); } + + void insert(const Row & row) override + { + stream.write_values(row); + } + }; + + struct PreparedInsert : Inserter + { + PreparedInsert(pqxx::connection & connection_, const String & table, const String & schema, + const ColumnsWithTypeAndName & columns, const String & on_conflict_) + : Inserter(connection_) + { + WriteBufferFromOwnString buf; + buf << getInsertQuery(schema, table, columns, IdentifierQuotingStyle::DoubleQuotes); + buf << " ("; + for (size_t i = 1; i <= columns.size(); ++i) + { + if (i > 1) + buf << ", "; + buf << "$" << i; + } + buf << ") "; + buf << on_conflict_; + connection.prepare("insert", buf.str()); + } + + void complete() override + { + connection.unprepare("insert"); + tx.commit(); + } + + void insert(const Row & row) override + { + pqxx::params params; + params.reserve(row.size()); + params.append_multi(row); + tx.exec_prepared("insert", params); + } }; StorageMetadataPtr metadata_snapshot; postgres::ConnectionHolderPtr connection_holder; - const String remote_table_name, remote_table_schema; - std::unique_ptr inserter; + const String remote_db_name, remote_table_name, remote_table_schema, on_conflict; + + std::unique_ptr inserter; }; SinkToStoragePtr StoragePostgreSQL::write( const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) { - return std::make_shared(metadata_snapshot, pool->get(), remote_table_name, remote_table_schema); + return std::make_shared(metadata_snapshot, pool->get(), remote_table_name, remote_table_schema, on_conflict); } @@ -308,9 +382,9 @@ void registerStoragePostgreSQL(StorageFactory & factory) { ASTs & engine_args = args.engine_args; - if (engine_args.size() < 5 || engine_args.size() > 6) + if (engine_args.size() < 5 || engine_args.size() > 7) throw Exception("Storage PostgreSQL requires from 5 to 6 parameters: " - "PostgreSQL('host:port', 'database', 'table', 'username', 'password' [, 'schema']", + "PostgreSQL('host:port', 'database', 'table', 'username', 'password' [, 'schema', 'ON CONFLICT ...']", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (auto & engine_arg : engine_args) @@ -326,9 +400,11 @@ void registerStoragePostgreSQL(StorageFactory & factory) const String & username = engine_args[3]->as().value.safeGet(); const String & password = engine_args[4]->as().value.safeGet(); - String remote_table_schema; - if (engine_args.size() == 6) + String remote_table_schema, on_conflict; + if (engine_args.size() >= 6) remote_table_schema = engine_args[5]->as().value.safeGet(); + if (engine_args.size() >= 7) + on_conflict = engine_args[6]->as().value.safeGet(); auto pool = std::make_shared( remote_database, @@ -345,7 +421,8 @@ void registerStoragePostgreSQL(StorageFactory & factory) args.columns, args.constraints, args.comment, - remote_table_schema); + remote_table_schema, + on_conflict); }, { .source_access_type = AccessType::POSTGRES, diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index bd5cd317c3d..a12b52e6e48 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -27,7 +27,8 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, - const std::string & remote_table_schema_ = ""); + const String & remote_table_schema_ = "", + const String & on_conflict = ""); String getName() const override { return "PostgreSQL"; } @@ -47,6 +48,7 @@ private: String remote_table_name; String remote_table_schema; + String on_conflict; postgres::PoolWithFailoverPtr pool; }; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index d701728479b..a3bb93d2655 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -37,7 +37,8 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, columns, ConstraintsDescription{}, String{}, - remote_table_schema); + remote_table_schema, + on_conflict); result->startup(); return result; @@ -67,9 +68,9 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, Contex ASTs & args = func_args.arguments->children; - if (args.size() < 5 || args.size() > 6) + if (args.size() < 5 || args.size() > 7) throw Exception("Table function 'PostgreSQL' requires from 5 to 6 parameters: " - "PostgreSQL('host:port', 'database', 'table', 'user', 'password', [, 'schema']).", + "PostgreSQL('host:port', 'database', 'table', 'user', 'password', [, 'schema', 'ON CONFLICT ...']).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (auto & arg : args) @@ -82,8 +83,10 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, Contex remote_table_name = args[2]->as().value.safeGet(); - if (args.size() == 6) + if (args.size() >= 6) remote_table_schema = args[5]->as().value.safeGet(); + if (args.size() >= 7) + on_conflict = args[6]->as().value.safeGet(); connection_pool = std::make_shared( args[1]->as().value.safeGet(), diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index c31d02fa955..e3810a0e391 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -28,7 +28,7 @@ private: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; String connection_str; - String remote_table_name, remote_table_schema; + String remote_table_name, remote_table_schema, on_conflict; postgres::PoolWithFailoverPtr connection_pool; }; diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 28a76631c0f..bb0e284eac9 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -291,7 +291,7 @@ def test_postgres_distributed(started_cluster): node2.query('DROP TABLE test_shards') node2.query('DROP TABLE test_replicas') - + def test_datetime_with_timezone(started_cluster): cursor = started_cluster.postgres_conn.cursor() cursor.execute("DROP TABLE IF EXISTS test_timezone") @@ -328,6 +328,32 @@ def test_postgres_ndim(started_cluster): cursor.execute("DROP TABLE arr1, arr2") +def test_postgres_on_conflict(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + table = 'test_conflict' + cursor.execute(f'DROP TABLE IF EXISTS {table}') + cursor.execute(f'CREATE TABLE {table} (a integer PRIMARY KEY, b text, c integer)') + + node1.query(''' + CREATE TABLE test_conflict (a UInt32, b String, c Int32) + ENGINE PostgreSQL('postgres1:5432', 'postgres', 'test_conflict', 'postgres', 'mysecretpassword', '', 'ON CONFLICT DO NOTHING'); + ''') + node1.query(f''' INSERT INTO {table} SELECT number, concat('name_', toString(number)), 3 from numbers(100)''') + node1.query(f''' INSERT INTO {table} SELECT number, concat('name_', toString(number)), 4 from numbers(100)''') + + check1 = f"SELECT count() FROM {table}" + assert (node1.query(check1)).rstrip() == '100' + + table_func = f'''postgresql('{started_cluster.postgres_ip}:{started_cluster.postgres_port}', 'postgres', '{table}', 'postgres', 'mysecretpassword', '', 'ON CONFLICT DO NOTHING')''' + node1.query(f'''INSERT INTO TABLE FUNCTION {table_func} SELECT number, concat('name_', toString(number)), 3 from numbers(100)''') + node1.query(f'''INSERT INTO TABLE FUNCTION {table_func} SELECT number, concat('name_', toString(number)), 3 from numbers(100)''') + + check1 = f"SELECT count() FROM {table}" + assert (node1.query(check1)).rstrip() == '100' + + cursor.execute(f'DROP TABLE {table} ') + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From ee7f710e3526def3599948f297f357a4b4fd2f8d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 24 Aug 2021 16:13:41 +0300 Subject: [PATCH 062/127] Update TableFunctionPostgreSQL.cpp --- src/TableFunctions/TableFunctionPostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index a3bb93d2655..568cc6171fd 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -69,7 +69,7 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, Contex ASTs & args = func_args.arguments->children; if (args.size() < 5 || args.size() > 7) - throw Exception("Table function 'PostgreSQL' requires from 5 to 6 parameters: " + throw Exception("Table function 'PostgreSQL' requires from 5 to 7 parameters: " "PostgreSQL('host:port', 'database', 'table', 'user', 'password', [, 'schema', 'ON CONFLICT ...']).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); From 5bbf5dacc5b59c6343f4725b992f3a46c53f2b0f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 24 Aug 2021 16:13:57 +0300 Subject: [PATCH 063/127] Update src/Storages/StoragePostgreSQL.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Storages/StoragePostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index f8dd1a0d64b..7726008c160 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -383,7 +383,7 @@ void registerStoragePostgreSQL(StorageFactory & factory) ASTs & engine_args = args.engine_args; if (engine_args.size() < 5 || engine_args.size() > 7) - throw Exception("Storage PostgreSQL requires from 5 to 6 parameters: " + throw Exception("Storage PostgreSQL requires from 5 to 7 parameters: " "PostgreSQL('host:port', 'database', 'table', 'username', 'password' [, 'schema', 'ON CONFLICT ...']", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); From 916e6cc9f2c532a4ea95d3c8de39c529fc90b3cc Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Aug 2021 17:06:10 +0300 Subject: [PATCH 064/127] Fix style --- src/Coordination/Changelog.cpp | 2 +- src/Coordination/KeeperServer.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 1bc33a5f386..836df92ac77 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -216,7 +216,7 @@ public: if (record.header.index < start_log_index) continue; - /// Create log entry for readed data + /// Create log entry for read data auto log_entry = nuraft::cs_new(record.header.term, record.blob, record.header.value_type); if (result.first_read_index == 0) result.first_read_index = record.header.index; diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 7c40047ce88..d1138ccef1a 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -77,7 +77,7 @@ public: bool isLeaderAlive() const; - /// Wait server intitialization (see callbackFunc) + /// Wait server initialization (see callbackFunc) void waitInit(); void shutdown(); From 4a86deaa7de1e0e4dbbddbb3ac8b1f9e4c1c33a8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 24 Aug 2021 18:56:32 +0300 Subject: [PATCH 065/127] fix --- tests/queries/0_stateless/replication.lib | 39 +++++++++++++---------- 1 file changed, 22 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 62417822c6b..8fe300b59e8 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -4,15 +4,17 @@ function try_sync_replicas() { + table_name_prefix=$1 + readarray -t empty_partitions_arr < <(${CLICKHOUSE_CLIENT} -q \ "SELECT DISTINCT substr(new_part_name, 1, position(new_part_name, '_') - 1) AS partition_id FROM system.replication_queue - WHERE (database = currentDatabase()) AND (table LIKE '$1%') AND (last_exception LIKE '%No active replica has part%') AND (partition_id NOT IN ( + WHERE (database = currentDatabase()) AND (table LIKE '$table_name_prefix%') AND (last_exception LIKE '%No active replica has part%') AND (partition_id NOT IN ( SELECT partition_id FROM system.parts - WHERE (database = currentDatabase()) AND (table LIKE '$1%') + WHERE (database = currentDatabase()) AND (table LIKE '$table_name_prefix%') ))") - readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' AND engine like '%Replicated%'") + readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' AND engine like '%Replicated%'") for t in "${tables_arr[@]}" do @@ -30,51 +32,54 @@ function try_sync_replicas() "select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t' order by database, table, node_name" & done wait - echo "Replication did not hang: synced all replicas of $1" + echo "Replication did not hang: synced all replicas of $table_name_prefix" } function check_replication_consistency() { + table_name_prefix=$1 + check_query_part=$2 + # Do not check anything if all replicas are readonly, # because is this case all replicas are probably lost (it may happen and it's not a bug) - res=$($CLICKHOUSE_CLIENT -q "SELECT count() - sum(is_readonly) FROM system.replicas WHERE database=currentDatabase() AND table LIKE '$1%'") + res=$($CLICKHOUSE_CLIENT -q "SELECT count() - sum(is_readonly) FROM system.replicas WHERE database=currentDatabase() AND table LIKE '$table_name_prefix%'") if [ $res -eq 0 ]; then # Print dummy lines - echo "Replication did not hang: synced all replicas of $1" + echo "Replication did not hang: synced all replicas of $table_name_prefix" echo "Consistency: 1" return 0 fi # Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them - some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' ORDER BY rand() LIMIT 1") + some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1") $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: - some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$1%' ORDER BY rand() LIMIT 1") + some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1") $CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||: # Forcefully cancel mutations to avoid waiting for them to finish - ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$1%'" > /dev/null + ${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet - wait_for_all_mutations "$1%" + wait_for_all_mutations "$table_name_prefix%" - try_sync_replicas "$1" + try_sync_replicas "$table_name_prefix" res=$($CLICKHOUSE_CLIENT -q \ "SELECT if((countDistinct(data) as c) == 0, 1, c) FROM ( - SELECT _table, ($2) AS data - FROM merge(currentDatabase(), '$1') GROUP BY _table + SELECT _table, ($check_query_part) AS data + FROM merge(currentDatabase(), '$table_name_prefix') GROUP BY _table )") echo "Consistency: $res" if [ $res -ne 1 ]; then echo "Replicas have diverged:" - $CLICKHOUSE_CLIENT -q "select 'data', _table, $2, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$1') group by _table order by _table" - $CLICKHOUSE_CLIENT -q "select 'queue', * from system.replication_queue where database=currentDatabase() and table like '$1%' order by database, table, node_name" - $CLICKHOUSE_CLIENT -q "select 'mutations', * from system.mutations where database=currentDatabase() and table like '$1%' order by database, table, mutation_id" - $CLICKHOUSE_CLIENT -q "select 'parts', * from system.parts where database=currentDatabase() and table like '$1%' order by database, table, name" + $CLICKHOUSE_CLIENT -q "select 'data', _table, $check_query_part, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$table_name_prefix') group by _table order by _table" + $CLICKHOUSE_CLIENT -q "select 'queue', * from system.replication_queue where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, node_name" + $CLICKHOUSE_CLIENT -q "select 'mutations', * from system.mutations where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, mutation_id" + $CLICKHOUSE_CLIENT -q "select 'parts', * from system.parts where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, name" echo "Good luck with debugging..." fi From c491d1a587855f418376a3b22e59f5ebbc59a040 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:51:20 +0300 Subject: [PATCH 066/127] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5d9e02b8200..e2a6111bfd8 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2238,7 +2238,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## queryID {#query-id} -Returns the ID of the current query, which can be used instantly in other queries. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. +Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. !!! warning "Warning" In contrast to [initialQueryID](#initial-query-id) function `queryID` can return different results on shards (see example). And then the server will argue that constant column has different values. From 77483f14c1c75951c786dde2eacb93ac7a35e90c Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:51:29 +0300 Subject: [PATCH 067/127] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index e2a6111bfd8..67d7205b9cd 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2273,7 +2273,7 @@ Result: ## initialQueryID {#initial-query-id} -Returns the ID of the initial current query, which can be used instantly in other queries. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `initial_query_id`. +Returns the ID of the initial current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `initial_query_id`. In contrast to [queryID](#query-id) function `initialQueryID` returns same results on shards (see example). From 41bf324f9c8e64d35a312d2a738327486eebb34f Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:51:56 +0300 Subject: [PATCH 068/127] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 67d7205b9cd..1d8ac892c84 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2275,7 +2275,7 @@ Result: Returns the ID of the initial current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `initial_query_id`. -In contrast to [queryID](#query-id) function `initialQueryID` returns same results on shards (see example). +In contrast to [queryID](#query-id) function, `initialQueryID` returns the same results on different shards (see example). **Syntax** From dd5e303e2914c8edca41be0e1848d64f742993f2 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:52:28 +0300 Subject: [PATCH 069/127] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 1d8ac892c84..567c4e863bc 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2241,7 +2241,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. !!! warning "Warning" - In contrast to [initialQueryID](#initial-query-id) function `queryID` can return different results on shards (see example). And then the server will argue that constant column has different values. + In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on shards (see example). And then the server will argue that constant column has different values. **Syntax** From 9a0b4f666f82a39be34486e886fa748c223d0229 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:52:38 +0300 Subject: [PATCH 070/127] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 5088bbbaceb..2ac4a97f9f7 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2188,7 +2188,7 @@ defaultRoles() ## queryID {#query-id} -Возвращает идентификатор текущего запроса, который может быть мгновенно использован в других запросах. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `query_id`. +Возвращает идентификатор текущего запроса. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `query_id`. !!! warning "Предупреждение" В отличие от [initialQueryID](#initial-query-id) функция `queryID` может возвращать различные значения на шардах (см. пример), что неверно для столбца констант. From 15e986b3d9a3c3c366258dae46d36caa693d238a Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:55:35 +0300 Subject: [PATCH 071/127] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 2ac4a97f9f7..bcfe7d84dff 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2191,7 +2191,7 @@ defaultRoles() Возвращает идентификатор текущего запроса. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `query_id`. !!! warning "Предупреждение" - В отличие от [initialQueryID](#initial-query-id) функция `queryID` может возвращать различные значения на шардах (см. пример), что неверно для столбца констант. + В отличие от [initialQueryID](#initial-query-id), функция `queryID` может возвращать различные значения для разных шардов (см. пример), что неверно для столбца констант. **Синтаксис** From 980720112ebf19cd39bedb7ff515c0f4a719e513 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:55:42 +0300 Subject: [PATCH 072/127] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index bcfe7d84dff..bd927327d18 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2223,7 +2223,7 @@ SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3} ## initialQueryID {#initial-query-id} -Возвращает идентификатор родительского запроса, который может быть мгновенно использован в других запросах. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `initial_query_id`. +Возвращает идентификатор родительского запроса. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `initial_query_id`. В отличие от [queryID](#query-id) функция `initialQueryID` возвращает одинаковые значения на шардах (см. пример). From 77296573e6889e4c3e4cd9aa299c536482cec3f4 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Tue, 24 Aug 2021 19:55:53 +0300 Subject: [PATCH 073/127] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index bd927327d18..741af425dee 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2225,7 +2225,7 @@ SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3} Возвращает идентификатор родительского запроса. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `initial_query_id`. -В отличие от [queryID](#query-id) функция `initialQueryID` возвращает одинаковые значения на шардах (см. пример). +В отличие от [queryID](#query-id), функция `initialQueryID` возвращает одинаковые значения для разных шардов (см. пример). **Синтаксис** From 7d9180cd8cd5d3a3b877d333580d4b1674abfca3 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 24 Aug 2021 20:08:39 +0300 Subject: [PATCH 074/127] Update other-functions.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил предупреждение. --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 567c4e863bc..ea9a0fc4c27 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2241,7 +2241,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. !!! warning "Warning" - In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on shards (see example). And then the server will argue that constant column has different values. + In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). And then the server will argue that the constant column has different values. **Syntax** From 2da3893b1ffffcde2a442faad4cc223cc4bc8be5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 24 Aug 2021 21:11:47 +0300 Subject: [PATCH 075/127] refactor isCompatibleEnumTypes to remove PVS warning --- src/Storages/StorageInMemoryMetadata.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 27ea8fcbb06..a1325cb0569 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -503,13 +503,13 @@ namespace */ bool isCompatibleEnumTypes(const IDataType * lhs, const IDataType * rhs) { - WhichDataType which{lhs}; - if (!which.isEnum()) - return false; - IDataTypeEnum const * enum_type = dynamic_cast(lhs); - if (!enum_type->contains(*rhs)) - return false; - return enum_type->getMaximumSizeOfValueInMemory() == rhs->getMaximumSizeOfValueInMemory(); + if (IDataTypeEnum const * enum_type = dynamic_cast(lhs)) + { + if (!enum_type->contains(*rhs)) + return false; + return enum_type->getMaximumSizeOfValueInMemory() == rhs->getMaximumSizeOfValueInMemory(); + } + return false; } } From 47bb5eb3303d72c76ecddec17a105b4df29cf0d2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 24 Aug 2021 21:14:56 +0300 Subject: [PATCH 076/127] support implicit conversions between index in operator [] and key of Map --- src/Functions/array/arrayElement.cpp | 326 +++++++++--------- .../02014_map_different_keys.reference | 22 ++ .../0_stateless/02014_map_different_keys.sql | 32 ++ 3 files changed, 219 insertions(+), 161 deletions(-) create mode 100644 tests/queries/0_stateless/02014_map_different_keys.reference create mode 100644 tests/queries/0_stateless/02014_map_different_keys.sql diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 59594a78401..a4cdc601d84 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -95,32 +96,30 @@ private: using Offsets = ColumnArray::Offsets; - static bool matchKeyToIndex(const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs); + static bool matchKeyToIndexNumber( + const IColumn & data, const Offsets & offsets, bool is_key_const, + const IColumn & index, PaddedPODArray & matched_idxs); - static bool matchKeyToIndexConst(const IColumn & data, const Offsets & offsets, + static bool matchKeyToIndexNumberConst( + const IColumn & data, const Offsets & offsets, const Field & index, PaddedPODArray & matched_idxs); - template - static bool matchKeyToIndexNumber(const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs); + static bool matchKeyToIndexString( + const IColumn & data, const Offsets & offsets, bool is_key_const, + const IColumn & index, PaddedPODArray & matched_idxs); - template - static bool matchKeyToIndexNumberConst(const IColumn & data, const Offsets & offsets, - const Field & index, PaddedPODArray & matched_idxs); - - static bool matchKeyToIndexString(const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs); - - static bool matchKeyToIndexFixedString(const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs); - - static bool matchKeyToIndexStringConst(const IColumn & data, const Offsets & offsets, + static bool matchKeyToIndexStringConst( + const IColumn & data, const Offsets & offsets, const Field & index, PaddedPODArray & matched_idxs); template static void executeMatchKeyToIndex(const Offsets & offsets, PaddedPODArray & matched_idxs, const Matcher & matcher); + + template + static void executeMatchConstKeyToIndex( + size_t num_rows, size_t num_values, + PaddedPODArray & matched_idxs, const Matcher & matcher); }; @@ -759,23 +758,11 @@ ColumnPtr FunctionArrayElement::executeTuple(const ColumnsWithTypeAndName & argu namespace { +template struct MatcherString { - const ColumnString & data; - const ColumnString & index; - - bool match(size_t row_data, size_t row_index) const - { - auto data_ref = data.getDataAt(row_data); - auto index_ref = index.getDataAt(row_index); - return memequalSmallAllowOverflow15(index_ref.data, index_ref.size, data_ref.data, data_ref.size); - } -}; - -struct MatcherFixedString -{ - const ColumnFixedString & data; - const ColumnFixedString & index; + const DataColumn & data; + const IndexColumn & index; bool match(size_t row_data, size_t row_index) const { @@ -785,9 +772,10 @@ struct MatcherFixedString } }; +template struct MatcherStringConst { - const ColumnString & data; + const DataColumn & data; const String & index; bool match(size_t row_data, size_t /* row_index */) const @@ -797,23 +785,23 @@ struct MatcherStringConst } }; -template +template struct MatcherNumber { - const PaddedPODArray & data; - const PaddedPODArray & index; + const PaddedPODArray & data; + const PaddedPODArray & index; bool match(size_t row_data, size_t row_index) const { - return data[row_data] == index[row_index]; + return data[row_data] == static_cast(index[row_index]); } }; -template +template struct MatcherNumberConst { - const PaddedPODArray & data; - T index; + const PaddedPODArray & data; + DataType index; bool match(size_t row_data, size_t /* row_index */) const { @@ -848,147 +836,158 @@ void FunctionArrayElement::executeMatchKeyToIndex( } } +template +void FunctionArrayElement::executeMatchConstKeyToIndex( + size_t num_rows, size_t num_values, + PaddedPODArray & matched_idxs, const Matcher & matcher) +{ + for (size_t i = 0; i < num_rows; ++i) + { + bool matched = false; + for (size_t j = 0; j < num_values; ++j) + { + if (matcher.match(j, i)) + { + matched_idxs.push_back(j + 1); + matched = true; + break; + } + } + + if (!matched) + matched_idxs.push_back(0); + } +} + +template +static bool castColumnString(const IColumn * column, F && f) +{ + return castTypeToEither(column, std::forward(f)); +} + bool FunctionArrayElement::matchKeyToIndexStringConst( const IColumn & data, const Offsets & offsets, const Field & index, PaddedPODArray & matched_idxs) { - const auto * data_string = checkAndGetColumn(&data); - if (!data_string) - return false; + return castColumnString(&data, [&](const auto & data_column) + { + using DataColumn = std::decay_t; - if (index.getType() != Field::Types::String) - return false; - - MatcherStringConst matcher{*data_string, get(index)}; - executeMatchKeyToIndex(offsets, matched_idxs, matcher); - return true; + MatcherStringConst matcher{data_column, get(index)}; + executeMatchKeyToIndex(offsets, matched_idxs, matcher); + return true; + }); } bool FunctionArrayElement::matchKeyToIndexString( - const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs) + const IColumn & data, const Offsets & offsets, bool is_key_const, + const IColumn & index, PaddedPODArray & matched_idxs) { - const auto * index_string = checkAndGetColumn(arguments[1].column.get()); - if (!index_string) - return false; + return castColumnString(&data, [&](const auto & data_column) + { + return castColumnString(&index, [&](const auto & index_column) + { + using DataColumn = std::decay_t; + using IndexColumn = std::decay_t; - const auto * data_string = checkAndGetColumn(&data); - if (!data_string) - return false; + MatcherString matcher{data_column, index_column}; + if (is_key_const) + executeMatchConstKeyToIndex(index.size(), data.size(), matched_idxs, matcher); + else + executeMatchKeyToIndex(offsets, matched_idxs, matcher); - MatcherString matcher{*data_string, *index_string}; - executeMatchKeyToIndex(offsets, matched_idxs, matcher); - return true; + return true; + }); + }); } -bool FunctionArrayElement::matchKeyToIndexFixedString( - const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs) +template +static constexpr bool areConvertibleTypes = + std::is_same_v + || (is_integer_v && is_integer_v + && std::is_convertible_v); + +template +static bool castColumnNumeric(const IColumn * column, F && f) { - const auto * index_string = checkAndGetColumn(arguments[1].column.get()); - if (!index_string) - return false; - - const auto * data_string = checkAndGetColumn(&data); - if (!data_string) - return false; - - MatcherFixedString matcher{*data_string, *index_string}; - executeMatchKeyToIndex(offsets, matched_idxs, matcher); - return true; + return castTypeToEither< + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector, + ColumnVector + >(column, std::forward(f)); } -template bool FunctionArrayElement::matchKeyToIndexNumberConst( const IColumn & data, const Offsets & offsets, const Field & index, PaddedPODArray & matched_idxs) { - const auto * data_numeric = checkAndGetColumn>(&data); - if (!data_numeric) - return false; - - std::optional index_as_integer; - Field::dispatch([&](const auto & value) + return castColumnNumeric(&data, [&](const auto & data_column) { - using FieldType = std::decay_t; - if constexpr (std::is_same_v || (is_integer_v && std::is_convertible_v)) - index_as_integer = static_cast(value); - }, index); + using DataType = typename std::decay_t::ValueType; + std::optional index_as_integer; - if (!index_as_integer) - return false; + Field::dispatch([&](const auto & value) + { + using FieldType = std::decay_t; + if constexpr (areConvertibleTypes) + index_as_integer = static_cast(value); + }, index); - MatcherNumberConst matcher{data_numeric->getData(), *index_as_integer}; - executeMatchKeyToIndex(offsets, matched_idxs, matcher); - return true; + if (!index_as_integer) + return false; + + MatcherNumberConst matcher{data_column.getData(), *index_as_integer}; + executeMatchKeyToIndex(offsets, matched_idxs, matcher); + return true; + }); } -template bool FunctionArrayElement::matchKeyToIndexNumber( - const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs) + const IColumn & data, const Offsets & offsets, bool is_key_const, + const IColumn & index, PaddedPODArray & matched_idxs) { - const auto * index_numeric = checkAndGetColumn>(arguments[1].column.get()); - if (!index_numeric) - return false; + return castColumnNumeric(&data, [&](const auto & data_column) + { + return castColumnNumeric(&index, [&](const auto & index_column) + { + using DataType = typename std::decay_t::ValueType; + using IndexType = typename std::decay_t::ValueType; - const auto * data_numeric = checkAndGetColumn>(&data); - if (!data_numeric) - return false; + if constexpr (areConvertibleTypes) + { + MatcherNumber matcher{data_column.getData(), index_column.getData()}; + if (is_key_const) + executeMatchConstKeyToIndex(index_column.size(), data_column.size(), matched_idxs, matcher); + else + executeMatchKeyToIndex(offsets, matched_idxs, matcher); - MatcherNumber matcher{data_numeric->getData(), index_numeric->getData()}; - executeMatchKeyToIndex(offsets, matched_idxs, matcher); - return true; -} + return true; + } -bool FunctionArrayElement::matchKeyToIndex( - const IColumn & data, const Offsets & offsets, - const ColumnsWithTypeAndName & arguments, PaddedPODArray & matched_idxs) -{ - return matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexNumber(data, offsets, arguments, matched_idxs) - || matchKeyToIndexString(data, offsets, arguments, matched_idxs) - || matchKeyToIndexFixedString(data, offsets, arguments, matched_idxs); -} - -bool FunctionArrayElement::matchKeyToIndexConst( - const IColumn & data, const Offsets & offsets, - const Field & index, PaddedPODArray & matched_idxs) -{ - return matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexNumberConst(data, offsets, index, matched_idxs) - || matchKeyToIndexStringConst(data, offsets, index, matched_idxs); + return false; + }); + }); } ColumnPtr FunctionArrayElement::executeMap( const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { - const ColumnMap * col_map = typeid_cast(arguments[0].column.get()); - if (!col_map) - return nullptr; + const auto * col_map = checkAndGetColumn(arguments[0].column.get()); + const auto * col_const_map = checkAndGetColumnConst(arguments[0].column.get()); + assert(col_map || col_const_map); + + if (col_const_map) + col_map = typeid_cast(&col_const_map->getDataColumn()); const auto & nested_column = col_map->getNestedColumn(); const auto & keys_data = col_map->getNestedData().getColumn(0); @@ -1000,29 +999,33 @@ ColumnPtr FunctionArrayElement::executeMap( indices_column->reserve(input_rows_count); auto & indices_data = assert_cast &>(*indices_column).getData(); + bool executed = false; if (!isColumnConst(*arguments[1].column)) { - if (input_rows_count > 0 && !matchKeyToIndex(keys_data, offsets, arguments, indices_data)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal types of arguments: {}, {} for function {}", - arguments[0].type->getName(), arguments[1].type->getName(), getName()); + executed = matchKeyToIndexNumber(keys_data, offsets, !!col_const_map, *arguments[1].column, indices_data) + || matchKeyToIndexString(keys_data, offsets, !!col_const_map, *arguments[1].column, indices_data); } else { Field index = (*arguments[1].column)[0]; - - // Get Matched key's value - if (input_rows_count > 0 && !matchKeyToIndexConst(keys_data, offsets, index, indices_data)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal types of arguments: {}, {} for function {}", - arguments[0].type->getName(), arguments[1].type->getName(), getName()); + executed = matchKeyToIndexNumberConst(keys_data, offsets, index, indices_data) + || matchKeyToIndexStringConst(keys_data, offsets, index, indices_data); } + if (!executed) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal types of arguments: {}, {} for function {}", + arguments[0].type->getName(), arguments[1].type->getName(), getName()); + + ColumnPtr values_array = ColumnArray::create(values_data.getPtr(), nested_column.getOffsetsPtr()); + if (col_const_map) + values_array = ColumnConst::create(values_array, input_rows_count); + /// Prepare arguments to call arrayElement for array with values and calculated indices at previous step. ColumnsWithTypeAndName new_arguments = { { - ColumnArray::create(values_data.getPtr(), nested_column.getOffsetsPtr()), + values_array, std::make_shared(result_type), "" }, @@ -1066,13 +1069,14 @@ DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments) ColumnPtr FunctionArrayElement::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { - /// Check nullability. - bool is_array_of_nullable = false; + const auto * col_map = checkAndGetColumn(arguments[0].column.get()); + const auto * col_const_map = checkAndGetColumnConst(arguments[0].column.get()); - const ColumnMap * col_map = checkAndGetColumn(arguments[0].column.get()); - if (col_map) + if (col_map || col_const_map) return executeMap(arguments, result_type, input_rows_count); + /// Check nullability. + bool is_array_of_nullable = false; const ColumnArray * col_array = nullptr; const ColumnArray * col_const_array = nullptr; diff --git a/tests/queries/0_stateless/02014_map_different_keys.reference b/tests/queries/0_stateless/02014_map_different_keys.reference new file mode 100644 index 00000000000..8af8f57f9df --- /dev/null +++ b/tests/queries/0_stateless/02014_map_different_keys.reference @@ -0,0 +1,22 @@ +...const maps... +0 +2 +0 +4 +0 +0 +2 +0 +4 +0 +4 +4 +...int keys... +foo bar bar + foo foo +...string keys... +foo foo +foo foo +foo foo +bar bar +0 diff --git a/tests/queries/0_stateless/02014_map_different_keys.sql b/tests/queries/0_stateless/02014_map_different_keys.sql new file mode 100644 index 00000000000..0998a9283f7 --- /dev/null +++ b/tests/queries/0_stateless/02014_map_different_keys.sql @@ -0,0 +1,32 @@ +SELECT '...const maps...'; + +WITH map(1, 2, 3, 4) AS m SELECT m[number] FROM numbers(5); +WITH map('1', 2, '3', 4) AS m SELECT m[toString(number)] FROM numbers(5); + +WITH map(1, 2, 3, 4) AS m SELECT m[3]; +WITH map('1', 2, '3', 4) AS m SELECT m['3']; + +DROP TABLE IF EXISTS t_map_02014; + +CREATE TABLE t_map_02014(i1 UInt64, i2 Int32, m1 Map(UInt32, String), m2 Map(Int8, String), m3 Map(Int128, String)) ENGINE = Memory; +INSERT INTO t_map_02014 VALUES (1, -1, map(1, 'foo', 2, 'bar'), map(-1, 'foo', 1, 'bar'), map(-1, 'foo', 1, 'bar')); + +SELECT '...int keys...'; + +SELECT m1[i1], m2[i1], m3[i1] FROM t_map_02014; +SELECT m1[i2], m2[i2], m3[i2] FROM t_map_02014; + +DROP TABLE IF EXISTS t_map_02014; + +CREATE TABLE t_map_02014(s String, fs FixedString(3), m1 Map(String, String), m2 Map(FixedString(3), String)) ENGINE = Memory; +INSERT INTO t_map_02014 VALUES ('aaa', 'bbb', map('aaa', 'foo', 'bbb', 'bar'), map('aaa', 'foo', 'bbb', 'bar')); + +SELECT '...string keys...'; + +SELECT m1['aaa'], m2['aaa'] FROM t_map_02014; +SELECT m1['aaa'::FixedString(3)], m2['aaa'::FixedString(3)] FROM t_map_02014; +SELECT m1[s], m2[s] FROM t_map_02014; +SELECT m1[fs], m2[fs] FROM t_map_02014; +SELECT length(m2['aaa'::FixedString(4)]) FROM t_map_02014; + +DROP TABLE IF EXISTS t_map_02014; From 8613d89001ac53cda7ed6681ef47ae9761f8826b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 24 Aug 2021 18:28:38 +0000 Subject: [PATCH 077/127] Fix build --- src/Common/getInsertQuery.cpp | 28 ++++++++++++++++++++++++++++ src/Common/getInsertQuery.h | 24 +++--------------------- 2 files changed, 31 insertions(+), 21 deletions(-) create mode 100644 src/Common/getInsertQuery.cpp diff --git a/src/Common/getInsertQuery.cpp b/src/Common/getInsertQuery.cpp new file mode 100644 index 00000000000..830374b6060 --- /dev/null +++ b/src/Common/getInsertQuery.cpp @@ -0,0 +1,28 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ +std::string getInsertQuery(const std::string & db_name, const std::string & table_name, const ColumnsWithTypeAndName & columns, IdentifierQuotingStyle quoting) +{ + ASTInsertQuery query; + query.table_id.database_name = db_name; + query.table_id.table_name = table_name; + query.columns = std::make_shared(','); + query.children.push_back(query.columns); + for (const auto & column : columns) + query.columns->children.emplace_back(std::make_shared(column.name)); + + WriteBufferFromOwnString buf; + IAST::FormatSettings settings(buf, true); + settings.always_quote_identifiers = true; + settings.identifier_quoting_style = quoting; + query.IAST::format(settings); + return buf.str(); +} +} diff --git a/src/Common/getInsertQuery.h b/src/Common/getInsertQuery.h index c0f77064e74..0bcb5e3660b 100644 --- a/src/Common/getInsertQuery.h +++ b/src/Common/getInsertQuery.h @@ -1,26 +1,8 @@ #pragma once -#include -#include -#include -#include +#include +#include namespace DB { -std::string getInsertQuery(const std::string & db_name, const std::string & table_name, const ColumnsWithTypeAndName & columns, IdentifierQuotingStyle quoting) -{ - ASTInsertQuery query; - query.table_id.database_name = db_name; - query.table_id.table_name = table_name; - query.columns = std::make_shared(','); - query.children.push_back(query.columns); - for (const auto & column : columns) - query.columns->children.emplace_back(std::make_shared(column.name)); - - WriteBufferFromOwnString buf; - IAST::FormatSettings settings(buf, true); - settings.always_quote_identifiers = true; - settings.identifier_quoting_style = quoting; - query.IAST::format(settings); - return buf.str(); -} +std::string getInsertQuery(const std::string & db_name, const std::string & table_name, const ColumnsWithTypeAndName & columns, IdentifierQuotingStyle quoting); } From b330cb855f163f573a269ac7720a6ee2035d751a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 24 Aug 2021 23:11:48 +0000 Subject: [PATCH 078/127] Fix build --- docker/packager/deb/build.sh | 6 ++++++ docker/packager/other/fuzzer.sh | 9 ++++++--- docker/packager/packager | 3 +++ 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index 5016e40dd2e..fb9c3f6cab7 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -33,6 +33,12 @@ fi # Also build fuzzers if any sanitizer specified if [ -n "$SANITIZER" ] then + # Script is supposed that we are in build directory. + mkdir -p build/build_docker + cd build/build_docker + # Launching build script + ../docker/packager/other/fuzzer.sh + cd build/docker/packager/other/fuzzer.sh fi diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh index d6609d3c632..fe4de5802dc 100755 --- a/docker/packager/other/fuzzer.sh +++ b/docker/packager/other/fuzzer.sh @@ -1,15 +1,18 @@ #!/usr/bin/env bash -set -x -e - # This script is responsible for building all fuzzers, and copy them to output directory # as an archive. +# Script is supposed that we are in build directory. + +set -x -e + +printenv # Delete previous cache, because we add a new flags -DENABLE_FUZZING=1 and -DFUZZER=libfuzzer rm -f CMakeCache.txt read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" # Hope, that the most part of files will be in cache, so we just link new executables -cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" \ +cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_C_COMPILER=$CC -DCMAKE_CXX_COMPILER=$CXX \ "-DSANITIZE=$SANITIZER" -DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0 \ -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. diff --git a/docker/packager/packager b/docker/packager/packager index 95b7fcd8568..673878bce43 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -105,6 +105,9 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ if image_type == "deb" or image_type == "unbundled": result.append("DEB_CC={}".format(cc)) result.append("DEB_CXX={}".format(cxx)) + # For building fuzzers + result.append("CC={}".format(cc)) + result.append("CXX={}".format(cxx)) elif image_type == "binary": result.append("CC={}".format(cc)) result.append("CXX={}".format(cxx)) From 009411542633a3bc45aae4c10d9f899a6811122f Mon Sep 17 00:00:00 2001 From: terrylin Date: Wed, 25 Aug 2021 10:57:17 +0800 Subject: [PATCH 079/127] improvement of materilize ttl --- .../0_stateless/01070_modify_ttl_recalc_only.reference | 7 +++++++ .../queries/0_stateless/01070_modify_ttl_recalc_only.sql | 8 ++++++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference index 8bfefa38ae1..fe9cba71c4c 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.reference @@ -2,6 +2,10 @@ 2000-10-10 2 2100-10-10 3 2100-10-10 4 +2000-10-11 00:00:00 2000-10-11 00:00:00 +2000-10-11 00:00:00 2000-10-11 00:00:00 +2100-10-11 00:00:00 2100-10-11 00:00:00 +2100-10-11 00:00:00 2100-10-11 00:00:00 2100-10-10 3 2100-10-10 4 ============= @@ -9,11 +13,13 @@ 2 b 3 c 4 d +2000-01-01 00:00:00 2100-01-01 00:00:00 1 a 3 c ============= 1 a 3 c +2000-01-01 00:00:00 2000-01-01 00:00:00 ============= 1 a 2 b @@ -37,6 +43,7 @@ 2 b 3 c 4 d +2000-01-01 00:00:00 2100-01-01 00:00:00 1 a 2 b 4 d diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql index 5dcea47c4d9..aafed1a7bce 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql @@ -13,6 +13,7 @@ insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); alter table ttl modify ttl d + interval 1 day; select * from ttl order by a; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where database = currentDatabase() and table = 'ttl' and active > 0 order by name asc; optimize table ttl final; select * from ttl order by a; select '============='; @@ -26,12 +27,14 @@ insert into ttl values (1, 'a') (2, 'b') (3, 'c') (4, 'd'); alter table ttl modify ttl i % 2 = 0 ? toDate('2000-01-01') : toDate('2100-01-01'); select * from ttl order by i; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where database = currentDatabase() and table = 'ttl' and active > 0; optimize table ttl final; select * from ttl order by i; select '============='; alter table ttl modify ttl toDate('2000-01-01'); select * from ttl order by i; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where database = currentDatabase() and table = 'ttl' and active > 0; optimize table ttl final; select * from ttl order by i; select '============='; @@ -64,6 +67,7 @@ insert into ttl values (toDate('2000-01-02'), 1, 'a') (toDate('2000-01-03'), 2, alter table ttl modify ttl i % 3 = 0 ? toDate('2000-01-01') : toDate('2100-01-01'); select i, s from ttl order by i; +select delete_ttl_info_min, delete_ttl_info_max from system.parts where database = currentDatabase() and table = 'ttl' and active > 0; optimize table ttl final; select i, s from ttl order by i; select '============='; @@ -88,7 +92,7 @@ select i, s, t from ttl order by i; optimize table ttl final; select i, s, t from ttl order by i; -- MATERIALIZE TTL ran only once -select count() from system.mutations where table = 'ttl' and is_done; +select count() from system.mutations where database = currentDatabase() and table = 'ttl' and is_done; select '============='; drop table if exists ttl; @@ -98,6 +102,6 @@ create table ttl (i Int, s String ttl toDate('2000-01-02')) engine = MergeTree o SETTINGS max_number_of_merges_with_ttl_in_pool=0,materialize_ttl_recalculate_only=true; alter table ttl modify column s String ttl toDate('2000-01-02'); -select count() from system.mutations where table = 'ttl' and is_done; +select count() from system.mutations where database = currentDatabase() and table = 'ttl' and is_done; drop table if exists ttl; From 225f44248c73de2b2bc4f1b3b25a3b5ed5958bfb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Aug 2021 09:06:54 +0300 Subject: [PATCH 080/127] Fix remote query cancelation (in case of remote server abnormaly terminated) In case the remote server abnormally terminated, neither FIN nor RST packet will be sent, and the initiator will not know that the connection died (unless tcp_keep_alive_timeout > 0). Fix this, using default timeouts. --- src/DataStreams/RemoteQueryExecutor.cpp | 11 +++++++++++ src/DataStreams/RemoteQueryExecutorReadContext.h | 1 + 2 files changed, 12 insertions(+) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 21e874691c1..3b207110a67 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -526,7 +526,18 @@ void RemoteQueryExecutor::tryCancel(const char * reason, std::unique_ptr 0). + /// + /// Also note that it is possible to get this situation even when + /// enough data already had been read. + (*read_context)->setTimer(); (*read_context)->cancel(); + } connections->sendCancel(); diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.h b/src/DataStreams/RemoteQueryExecutorReadContext.h index 5c56bb73dd6..8cf5f5be1b7 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.h +++ b/src/DataStreams/RemoteQueryExecutorReadContext.h @@ -75,6 +75,7 @@ class RemoteQueryExecutorReadContext { public: void cancel() {} + void setTimer() {} }; } From 2df0411b3fdf6f79f15da7aeeaded807280a7e4f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 23 Aug 2021 21:17:11 +0300 Subject: [PATCH 081/127] Add TimerDescriptor::setRelative(uint64_t usec) --- src/Common/TimerDescriptor.cpp | 13 ++++++++++--- src/Common/TimerDescriptor.h | 1 + 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 793f7ed1352..086d462eeb2 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -74,17 +74,24 @@ void TimerDescriptor::drain() const } } -void TimerDescriptor::setRelative(Poco::Timespan timespan) const +void TimerDescriptor::setRelative(uint64_t usec) const { + static constexpr uint32_t TIMER_PRECISION = 1e6; + itimerspec spec; spec.it_interval.tv_nsec = 0; spec.it_interval.tv_sec = 0; - spec.it_value.tv_sec = timespan.totalSeconds(); - spec.it_value.tv_nsec = timespan.useconds() * 1000; + spec.it_value.tv_sec = usec / TIMER_PRECISION; + spec.it_value.tv_nsec = (usec % TIMER_PRECISION) * 1'000; if (-1 == timerfd_settime(timer_fd, 0 /*relative timer */, &spec, nullptr)) throwFromErrno("Cannot set time for timer_fd", ErrorCodes::CANNOT_SET_TIMER_PERIOD); } +void TimerDescriptor::setRelative(Poco::Timespan timespan) const +{ + setRelative(timespan.totalMicroseconds()); +} + } #endif diff --git a/src/Common/TimerDescriptor.h b/src/Common/TimerDescriptor.h index 30a610f37f0..8ca69344b53 100644 --- a/src/Common/TimerDescriptor.h +++ b/src/Common/TimerDescriptor.h @@ -24,6 +24,7 @@ public: void reset() const; void drain() const; + void setRelative(uint64_t usec) const; void setRelative(Poco::Timespan timespan) const; }; From e8d87053c04e8e30bb35fa46298abb521818731f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 23 Aug 2021 21:17:11 +0300 Subject: [PATCH 082/127] Fix data-race between setConnectionFD() and setTimer() from cancellation path TSan report [1]: ================== WARNING: ThreadSanitizer: data race (pid=7) Write of size 8 at 0x7b5000d83768 by thread T539: 0 Poco::Timespan::operator=(Poco::Timespan const&) obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Timespan.cpp:61:8 (clickhouse+0x18e2da08) 1 DB::RemoteQueryExecutorReadContext::setConnectionFD(int, Poco::Timespan, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../src/DataStreams/RemoteQueryExecutorReadContext.cpp:103:21 (clickhouse+0x142c4f41) 2 DB::RemoteQueryExecutorRoutine::ReadCallback::operator()(int, Poco::Timespan, std::__1::basic_string, std::__1::allocator >) obj-x86_64-linux-gnu/../src/DataStreams/RemoteQueryExecutorReadContext.cpp:26:30 (clickhouse+0x142c7dae) 3 decltype(std::__1::forward(fp)(std::__1::forward(fp0), std::__1::forward(fp0), std::__1::forward, std::__1::allocator > const&>(fp0))) std::__1::__invoke, std::__1::allocator > const&>(DB::RemoteQueryExecutorRoutine::ReadCallback&, int&&, Poco::Timespan&&, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse+0x142c7cf7) 4 void std::__1::__invoke_void_return_wrapper::__call, std::__1::allocator > const&>(DB::RemoteQueryExecutorRoutine::ReadCallback&, int&&, Poco::Timespan&&, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse+0x142c7cf7) 5 std::__1::__function::__default_alloc_func, std::__1::allocator > const&)>::operator()(int&&, Poco::Timespan&&, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse+0x142c7c21) 6 void std::__1::__function::__policy_invoker, std::__1::allocator > const&)>::__call_impl, std::__1::allocator > const&)> >(std::__1::__function::__policy_storage const*, int, Poco::Timespan&&, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse+0x142c7c21) 7 std::__1::__function::__policy_func, std::__1::allocator > const&)>::operator()(int&&, Poco::Timespan&&, std::__1::basic_string, std::__1::allocator > const&) const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse+0x1589a8c1) 8 std::__1::function, std::__1::allocator > const&)>::operator()(int, Poco::Timespan, std::__1::basic_string, std::__1::allocator > const&) const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse+0x1589a8c1) 9 DB::HedgedConnections::getReadyFileDescriptor(std::__1::function, std::__1::allocator > const&)>) obj-x86_64-linux-gnu/../src/Client/HedgedConnections.cpp:379:13 (clickhouse+0x1589a8c1) 10 DB::HedgedConnections::getReadyReplicaLocation(std::__1::function, std::__1::allocator > const&)>) obj-x86_64-linux-gnu/../src/Client/HedgedConnections.cpp:317:20 (clickhouse+0x1589a8c1) 11 DB::HedgedConnections::receivePacketUnlocked(std::__1::function, std::__1::allocator > const&)>, bool) obj-x86_64-linux-gnu/../src/Client/HedgedConnections.cpp:298:32 (clickhouse+0x1589b531) 12 DB::RemoteQueryExecutorRoutine::operator()(boost::context::fiber&&) const obj-x86_64-linux-gnu/../src/DataStreams/RemoteQueryExecutorReadContext.cpp:46:51 (clickhouse+0x142c7013) 13 std::__1::enable_if::type>::value), std::__1::result_of::type>::type boost::context::detail::invoke(DB::RemoteQueryExecutorRoutine&, boost::context::fiber&&) obj-x86_64-linux-gnu/../contrib/boost/boost/context/detail/invoke.hpp:41:12 (clickhouse+0x142c6be1) 14 boost::context::detail::fiber_capture_record::run() obj-x86_64-linux-gnu/../contrib/boost/boost/context/fiber_ucontext.hpp:291:17 (clickhouse+0x142c6be1) 15 void boost::context::detail::fiber_entry_func >(void*) obj-x86_64-linux-gnu/../contrib/boost/boost/context/fiber_ucontext.hpp:72:13 (clickhouse+0x142c5f79) 16 (libc.so.6+0x5e65f) Previous read of size 8 at 0x7b5000d83768 by thread T321 (mutexes: write M861449115349678976, write M1059607378698108960, write M747733173224015304): 0 Poco::Timespan::totalMicroseconds() const obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/Timespan.h:213:9 (clickhouse+0x142c54b3) 1 DB::RemoteQueryExecutorReadContext::setTimer() const obj-x86_64-linux-gnu/../src/DataStreams/RemoteQueryExecutorReadContext.cpp:160:25 (clickhouse+0x142c54b3) 2 DB::RemoteQueryExecutor::tryCancel(char const*, std::__1::unique_ptr >*) obj-x86_64-linux-gnu/../src/DataStreams/RemoteQueryExecutor.cpp:538:26 (clickhouse+0x142be3a5) 3 DB::RemoteQueryExecutor::cancel(std::__1::unique_ptr >*) obj-x86_64-linux-gnu/../src/DataStreams/RemoteQueryExecutor.cpp:462:5 (clickhouse+0x142bdd8a) 4 DB::RemoteSource::onCancel() obj-x86_64-linux-gnu/../src/Processors/Sources/RemoteSource.cpp:112:21 (clickhouse+0x15bf52c6) 5 DB::IProcessor::cancel() obj-x86_64-linux-gnu/../src/Processors/IProcessor.h:237:9 (clickhouse+0x159db37e) 6 DB::PipelineExecutor::cancel() obj-x86_64-linux-gnu/../src/Processors/Executors/PipelineExecutor.cpp:376:20 (clickhouse+0x159db37e) 7 DB::PullingAsyncPipelineExecutor::cancel() obj-x86_64-linux-gnu/../src/Processors/Executors/PullingAsyncPipelineExecutor.cpp:175:25 (clickhouse+0x159e9748) 8 DB::TCPHandler::processOrdinaryQueryWithProcessors() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:700:26 (clickhouse+0x1594f554) 9 DB::TCPHandler::runImpl() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:329:17 (clickhouse+0x15949a2e) 10 DB::TCPHandler::run() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1658:9 (clickhouse+0x15959007) 11 Poco::Net::TCPServerConnection::start() obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:43:3 (clickhouse+0x18cb2e42) 12 Poco::Net::TCPServerDispatcher::run() obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:115:20 (clickhouse+0x18cb35cf) 13 Poco::PooledThread::run() obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:199:14 (clickhouse+0x18e29c21) 14 Poco::(anonymous namespace)::RunnableHolder::run() obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread.cpp:55:11 (clickhouse+0x18e280af) 15 Poco::ThreadImpl::runnableEntry(void*) obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:345:27 (clickhouse+0x18e26827) [1]: https://clickhouse-test-reports.s3.yandex.net/27881/b328d9687238617e89c0806c9fa2dd7dd44b5fd7/integration_tests_(thread).htmlfail1 --- src/DataStreams/RemoteQueryExecutorReadContext.cpp | 6 +++--- src/DataStreams/RemoteQueryExecutorReadContext.h | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/DataStreams/RemoteQueryExecutorReadContext.cpp index c1f415bb597..6bdf52d2831 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/DataStreams/RemoteQueryExecutorReadContext.cpp @@ -100,7 +100,7 @@ void RemoteQueryExecutorReadContext::setConnectionFD(int fd, Poco::Timespan time connection_fd = fd; epoll.add(connection_fd); - receive_timeout = timeout; + receive_timeout_usec = timeout.totalMicroseconds(); connection_fd_description = fd_description; } @@ -157,8 +157,8 @@ void RemoteQueryExecutorReadContext::setTimer() const /// Did not get packet yet. Init timeout for the next async reading. timer.reset(); - if (receive_timeout.totalMicroseconds()) - timer.setRelative(receive_timeout); + if (receive_timeout_usec) + timer.setRelative(receive_timeout_usec); } bool RemoteQueryExecutorReadContext::resumeRoutine() diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.h b/src/DataStreams/RemoteQueryExecutorReadContext.h index 8cf5f5be1b7..91e34dbb82c 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.h +++ b/src/DataStreams/RemoteQueryExecutorReadContext.h @@ -34,7 +34,8 @@ public: /// This mutex for fiber is needed because fiber could be destroyed in cancel method from another thread. std::mutex fiber_lock; - Poco::Timespan receive_timeout; + /// atomic is required due to data-race between setConnectionFD() and setTimer() from the cancellation path. + std::atomic receive_timeout_usec = 0; IConnections & connections; Poco::Net::Socket * last_used_socket = nullptr; From 9491c0d6509ed6eb125d718d21746450705fd2c9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Aug 2021 10:21:33 +0300 Subject: [PATCH 083/127] Update tests/queries/skip_list.json --- tests/queries/skip_list.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 9d8c3ca979e..56996f0ebc5 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -161,7 +161,7 @@ "00980_zookeeper_merge_tree_alter_settings", "00980_merge_alter_settings", "02009_array_join_partition", - "02012_changed_enum_type_non_replicated", + "02012_changed_enum_type_non_replicated", "02012_zookeeper_changed_enum_type", "02012_zookeeper_changed_enum_type_incompatible", /// Old syntax is not allowed From 875cfb9f9843596d3f4b84c18c6b9af12a42495d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Aug 2021 10:21:38 +0300 Subject: [PATCH 084/127] Update tests/queries/skip_list.json --- tests/queries/skip_list.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 56996f0ebc5..32cb27c85bb 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -162,7 +162,7 @@ "00980_merge_alter_settings", "02009_array_join_partition", "02012_changed_enum_type_non_replicated", - "02012_zookeeper_changed_enum_type", + "02012_zookeeper_changed_enum_type", "02012_zookeeper_changed_enum_type_incompatible", /// Old syntax is not allowed "01062_alter_on_mutataion_zookeeper", From a0b65b35af4bbc7261669b1479e570193b30844b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Aug 2021 10:21:44 +0300 Subject: [PATCH 085/127] Update tests/queries/skip_list.json --- tests/queries/skip_list.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 32cb27c85bb..385339101b5 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -163,7 +163,7 @@ "02009_array_join_partition", "02012_changed_enum_type_non_replicated", "02012_zookeeper_changed_enum_type", - "02012_zookeeper_changed_enum_type_incompatible", + "02012_zookeeper_changed_enum_type_incompatible", /// Old syntax is not allowed "01062_alter_on_mutataion_zookeeper", "00925_zookeeper_empty_replicated_merge_tree_optimize_final", From 0e37203dcd4e560de184cb072cfbaaa63b2fdc5f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Aug 2021 15:27:50 +0300 Subject: [PATCH 086/127] Use jinja template tests in fuzzer --- docker/test/fuzzer/Dockerfile | 4 ++ docker/test/fuzzer/generate-test-j2.py | 62 ++++++++++++++++++++++++++ docker/test/fuzzer/run-fuzzer.sh | 10 +++-- 3 files changed, 72 insertions(+), 4 deletions(-) create mode 100644 docker/test/fuzzer/generate-test-j2.py diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index 18684145636..9a96ac1dfa7 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -16,6 +16,8 @@ RUN apt-get update \ p7zip-full \ parallel \ psmisc \ + python3 \ + python3-pip \ rsync \ tree \ tzdata \ @@ -25,6 +27,8 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* +RUN pip3 install Jinja2 + COPY * / SHELL ["/bin/bash", "-c"] diff --git a/docker/test/fuzzer/generate-test-j2.py b/docker/test/fuzzer/generate-test-j2.py new file mode 100644 index 00000000000..bcc1bf6bc84 --- /dev/null +++ b/docker/test/fuzzer/generate-test-j2.py @@ -0,0 +1,62 @@ +#!/usr/bin/env python3 + +from argparse import ArgumentParser +import os +import jinja2 + + +def removesuffix(text, suffix): + """ + Added in python 3.9 + https://www.python.org/dev/peps/pep-0616/ + """ + if suffix and text.endswith(suffix): + return text[:-len(suffix)] + else: + return text[:] + + +def render_test_template(j2env, suite_dir, test_name): + """ + Render template for test and reference file if needed + """ + + test_base_name = removesuffix(test_name, ".sql.j2") + + reference_file_name = test_base_name + ".reference.j2" + reference_file_path = os.path.join(suite_dir, reference_file_name) + if os.path.isfile(reference_file_path): + tpl = j2env.get_template(reference_file_name) + tpl.stream().dump(os.path.join(suite_dir, test_base_name) + ".gen.reference") + + if test_name.endswith(".sql.j2"): + tpl = j2env.get_template(test_name) + generated_test_name = test_base_name + ".gen.sql" + tpl.stream().dump(os.path.join(suite_dir, generated_test_name)) + return generated_test_name + + return test_name + + +def main(args): + suite_dir = args.path + + print(f"Scanning {suite_dir} directory...") + + j2env = jinja2.Environment( + loader=jinja2.FileSystemLoader(suite_dir), + keep_trailing_newline=True, + ) + + test_names = os.listdir(suite_dir) + for test_name in test_names: + if not test_name.endswith(".sql.j2"): + continue + new_name = render_test_template(j2env, suite_dir, test_name) + print(f"File {new_name} generated") + + +if __name__ == "__main__": + parser = ArgumentParser(description="Jinja2 test generator") + parser.add_argument("-p", "--path", help="Path to test dir", required=True) + main(parser.parse_args()) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 44183a50ae5..8cf85a014a3 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -71,12 +71,12 @@ function watchdog kill -9 -- $fuzzer_pid ||: } -function filter_exists +function filter_exists_and_template { local path for path in "$@"; do if [ -e "$path" ]; then - echo "$path" + echo "$path" | sed -n 's/\.sql\.j2$/.gen.sql/' else echo "'$path' does not exists" >&2 fi @@ -87,9 +87,9 @@ function fuzz { # Obtain the list of newly added tests. They will be fuzzed in more extreme way than other tests. # Don't overwrite the NEW_TESTS_OPT so that it can be set from the environment. - NEW_TESTS="$(sed -n 's!\(^tests/queries/0_stateless/.*\.sql\)$!ch/\1!p' ci-changed-files.txt | sort -R)" + NEW_TESTS="$(sed -n 's!\(^tests/queries/0_stateless/.*\.sql\(\.j2\)\?\)$!ch/\1!p' ci-changed-files.txt | sort -R)" # ci-changed-files.txt contains also files that has been deleted/renamed, filter them out. - NEW_TESTS="$(filter_exists $NEW_TESTS)" + NEW_TESTS="$(filter_exists_and_template $NEW_TESTS)" if [[ -n "$NEW_TESTS" ]] then NEW_TESTS_OPT="${NEW_TESTS_OPT:---interleave-queries-file ${NEW_TESTS}}" @@ -97,6 +97,8 @@ function fuzz NEW_TESTS_OPT="${NEW_TESTS_OPT:-}" fi + /generate-test-j2.py --path ch/tests/queries/0_stateless + export CLICKHOUSE_WATCHDOG_ENABLE=0 # interferes with gdb clickhouse-server --config-file db/config.xml -- --path db 2>&1 | tail -100000 > server.log & server_pid=$! From 9ea6ee7fb2a64f723a5505357799af7bc92a1ca4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Aug 2021 16:15:58 +0300 Subject: [PATCH 087/127] Dummy change in 01720_join_implicit_cast.sql.j2 --- tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 index f2b13e9824b..d1de6d06593 100644 --- a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 +++ b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 @@ -42,7 +42,6 @@ SELECT sum(a) + sum(t_ab2.a) - 1, sum(b) + sum(t_ab2.b) - 1 FROM t_ab1 RIGHT JOI SELECT sum(a) + sum(t_ab2.a) - 1, sum(b) + sum(t_ab2.b) - 1 FROM t_ab1 INNER JOIN t_ab2 ON (t_ab1.a == t_ab2.a AND t_ab1.b == t_ab2.b); SELECT '= types ='; - SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 FULL JOIN t_ab2 USING (a, b); SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 LEFT JOIN t_ab2 USING (a, b); SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 RIGHT JOIN t_ab2 USING (a, b); From b9b0b17ad09673c0ba3dad0e45fae6b743ce5b54 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 25 Aug 2021 11:20:42 +0300 Subject: [PATCH 088/127] chmod +x docker/test/fuzzer/generate-test-j2.py --- docker/test/fuzzer/generate-test-j2.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 docker/test/fuzzer/generate-test-j2.py diff --git a/docker/test/fuzzer/generate-test-j2.py b/docker/test/fuzzer/generate-test-j2.py old mode 100644 new mode 100755 From 6cece7d1d956997ae0202afcc73e83e26b2132be Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 11:22:16 +0300 Subject: [PATCH 089/127] Update build.sh --- docker/packager/deb/build.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index fb9c3f6cab7..46f6404363d 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -39,7 +39,6 @@ then # Launching build script ../docker/packager/other/fuzzer.sh cd - build/docker/packager/other/fuzzer.sh fi ccache --show-config ||: From dc576e952aa657e3c0a2f095e5592e4e5dbce124 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Aug 2021 12:31:02 +0300 Subject: [PATCH 090/127] Update KeeperStorage.cpp --- src/Coordination/KeeperStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 9df8329c5a9..8bffdbe0222 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1156,7 +1156,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina void KeeperStorage::clearDeadWatches(int64_t session_id) { - /// Clear all wathces for this session + /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); if (watches_it != sessions_and_watchers.end()) { From ab22fdaa809e4e98bca7a6ebe89bfcfa03dd96e0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 11:16:06 +0000 Subject: [PATCH 091/127] Fix build --- docker/packager/other/fuzzer.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh index fe4de5802dc..33ad7674961 100755 --- a/docker/packager/other/fuzzer.sh +++ b/docker/packager/other/fuzzer.sh @@ -12,8 +12,10 @@ printenv rm -f CMakeCache.txt read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" # Hope, that the most part of files will be in cache, so we just link new executables -cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_C_COMPILER=$CC -DCMAKE_CXX_COMPILER=$CXX \ - "-DSANITIZE=$SANITIZER" -DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0 \ +cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_C_COMPILER="$CC" -DCMAKE_CXX_COMPILER="$CXX" -DENABLE_CLICKHOUSE_ODBC_BRIDGE=OFF \ + -DENABLE_LIBRARIES=0 -DENABLE_SSL=1 -DUSE_INTERNAL_SSL_LIBRARY=1 -DUSE_UNWIND=ON -DENABLE_EMBEDDED_COMPILER=0 \ + -DENABLE_EXAMPLES=0 -DENABLE_UTILS=0 "-DSANITIZE=$SANITIZER" \ + -DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0 \ -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') From b8ef09a1f84a2eed17e0338dd05cdc1427e00018 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 11:18:57 +0000 Subject: [PATCH 092/127] Fix build --- docker/packager/other/fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh index 33ad7674961..1a8b80c3f77 100755 --- a/docker/packager/other/fuzzer.sh +++ b/docker/packager/other/fuzzer.sh @@ -14,7 +14,7 @@ read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" # Hope, that the most part of files will be in cache, so we just link new executables cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_C_COMPILER="$CC" -DCMAKE_CXX_COMPILER="$CXX" -DENABLE_CLICKHOUSE_ODBC_BRIDGE=OFF \ -DENABLE_LIBRARIES=0 -DENABLE_SSL=1 -DUSE_INTERNAL_SSL_LIBRARY=1 -DUSE_UNWIND=ON -DENABLE_EMBEDDED_COMPILER=0 \ - -DENABLE_EXAMPLES=0 -DENABLE_UTILS=0 "-DSANITIZE=$SANITIZER" \ + -DENABLE_EXAMPLES=0 -DENABLE_UTILS=0 -DENABLE_THINLTO=0 "-DSANITIZE=$SANITIZER" \ -DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0 \ -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. From c6f40c5c2d287c07e96c1c83afe3df765a1d4c0f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 25 Aug 2021 14:19:20 +0300 Subject: [PATCH 093/127] Revert "Revert "less sys calls #2: make vdso work again"" --- base/glibc-compatibility/musl/getauxval.c | 49 +++++++++++++++++------ 1 file changed, 37 insertions(+), 12 deletions(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index a429273fa1a..dad7aa938d7 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -1,4 +1,5 @@ #include +#include "atomic.h" #include // __environ #include @@ -17,18 +18,7 @@ static size_t __find_auxv(unsigned long type) return (size_t) -1; } -__attribute__((constructor)) static void __auxv_init() -{ - size_t i; - for (i = 0; __environ[i]; i++); - __auxv = (unsigned long *) (__environ + i + 1); - - size_t secure_idx = __find_auxv(AT_SECURE); - if (secure_idx != ((size_t) -1)) - __auxv_secure = __auxv[secure_idx]; -} - -unsigned long getauxval(unsigned long type) +unsigned long __getauxval(unsigned long type) { if (type == AT_SECURE) return __auxv_secure; @@ -43,3 +33,38 @@ unsigned long getauxval(unsigned long type) errno = ENOENT; return 0; } + +static void * volatile getauxval_func; + +static unsigned long __auxv_init(unsigned long type) +{ + if (!__environ) + { + // __environ is not initialized yet so we can't initialize __auxv right now. + // That's normally occurred only when getauxval() is called from some sanitizer's internal code. + errno = ENOENT; + return 0; + } + + // Initialize __auxv and __auxv_secure. + size_t i; + for (i = 0; __environ[i]; i++); + __auxv = (unsigned long *) (__environ + i + 1); + + size_t secure_idx = __find_auxv(AT_SECURE); + if (secure_idx != ((size_t) -1)) + __auxv_secure = __auxv[secure_idx]; + + // Now we've initialized __auxv, next time getauxval() will only call __get_auxval(). + a_cas_p(&getauxval_func, (void *)__auxv_init, (void *)__getauxval); + + return __getauxval(type); +} + +// First time getauxval() will call __auxv_init(). +static void * volatile getauxval_func = (void *)__auxv_init; + +unsigned long getauxval(unsigned long type) +{ + return ((unsigned long (*)(unsigned long))getauxval_func)(type); +} From 89a4ca27ae356836deaabe56bc5793e525492127 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Wed, 25 Aug 2021 15:15:39 +0300 Subject: [PATCH 094/127] Apply suggestions from code review Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- docs/ru/operations/settings/settings.md | 4 ++-- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 23200ee78a8..229c8a0d785 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -391,9 +391,9 @@ Functions with a constant argument that is less than ngram size can’t be used ## Projections {#projections} Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. -Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection ](../../../operations/settings/settings.md#force-optimize-projection) setting. +Projections are an experimental feature. To enable them you must set the [allow_experimental_projection_optimization](../../../operations/settings/settings.md#allow-experimental-projection-optimization) to `1`. See also the [force_optimize_projection](../../../operations/settings/settings.md#force-optimize-projection) setting. -Projections are not supported in the `SELECT` statements with [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. +Projections are not supported in the `SELECT` statements with the [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. ### Projection Query {#projection-query} A projection query is what defines a projection. It implicitly selects data from the parent table. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index b5917783a55..1182e97ee3f 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3255,7 +3255,7 @@ SETTINGS index_granularity = 8192 │ Возможные значения: -- 0 — Проекции обязательно используются. -- 1 — Проекции используются опционально. +- 0 — Проекции используются опционально. +- 1 — Проекции обязательно используются. Значение по умолчанию: `0`. \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index 967c36e21d3..4b0d7f7865b 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -15,7 +15,7 @@ toc_title: PROJECTION - `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` — удаляет файлы проекции с диска без удаления описания. -Комманды `ADD`, `DROP` и `CLEAR` — легковесны, поскольку они только меняют метаданные или удаляют файлы. +Команды `ADD`, `DROP` и `CLEAR` — легковесны, поскольку они только меняют метаданные или удаляют файлы. Также команды реплицируются, синхронизируя описания проекций в метаданных с помощью ZooKeeper. From a6b2a6ee36c7bae166c3cf6aa77af23ddf9e9d11 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 25 Aug 2021 20:54:27 +0800 Subject: [PATCH 095/127] Get rid of useless projection columns during merge --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index b6d55828e85..294ce7e61a5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -493,7 +493,6 @@ static void extractMergingAndGatheringColumns( const NamesAndTypesList & storage_columns, const ExpressionActionsPtr & sorting_key_expr, const IndicesDescription & indexes, - const ProjectionsDescription & projections, const MergeTreeData::MergingParams & merging_params, NamesAndTypesList & gathering_columns, Names & gathering_column_names, NamesAndTypesList & merging_columns, Names & merging_column_names) @@ -507,13 +506,6 @@ static void extractMergingAndGatheringColumns( std::inserter(key_columns, key_columns.end())); } - for (const auto & projection : projections) - { - Names projection_columns_vec = projection.required_columns; - std::copy(projection_columns_vec.cbegin(), projection_columns_vec.cend(), - std::inserter(key_columns, key_columns.end())); - } - /// Force sign column for Collapsing mode if (merging_params.mode == MergeTreeData::MergingParams::Collapsing) key_columns.emplace(merging_params.sign_column); @@ -727,7 +719,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor storage_columns, metadata_snapshot->getSortingKey().expression, metadata_snapshot->getSecondaryIndices(), - metadata_snapshot->getProjections(), merging_params, gathering_columns, gathering_column_names, From 92f073a4350213a66700625f6623c20284966c40 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 25 Aug 2021 16:05:16 +0300 Subject: [PATCH 096/127] Call generate-test-j2.py in run-fuzzer.sh before calculating NEW_TESTS_OPT --- docker/test/fuzzer/run-fuzzer.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 8cf85a014a3..9a389edc5b2 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -85,6 +85,8 @@ function filter_exists_and_template function fuzz { + /generate-test-j2.py --path ch/tests/queries/0_stateless + # Obtain the list of newly added tests. They will be fuzzed in more extreme way than other tests. # Don't overwrite the NEW_TESTS_OPT so that it can be set from the environment. NEW_TESTS="$(sed -n 's!\(^tests/queries/0_stateless/.*\.sql\(\.j2\)\?\)$!ch/\1!p' ci-changed-files.txt | sort -R)" @@ -97,8 +99,6 @@ function fuzz NEW_TESTS_OPT="${NEW_TESTS_OPT:-}" fi - /generate-test-j2.py --path ch/tests/queries/0_stateless - export CLICKHOUSE_WATCHDOG_ENABLE=0 # interferes with gdb clickhouse-server --config-file db/config.xml -- --path db 2>&1 | tail -100000 > server.log & server_pid=$! From eecd5b21c9e39bfec3b5299be8b4809ca6c57382 Mon Sep 17 00:00:00 2001 From: lthaooo <354210402@qq.com> Date: Wed, 25 Aug 2021 23:02:13 +0800 Subject: [PATCH 097/127] fix total_size_of_queried_columns bug --- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 2da20073427..806c861cf00 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -47,8 +47,12 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( if (!primary_key.column_names.empty()) first_primary_key_column = primary_key.column_names[0]; - for (const auto & [_, size] : column_sizes) - total_size_of_queried_columns += size; + for (const auto & name : queried_columns) + { + auto it = column_sizes.find(name); + if (it != column_sizes.end()) + total_size_of_queried_columns += it->second; + } determineArrayJoinedNames(query_info.query->as()); optimize(query_info.query->as()); From 31c7e22bd684eec8f97ea614c54c240cbcf0359c Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 25 Aug 2021 18:29:25 +0300 Subject: [PATCH 098/127] correct style --- docs/en/sql-reference/statements/create/table.md | 3 ++- programs/client/Client.cpp | 1 - src/Parsers/ParserInsertQuery.cpp | 5 ----- tests/queries/0_stateless/02009_from_infile.sh | 3 --- 4 files changed, 2 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index c20981b6bbf..d09ff24efcd 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -254,6 +254,7 @@ CREATE TABLE codec_example ENGINE = MergeTree() ``` + ## Temporary Tables {#temporary-tables} ClickHouse supports temporary tables which have the following characteristics: diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index ca7fd98b77e..65e245750b3 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -62,7 +62,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index 8f577288e65..19457f027bf 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -111,11 +111,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { data = pos->begin; } - else if (s_from_infile.ignore(pos, expected)) - { - if (!infile_name_p.parse(pos, infile, expected)) - return false; - } else if (s_format.ignore(pos, expected)) { if (!name_p.parse(pos, format, expected)) diff --git a/tests/queries/0_stateless/02009_from_infile.sh b/tests/queries/0_stateless/02009_from_infile.sh index fa3664eb0f8..578ac14f558 100755 --- a/tests/queries/0_stateless/02009_from_infile.sh +++ b/tests/queries/0_stateless/02009_from_infile.sh @@ -9,14 +9,12 @@ set -e [ -e "${CLICKHOUSE_TMP}"/test_infile.gz ] && rm "${CLICKHOUSE_TMP}"/test_infile.gz [ -e "${CLICKHOUSE_TMP}"/test_infile ] && rm "${CLICKHOUSE_TMP}"/test_infile - echo "Hello" > "${CLICKHOUSE_TMP}"/test_infile gzip "${CLICKHOUSE_TMP}"/test_infile ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_infile;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_infile (word String) ENGINE=Memory();" - ${CLICKHOUSE_CLIENT} --query "INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV;" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_infile;" @@ -28,4 +26,3 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=CREATE" -d 'TABLE test_infile_ur ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "INSERT INTO test_infile_url FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV" 2>&1 | grep -q "UNKNOWN_TYPE_OF_QUERY" && echo "Correct URL" || echo 'Fail' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT x FROM test_infile_url' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=DROP+TABLE" -d 'test_infile_url' - From e63da5969f6526fae57cb2b786ece304c1143a29 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 12:52:24 +0000 Subject: [PATCH 099/127] First blood --- src/Common/examples/YAML_fuzzer.cpp | 39 +++++++++++++++ src/Compression/CMakeLists.txt | 11 ----- src/Compression/CompressionCodecDelta.cpp | 6 +++ .../CompressionCodecDoubleDelta.cpp | 1 + src/Compression/CompressionCodecLZ4.cpp | 6 +++ src/Compression/ICompressionCodec.h | 4 ++ src/Compression/fuzzers/CMakeLists.txt | 10 +++- .../fuzzers/delta_decompress_fuzzer.cpp | 47 ++++++++++++++++++ .../fuzzers/lz4_decompress_fuzzer.cpp | 48 +++++++++++++++++++ 9 files changed, 159 insertions(+), 13 deletions(-) create mode 100644 src/Common/examples/YAML_fuzzer.cpp create mode 100644 src/Compression/fuzzers/delta_decompress_fuzzer.cpp create mode 100644 src/Compression/fuzzers/lz4_decompress_fuzzer.cpp diff --git a/src/Common/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp new file mode 100644 index 00000000000..63908245a66 --- /dev/null +++ b/src/Common/examples/YAML_fuzzer.cpp @@ -0,0 +1,39 @@ +#include +#include +#include +#include +#include +#include + +#include + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +{ + /// How to test: + /// build ClickHouse with YAML_fuzzer.cpp + /// ./YAML_fuzzer YAML_CORPUS + /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer + char file_name[L_tmpnam]; + if (!std::tmpnam(file_name)) + { + std::cerr << "Cannot create temp file!\n"; + return 1; + } + std::string input = std::string(reinterpret_cast(data), size); + + { + std::ofstream temp_file(file_name); + temp_file << input; + } + + try + { + DB::YAMLParserImpl::parse(std::string(file_name)); + } + catch (...) + { + std::cerr << "YAML_fuzzer failed: " << DB::getCurrentExceptionMessage(__PRETTY_FUNCTION__) << std::endl; + return 1; + } + return 0; +} diff --git a/src/Compression/CMakeLists.txt b/src/Compression/CMakeLists.txt index 34369d8dbc8..563a77bc168 100644 --- a/src/Compression/CMakeLists.txt +++ b/src/Compression/CMakeLists.txt @@ -1,14 +1,3 @@ -if (ENABLE_FUZZING) - include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") - add_headers_and_sources(fuzz_compression .) - - # Remove this file, because it has dependencies on DataTypes - list(REMOVE_ITEM ${fuzz_compression_sources} CompressionFactoryAdditions.cpp) - - add_library(fuzz_compression ${fuzz_compression_headers} ${fuzz_compression_sources}) - target_link_libraries(fuzz_compression PUBLIC clickhouse_parsers clickhouse_common_io common lz4) -endif() - if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index e281609ff43..d3f941efdd1 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -209,4 +209,10 @@ void registerCodecDelta(CompressionCodecFactory & factory) return std::make_shared(delta_bytes_size); }); } + +CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size) +{ + return std::make_shared(delta_bytes_size); +} + } diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index c416582eb6b..935efad5806 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -543,4 +543,5 @@ void registerCodecDoubleDelta(CompressionCodecFactory & factory) return std::make_shared(data_bytes_size); }); } + } diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index 396f6fad2c3..12f138dc95a 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -147,4 +147,10 @@ CompressionCodecLZ4HC::CompressionCodecLZ4HC(int level_) setCodecDescription("LZ4HC", {std::make_shared(static_cast(level))}); } + +CompressionCodecPtr getCompressionCodecLZ4(int level) +{ + return std::make_shared(level); +} + } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index c49c16d8bad..dfeff3e77c8 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -18,6 +18,8 @@ using Codecs = std::vector; class IDataType; +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); + /** * Represents interface for compression codecs like LZ4, ZSTD, etc. */ @@ -85,6 +87,8 @@ public: protected: + friend int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); + /// Return size of compressed data without header virtual UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const { return uncompressed_size; } diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt index 73b95d09fa3..3b8e056e000 100644 --- a/src/Compression/fuzzers/CMakeLists.txt +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -1,8 +1,14 @@ -add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) - # Our code has strong cohesion and target associated with `Compression` also depends on `DataTypes`. # But we can exclude some files which have dependencies in case of # fuzzer related build (we are interested in fuzzing only particular part of our code). # So, some symbols will be declared, but not defined. Unfortunately, this trick doesn't work with UBSan. # If you want really small size of the resulted binary, just link with fuzz_compression and clickhouse_common_io + +add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) + +add_executable (lz4_decompress_fuzzer lz4_decompress_fuzzer.cpp) +target_link_libraries (lz4_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) + +add_executable (delta_decompress_fuzzer delta_decompress_fuzzer.cpp) +target_link_libraries (delta_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp new file mode 100644 index 00000000000..861d503712a --- /dev/null +++ b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp @@ -0,0 +1,47 @@ +#include +#include + +#include +#include + +namespace DB +{ + CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); +} + +struct AuxiliaryRandomData +{ + UInt8 delta_size_bytes; + size_t decompressed_size; +}; + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + if (size < sizeof(AuxiliaryRandomData)) + return 0; + + auto * p = reinterpret_cast(data); + auto codec = DB::getCompressionCodecDelta(p->delta_size_bytes); + + size_t output_buffer_size = p->decompressed_size % 65536; + size -= sizeof(AuxiliaryRandomData); + data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); + + std::string input = std::string(reinterpret_cast(data), size); + fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); + + if (output_buffer_size < size) + return 0; + + DB::Memory<> memory; + memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); + + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); + + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp new file mode 100644 index 00000000000..12cec0e20d5 --- /dev/null +++ b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp @@ -0,0 +1,48 @@ +#include +#include + +#include +#include + +namespace DB +{ + CompressionCodecPtr getCompressionCodecLZ4(int level); +} + +struct AuxiliaryRandomData +{ + size_t level; + size_t decompressed_size; +}; + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + + if (size < sizeof(AuxiliaryRandomData)) + return 0; + + auto * p = reinterpret_cast(data); + auto codec = DB::getCompressionCodecLZ4(p->level); + + size_t output_buffer_size = p->decompressed_size % 65536; + size -= sizeof(AuxiliaryRandomData); + data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); + + std::string input = std::string(reinterpret_cast(data), size); + fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); + + if (output_buffer_size < size) + return 0; + + DB::Memory<> memory; + memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); + + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); + + return 0; +} +catch (...) +{ + return 1; +} From 96b9635c2831efc187607a698811fb6f76c54f5f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Aug 2021 17:52:54 +0300 Subject: [PATCH 100/127] Update CMakeLists.txt --- src/Common/examples/CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 64d28fec5c2..46619aaf603 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -80,3 +80,8 @@ target_link_libraries (average PRIVATE clickhouse_common_io) add_executable (shell_command_inout shell_command_inout.cpp) target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) + +if (ENABLE_FUZZING) + add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS}) + target_link_libraries(YAML_fuzzer PRIVATE clickhouse_common_config ${LIB_FUZZING_ENGINE}) +endif () From ad4966f6d2bf8430d55e8b489334b0a25516599e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 15:19:07 +0000 Subject: [PATCH 101/127] better --- .../CompressionCodecDoubleDelta.cpp | 5 ++ src/Compression/CompressionCodecEncrypted.h | 9 +++- src/Compression/CompressionCodecMultiple.h | 2 +- src/Compression/CompressionCodecZSTD.cpp | 5 ++ src/Compression/ICompressionCodec.h | 2 +- src/Compression/fuzzers/CMakeLists.txt | 9 ++++ .../fuzzers/delta_decompress_fuzzer.cpp | 2 +- .../double_delta_decompress_fuzzer.cpp | 47 +++++++++++++++++ .../fuzzers/encrypted_decompress_fuzzer.cpp | 51 +++++++++++++++++++ .../fuzzers/lz4_decompress_fuzzer.cpp | 2 +- .../fuzzers/zstd_decompress_fuzzer.cpp | 47 +++++++++++++++++ src/Core/CMakeLists.txt | 2 +- src/Parsers/CMakeLists.txt | 2 +- src/Storages/examples/CMakeLists.txt | 1 - src/Storages/fuzzers/CMakeLists.txt | 1 - 15 files changed, 177 insertions(+), 10 deletions(-) create mode 100644 src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp create mode 100644 src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp create mode 100644 src/Compression/fuzzers/zstd_decompress_fuzzer.cpp diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 935efad5806..e781f9e5cb8 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -544,4 +544,9 @@ void registerCodecDoubleDelta(CompressionCodecFactory & factory) }); } +CompressionCodecPtr getCompressionCodecDoubleDelta(UInt8 data_bytes_size) +{ + return std::make_shared(data_bytes_size); +} + } diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index bacd58bcd2f..9875d5c1e61 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -51,7 +51,7 @@ namespace DB */ static void setMasterKey(const std::string_view & master_key); - CompressionCodecEncrypted(const std::string_view & cipher); + explicit CompressionCodecEncrypted(const std::string_view & cipher); uint8_t getMethodByte() const override; void updateHash(SipHash & hash) const override; @@ -88,7 +88,7 @@ namespace DB */ struct KeyHolder : private boost::noncopyable { - KeyHolder(const std::string_view & master_key); + explicit KeyHolder(const std::string_view & master_key); ~KeyHolder(); std::string keygen_key; @@ -99,6 +99,11 @@ namespace DB static inline std::optional keys; }; + + CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key) + { + return std::make_shared(master_key); + } } #endif /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */ diff --git a/src/Compression/CompressionCodecMultiple.h b/src/Compression/CompressionCodecMultiple.h index 1eb61842048..1d63fe1da55 100644 --- a/src/Compression/CompressionCodecMultiple.h +++ b/src/Compression/CompressionCodecMultiple.h @@ -9,7 +9,7 @@ class CompressionCodecMultiple final : public ICompressionCodec { public: CompressionCodecMultiple() = default; /// Need for CompressionFactory to register codec by method byte. - CompressionCodecMultiple(Codecs codecs_); + explicit CompressionCodecMultiple(Codecs codecs_); uint8_t getMethodByte() const override; diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index a950f1fee74..def2188d90a 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -156,4 +156,9 @@ void registerCodecZSTD(CompressionCodecFactory & factory) }); } +CompressionCodecPtr getCompressionCodecZSTD(int level) +{ + return std::make_shared(level); +} + } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index dfeff3e77c8..3f603087f2b 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -86,7 +86,7 @@ public: virtual bool isNone() const { return false; } protected: - + /// This is used for fuzz testing friend int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); /// Return size of compressed data without header diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt index 3b8e056e000..f36921b4a6b 100644 --- a/src/Compression/fuzzers/CMakeLists.txt +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -12,3 +12,12 @@ target_link_libraries (lz4_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) add_executable (delta_decompress_fuzzer delta_decompress_fuzzer.cpp) target_link_libraries (delta_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) + +add_executable (double_delta_decompress_fuzzer double_delta_decompress_fuzzer.cpp) +target_link_libraries (double_delta_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) + +add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp) +target_link_libraries (encrypted_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) + +add_executable (zstd_decompress_fuzzer zstd_decompress_fuzzer.cpp) +target_link_libraries (zstd_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp index 861d503712a..2df237d3239 100644 --- a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp @@ -21,7 +21,7 @@ try if (size < sizeof(AuxiliaryRandomData)) return 0; - auto * p = reinterpret_cast(data); + const auto * p = reinterpret_cast(data); auto codec = DB::getCompressionCodecDelta(p->delta_size_bytes); size_t output_buffer_size = p->decompressed_size % 65536; diff --git a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp new file mode 100644 index 00000000000..61a66c22b16 --- /dev/null +++ b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp @@ -0,0 +1,47 @@ +#include +#include + +#include +#include + +namespace DB +{ + CompressionCodecPtr getCompressionCodecDoubleDelta(UInt8 data_bytes_size); +} + +struct AuxiliaryRandomData +{ + UInt8 data_bytes_size; + size_t decompressed_size; +}; + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + if (size < sizeof(AuxiliaryRandomData)) + return 0; + + const auto * p = reinterpret_cast(data); + auto codec = DB::getCompressionCodecDoubleDelta(p->data_bytes_size); + + size_t output_buffer_size = p->decompressed_size % 65536; + size -= sizeof(AuxiliaryRandomData); + data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); + + std::string input = std::string(reinterpret_cast(data), size); + fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); + + if (output_buffer_size < size) + return 0; + + DB::Memory<> memory; + memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); + + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); + + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp new file mode 100644 index 00000000000..5dda1998029 --- /dev/null +++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp @@ -0,0 +1,51 @@ +#include +#include + +#include +#include + +namespace DB +{ + CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key); +} + +constexpr size_t key_size = 20; + +struct AuxiliaryRandomData +{ + char key[key_size]; + size_t decompressed_size; +}; + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + if (size < sizeof(AuxiliaryRandomData)) + return 0; + + const auto * p = reinterpret_cast(data); + + std::string key = std::string(p->key, key_size); + auto codec = DB::getCompressionCodecEncrypted(key); + + size_t output_buffer_size = p->decompressed_size % 65536; + size -= sizeof(AuxiliaryRandomData); + data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); + + std::string input = std::string(reinterpret_cast(data), size); + fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); + + if (output_buffer_size < size) + return 0; + + DB::Memory<> memory; + memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); + + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); + + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp index 12cec0e20d5..9fcfe739728 100644 --- a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp @@ -22,7 +22,7 @@ try if (size < sizeof(AuxiliaryRandomData)) return 0; - auto * p = reinterpret_cast(data); + const auto * p = reinterpret_cast(data); auto codec = DB::getCompressionCodecLZ4(p->level); size_t output_buffer_size = p->decompressed_size % 65536; diff --git a/src/Compression/fuzzers/zstd_decompress_fuzzer.cpp b/src/Compression/fuzzers/zstd_decompress_fuzzer.cpp new file mode 100644 index 00000000000..2c214fd4329 --- /dev/null +++ b/src/Compression/fuzzers/zstd_decompress_fuzzer.cpp @@ -0,0 +1,47 @@ +#include +#include + +#include +#include + +namespace DB +{ + CompressionCodecPtr getCompressionCodecZSTD(int level); +} + +struct AuxiliaryRandomData +{ + int level; + size_t decompressed_size; +}; + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + if (size < sizeof(AuxiliaryRandomData)) + return 0; + + const auto * p = reinterpret_cast(data); + auto codec = DB::getCompressionCodecZSTD(p->level); + + size_t output_buffer_size = p->decompressed_size % 65536; + size -= sizeof(AuxiliaryRandomData); + data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); + + std::string input = std::string(reinterpret_cast(data), size); + fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); + + if (output_buffer_size < size) + return 0; + + DB::Memory<> memory; + memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); + + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); + + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/Core/CMakeLists.txt b/src/Core/CMakeLists.txt index 85e2008753d..52e68e4dd73 100644 --- a/src/Core/CMakeLists.txt +++ b/src/Core/CMakeLists.txt @@ -4,4 +4,4 @@ endif () if (ENABLE_FUZZING) add_subdirectory(fuzzers) -endif() +endif () diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index a20dd3567a9..2f48fb0c548 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -9,7 +9,7 @@ if (USE_DEBUG_HELPERS) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") endif () -if(ENABLE_EXAMPLES) +if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() diff --git a/src/Storages/examples/CMakeLists.txt b/src/Storages/examples/CMakeLists.txt index cb03ae751e3..103972a106f 100644 --- a/src/Storages/examples/CMakeLists.txt +++ b/src/Storages/examples/CMakeLists.txt @@ -22,4 +22,3 @@ target_link_libraries (transform_part_zk_nodes dbms string_utils ) - diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt index dcbd07ccd91..d41e96868ad 100644 --- a/src/Storages/fuzzers/CMakeLists.txt +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -1,6 +1,5 @@ add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.cpp) - # Look at comment around fuzz_compression target declaration target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) From 82a0c9d4df9f200ff658d9c47c7d44ae074fefab Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 23:47:43 +0000 Subject: [PATCH 102/127] Fix build --- src/Compression/CompressionCodecEncrypted.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index 9875d5c1e61..cef9b2e6072 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -100,7 +100,7 @@ namespace DB static inline std::optional keys; }; - CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key) + inline CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key) { return std::make_shared(master_key); } From 2c09533c831fc63b57d5da50788dcb08f8d02716 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 20 Aug 2021 16:22:12 +0000 Subject: [PATCH 103/127] lower a binary size for compression fuzzers --- src/Compression/CMakeLists.txt | 9 + src/Compression/CompressionCodecDelta.cpp | 6 +- .../CompressionCodecDoubleDelta.cpp | 6 +- src/Compression/CompressionCodecGorilla.cpp | 6 +- src/Compression/CompressionCodecT64.cpp | 3 +- src/Compression/CompressionFactory.h | 2 + src/Compression/examples/CMakeLists.txt | 4 +- src/Compression/fuzzers/CMakeLists.txt | 13 +- .../fuzzers/encrypted_decompress_fuzzer.cpp | 1 + .../fuzzers/zstd_decompress_fuzzer.cpp | 47 ---- .../СompressionFactoryAdditions.cpp | 206 ++++++++++++++++++ .../Serializations/ISerialization.cpp | 1 + src/Storages/ColumnsDescription.cpp | 3 + 13 files changed, 237 insertions(+), 70 deletions(-) delete mode 100644 src/Compression/fuzzers/zstd_decompress_fuzzer.cpp create mode 100644 src/Compression/СompressionFactoryAdditions.cpp diff --git a/src/Compression/CMakeLists.txt b/src/Compression/CMakeLists.txt index 563a77bc168..16c93d060cd 100644 --- a/src/Compression/CMakeLists.txt +++ b/src/Compression/CMakeLists.txt @@ -1,3 +1,12 @@ +include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") +add_headers_and_sources(clickhouse_compression .) + +# Remove this file, because it has dependencies on DataTypes +list(REMOVE_ITEM ${clickhouse_compression_sources} CompressionFactoryAdditions.cpp) + +add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources}) +target_link_libraries(clickhouse_compression PUBLIC clickhouse_parsers clickhouse_common_io common lz4) + if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index d3f941efdd1..4dd50128c6a 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -166,15 +166,13 @@ namespace UInt8 getDeltaBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is not applicable for {} because the data type is not of fixed size", - column_type->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is not applicable for because the data type is not of fixed size"); size_t max_size = column_type->getSizeOfValueInMemory(); if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", - column_type->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes"); } } diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index e781f9e5cb8..6c73a96586a 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -427,15 +427,13 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) UInt8 getDataBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec DoubleDelta is not applicable for {} because the data type is not of fixed size", - column_type->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec DoubleDelta is not applicable for {} because the data type is not of fixed size"); size_t max_size = column_type->getSizeOfValueInMemory(); if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", - column_type->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes"); } } diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 1276ac911f1..6aabc6b8746 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -334,15 +334,13 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) UInt8 getDataBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Gorilla is not applicable for {} because the data type is not of fixed size", - column_type->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Gorilla is not applicable for because the data type is not of fixed size"); size_t max_size = column_type->getSizeOfValueInMemory(); if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", - column_type->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes"); } } diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index 377db915bce..302fce833c5 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -728,7 +728,8 @@ void registerCodecT64(CompressionCodecFactory & factory) auto type_idx = typeIdx(type); if (type && type_idx == TypeIndex::Nothing) - throw Exception("T64 codec is not supported for specified type " + type->getName(), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + throw Exception("T64 codec is not supported for specified type ", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + return std::make_shared(type_idx, variant); }; diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index f00e5071990..23604cf8f01 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -22,6 +22,8 @@ using CompressionCodecPtr = std::shared_ptr; using CodecNameWithLevel = std::pair>; +static constexpr auto DEFAULT_CODEC_NAME = "Default"; + /** Creates a codec object by name of compression algorithm family and parameters. */ class CompressionCodecFactory final : private boost::noncopyable diff --git a/src/Compression/examples/CMakeLists.txt b/src/Compression/examples/CMakeLists.txt index 3cfc0ccb7dc..04c17f12b42 100644 --- a/src/Compression/examples/CMakeLists.txt +++ b/src/Compression/examples/CMakeLists.txt @@ -1,5 +1,5 @@ add_executable (compressed_buffer compressed_buffer.cpp) -target_link_libraries (compressed_buffer PRIVATE dbms) +target_link_libraries (compressed_buffer PRIVATE clickhouse_compression common) add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) -target_link_libraries (cached_compressed_read_buffer PRIVATE dbms) +target_link_libraries (cached_compressed_read_buffer PRIVATE clickhouse_compression common) diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt index f36921b4a6b..e677e158a20 100644 --- a/src/Compression/fuzzers/CMakeLists.txt +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -5,19 +5,16 @@ # If you want really small size of the resulted binary, just link with fuzz_compression and clickhouse_common_io add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) -target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries (compressed_buffer_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) add_executable (lz4_decompress_fuzzer lz4_decompress_fuzzer.cpp) -target_link_libraries (lz4_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries (lz4_decompress_fuzzer PUBLIC clickhouse_compression lz4 ${LIB_FUZZING_ENGINE}) add_executable (delta_decompress_fuzzer delta_decompress_fuzzer.cpp) -target_link_libraries (delta_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries (delta_decompress_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) add_executable (double_delta_decompress_fuzzer double_delta_decompress_fuzzer.cpp) -target_link_libraries (double_delta_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries (double_delta_decompress_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp) -target_link_libraries (encrypted_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) - -add_executable (zstd_decompress_fuzzer zstd_decompress_fuzzer.cpp) -target_link_libraries (zstd_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries (encrypted_decompress_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp index 5dda1998029..6211fd4b9cc 100644 --- a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB diff --git a/src/Compression/fuzzers/zstd_decompress_fuzzer.cpp b/src/Compression/fuzzers/zstd_decompress_fuzzer.cpp deleted file mode 100644 index 2c214fd4329..00000000000 --- a/src/Compression/fuzzers/zstd_decompress_fuzzer.cpp +++ /dev/null @@ -1,47 +0,0 @@ -#include -#include - -#include -#include - -namespace DB -{ - CompressionCodecPtr getCompressionCodecZSTD(int level); -} - -struct AuxiliaryRandomData -{ - int level; - size_t decompressed_size; -}; - -extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) -try -{ - if (size < sizeof(AuxiliaryRandomData)) - return 0; - - const auto * p = reinterpret_cast(data); - auto codec = DB::getCompressionCodecZSTD(p->level); - - size_t output_buffer_size = p->decompressed_size % 65536; - size -= sizeof(AuxiliaryRandomData); - data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); - - std::string input = std::string(reinterpret_cast(data), size); - fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); - - if (output_buffer_size < size) - return 0; - - DB::Memory<> memory; - memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); - - codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); - - return 0; -} -catch (...) -{ - return 1; -} diff --git a/src/Compression/СompressionFactoryAdditions.cpp b/src/Compression/СompressionFactoryAdditions.cpp new file mode 100644 index 00000000000..90ad45d10d6 --- /dev/null +++ b/src/Compression/СompressionFactoryAdditions.cpp @@ -0,0 +1,206 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNEXPECTED_AST_STRUCTURE; + extern const int UNKNOWN_CODEC; +} + + +void CompressionCodecFactory::validateCodec( + const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const +{ + if (family_name.empty()) + throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS); + + if (level) + { + auto literal = std::make_shared(static_cast(*level)); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), + {}, sanity_check, allow_experimental_codecs); + } + else + { + auto identifier = std::make_shared(Poco::toUpper(family_name)); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), + {}, sanity_check, allow_experimental_codecs); + } +} + +ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( + const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const +{ + if (const auto * func = ast->as()) + { + ASTPtr codecs_descriptions = std::make_shared(); + + bool is_compression = false; + bool has_none = false; + std::optional generic_compression_codec_pos; + std::set post_processing_codecs; + + bool can_substitute_codec_arguments = true; + for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) + { + const auto & inner_codec_ast = func->arguments->children[i]; + String codec_family_name; + ASTPtr codec_arguments; + if (const auto * family_name = inner_codec_ast->as()) + { + codec_family_name = family_name->name(); + codec_arguments = {}; + } + else if (const auto * ast_func = inner_codec_ast->as()) + { + codec_family_name = ast_func->name; + codec_arguments = ast_func->arguments; + } + else + throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + + /// Default codec replaced with current default codec which may depend on different + /// settings (and properties of data) in runtime. + CompressionCodecPtr result_codec; + if (codec_family_name == DEFAULT_CODEC_NAME) + { + if (codec_arguments != nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME); + + result_codec = default_codec; + codecs_descriptions->children.emplace_back(std::make_shared(DEFAULT_CODEC_NAME)); + } + else + { + if (column_type) + { + CompressionCodecPtr prev_codec; + IDataType::StreamCallbackWithType callback = [&]( + const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) + { + if (ISerialization::isSpecialCompressionAllowed(substream_path)) + { + result_codec = getImpl(codec_family_name, codec_arguments, &substream_type); + + /// Case for column Tuple, which compressed with codec which depends on data type, like Delta. + /// We cannot substitute parameters for such codecs. + if (prev_codec && prev_codec->getHash() != result_codec->getHash()) + can_substitute_codec_arguments = false; + prev_codec = result_codec; + } + }; + + ISerialization::SubstreamPath stream_path; + column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path); + + if (!result_codec) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); + } + else + { + result_codec = getImpl(codec_family_name, codec_arguments, nullptr); + } + + if (!allow_experimental_codecs && result_codec->isExperimental()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Codec {} is experimental and not meant to be used in production." + " You can enable it with the 'allow_experimental_codecs' setting.", + codec_family_name); + + codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); + } + + is_compression |= result_codec->isCompression(); + has_none |= result_codec->isNone(); + + if (!generic_compression_codec_pos && result_codec->isGenericCompression()) + generic_compression_codec_pos = i; + + if (result_codec->isPostProcessing()) + post_processing_codecs.insert(i); + } + + String codec_description = queryToString(codecs_descriptions); + + if (sanity_check) + { + if (codecs_descriptions->children.size() > 1 && has_none) + throw Exception( + "It does not make sense to have codec NONE along with other compression codecs: " + codec_description + + ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", + ErrorCodes::BAD_ARGUMENTS); + + /// Allow to explicitly specify single NONE codec if user don't want any compression. + /// But applying other transformations solely without compression (e.g. Delta) does not make sense. + /// It's okay to apply post-processing codecs solely without anything else. + if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size()) + throw Exception( + "Compression codec " + codec_description + + " does not compress anything." + " You may want to add generic compression algorithm after other transformations, like: " + + codec_description + + ", LZ4." + " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", + ErrorCodes::BAD_ARGUMENTS); + + /// It does not make sense to apply any non-post-processing codecs + /// after post-processing one. + if (!post_processing_codecs.empty() && + *post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size()) + throw Exception("The combination of compression codecs " + codec_description + " is meaningless," + " because it does not make sense to apply any non-post-processing codecs after" + " post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'" + " to skip this check).", ErrorCodes::BAD_ARGUMENTS); + + /// It does not make sense to apply any transformations after generic compression algorithm + /// So, generic compression can be only one and only at the end. + if (generic_compression_codec_pos && + *generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size()) + throw Exception("The combination of compression codecs " + codec_description + " is meaningless," + " because it does not make sense to apply any transformations after generic compression algorithm." + " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); + + } + + /// For columns with nested types like Tuple(UInt32, UInt64) we + /// obviously cannot substitute parameters for codecs which depend on + /// data type, because for the first column Delta(4) is suitable and + /// Delta(8) for the second. So we should leave codec description as is + /// and deduce them in get method for each subtype separately. For all + /// other types it's better to substitute parameters, for better + /// readability and backward compatibility. + if (can_substitute_codec_arguments) + { + std::shared_ptr result = std::make_shared(); + result->name = "CODEC"; + result->arguments = codecs_descriptions; + return result; + } + else + { + return ast; + } + } + + throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); +} + + + +} diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index ab2e8e1958b..7077c5bfa14 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index c05441148df..fabea01441f 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include #include @@ -688,4 +690,5 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N } } + } From 08fbab09b023e32db1c4a09dfb161e394492d561 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 21 Aug 2021 22:58:33 +0000 Subject: [PATCH 104/127] Fix some issues --- src/Compression/CompressionCodecDelta.cpp | 16 +++++++++++----- .../CompressionCodecDoubleDelta.cpp | 18 +++++++++++++----- src/Compression/LZ4_decompress_faster.cpp | 11 +++++++++++ .../fuzzers/delta_decompress_fuzzer.cpp | 7 ++----- .../fuzzers/double_delta_decompress_fuzzer.cpp | 7 ++----- .../fuzzers/lz4_decompress_fuzzer.cpp | 7 ++----- 6 files changed, 41 insertions(+), 25 deletions(-) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 4dd50128c6a..b995a425429 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -41,6 +41,7 @@ namespace ErrorCodes extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int ILLEGAL_CODEC_PARAMETER; extern const int BAD_ARGUMENTS; + extern const int CANNOT_DECOMPRESS; } CompressionCodecDelta::CompressionCodecDelta(UInt8 delta_bytes_size_) @@ -82,8 +83,10 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest) } template -void decompressDataForType(const char * source, UInt32 source_size, char * dest) +void decompressDataForType(const char * source, UInt32 source_size, char * dest, UInt32 output_size) { + const char * output_end = dest + output_size; + if (source_size % sizeof(T) != 0) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot delta decompress, data size {} is not aligned to {}", source_size, sizeof(T)); @@ -92,6 +95,8 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) while (source < source_end) { accumulator += unalignedLoad(source); + if (dest + sizeof(accumulator) > output_end) + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); unalignedStore(dest, accumulator); source += sizeof(T); @@ -137,6 +142,7 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_ throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); UInt8 bytes_to_skip = uncompressed_size % bytes_size; + UInt32 output_size = uncompressed_size - bytes_to_skip; if (UInt32(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); @@ -146,16 +152,16 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_ switch (bytes_size) { case 1: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; case 2: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; case 4: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; case 8: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; } } diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 6c73a96586a..0818b951036 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -353,12 +353,13 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest) } template -void decompressDataForType(const char * source, UInt32 source_size, char * dest) +void decompressDataForType(const char * source, UInt32 source_size, char * dest, UInt32 output_size) { static_assert(is_unsigned_v, "ValueType must be unsigned."); using UnsignedDeltaType = ValueType; const char * source_end = source + source_size; + const char * output_end = dest + output_size; if (source + sizeof(UInt32) > source_end) return; @@ -374,6 +375,8 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) return; prev_value = unalignedLoad(source); + if (dest + sizeof(prev_value) > output_end) + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); unalignedStore(dest, prev_value); source += sizeof(prev_value); @@ -385,6 +388,8 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) prev_delta = unalignedLoad(source); prev_value = prev_value + static_cast(prev_delta); + if (dest + sizeof(prev_value) > output_end) + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); unalignedStore(dest, prev_value); source += sizeof(prev_delta); @@ -416,6 +421,8 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) const UnsignedDeltaType delta = double_delta + prev_delta; const ValueType curr_value = prev_value + delta; + if (dest + sizeof(curr_value) > output_end) + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); unalignedStore(dest, curr_value); dest += sizeof(curr_value); @@ -505,6 +512,7 @@ void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 s throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); UInt8 bytes_to_skip = uncompressed_size % bytes_size; + UInt32 output_size = uncompressed_size - bytes_to_skip; if (UInt32(2 + bytes_to_skip) > source_size) throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); @@ -514,16 +522,16 @@ void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 s switch (bytes_size) { case 1: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; case 2: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; case 4: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; case 8: - decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip]); + decompressDataForType(&source[2 + bytes_to_skip], source_size_no_header, &dest[bytes_to_skip], output_size); break; } } diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 28a285f00f4..bbe3e899620 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -450,7 +450,11 @@ bool NO_INLINE decompressImpl( const unsigned token = *ip++; length = token >> 4; if (length == 0x0F) + { + if (unlikely(ip + 1 >= input_end)) + return false; continue_read_length(); + } /// Copy literals. @@ -470,6 +474,9 @@ bool NO_INLINE decompressImpl( if (unlikely(copy_end > output_end)) return false; + if (unlikely(ip + std::max(copy_amount, static_cast(std::ceil(static_cast(length) / copy_amount) * copy_amount)) >= input_end)) + return false; + wildCopy(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer. if (copy_end == output_end) @@ -494,7 +501,11 @@ bool NO_INLINE decompressImpl( length = token & 0x0F; if (length == 0x0F) + { + if (unlikely(ip + 1 >= input_end)) + return false; continue_read_length(); + } length += 4; /// Copy match within block, that produce overlapping pattern. Match may replicate itself. diff --git a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp index 2df237d3239..b039777da15 100644 --- a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp @@ -28,11 +28,8 @@ try size -= sizeof(AuxiliaryRandomData); data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); - std::string input = std::string(reinterpret_cast(data), size); - fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); - - if (output_buffer_size < size) - return 0; + // std::string input = std::string(reinterpret_cast(data), size); + // fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); DB::Memory<> memory; memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); diff --git a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp index 61a66c22b16..f9822daa3bd 100644 --- a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp @@ -28,11 +28,8 @@ try size -= sizeof(AuxiliaryRandomData); data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); - std::string input = std::string(reinterpret_cast(data), size); - fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); - - if (output_buffer_size < size) - return 0; + // std::string input = std::string(reinterpret_cast(data), size); + // fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); DB::Memory<> memory; memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); diff --git a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp index 9fcfe739728..f13c776cda9 100644 --- a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp @@ -29,11 +29,8 @@ try size -= sizeof(AuxiliaryRandomData); data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); - std::string input = std::string(reinterpret_cast(data), size); - fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); - - if (output_buffer_size < size) - return 0; + // std::string input = std::string(reinterpret_cast(data), size); + // fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); DB::Memory<> memory; memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); From 3538f1520b9dae090eef3e1e583a3da81d85919d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 21 Aug 2021 23:08:04 +0000 Subject: [PATCH 105/127] Build fixes after merge --- src/Compression/CompressionFactory.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 23604cf8f01..f00e5071990 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -22,8 +22,6 @@ using CompressionCodecPtr = std::shared_ptr; using CodecNameWithLevel = std::pair>; -static constexpr auto DEFAULT_CODEC_NAME = "Default"; - /** Creates a codec object by name of compression algorithm family and parameters. */ class CompressionCodecFactory final : private boost::noncopyable From 43fb3098aeeb4627934012ec76e041eba3f8bc13 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 21 Aug 2021 23:16:08 +0000 Subject: [PATCH 106/127] Deleted some unneded changes --- src/Common/examples/CMakeLists.txt | 5 --- src/Common/examples/YAML_fuzzer.cpp | 39 ------------------- src/Compression/CMakeLists.txt | 14 ++++--- src/Compression/CompressionCodecDelta.cpp | 6 ++- .../CompressionCodecDoubleDelta.cpp | 6 ++- src/Compression/CompressionCodecGorilla.cpp | 6 ++- src/Compression/CompressionCodecT64.cpp | 3 +- src/Compression/examples/CMakeLists.txt | 4 +- src/Compression/fuzzers/CMakeLists.txt | 10 ++--- src/Core/CMakeLists.txt | 2 +- src/Parsers/CMakeLists.txt | 2 +- src/Storages/ColumnsDescription.cpp | 3 -- 12 files changed, 30 insertions(+), 70 deletions(-) delete mode 100644 src/Common/examples/YAML_fuzzer.cpp diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 46619aaf603..64d28fec5c2 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -80,8 +80,3 @@ target_link_libraries (average PRIVATE clickhouse_common_io) add_executable (shell_command_inout shell_command_inout.cpp) target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) - -if (ENABLE_FUZZING) - add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS}) - target_link_libraries(YAML_fuzzer PRIVATE clickhouse_common_config ${LIB_FUZZING_ENGINE}) -endif () diff --git a/src/Common/examples/YAML_fuzzer.cpp b/src/Common/examples/YAML_fuzzer.cpp deleted file mode 100644 index 63908245a66..00000000000 --- a/src/Common/examples/YAML_fuzzer.cpp +++ /dev/null @@ -1,39 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#include - -extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) -{ - /// How to test: - /// build ClickHouse with YAML_fuzzer.cpp - /// ./YAML_fuzzer YAML_CORPUS - /// where YAML_CORPUS is a directory with different YAML configs for libfuzzer - char file_name[L_tmpnam]; - if (!std::tmpnam(file_name)) - { - std::cerr << "Cannot create temp file!\n"; - return 1; - } - std::string input = std::string(reinterpret_cast(data), size); - - { - std::ofstream temp_file(file_name); - temp_file << input; - } - - try - { - DB::YAMLParserImpl::parse(std::string(file_name)); - } - catch (...) - { - std::cerr << "YAML_fuzzer failed: " << DB::getCurrentExceptionMessage(__PRETTY_FUNCTION__) << std::endl; - return 1; - } - return 0; -} diff --git a/src/Compression/CMakeLists.txt b/src/Compression/CMakeLists.txt index 16c93d060cd..34369d8dbc8 100644 --- a/src/Compression/CMakeLists.txt +++ b/src/Compression/CMakeLists.txt @@ -1,11 +1,13 @@ -include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") -add_headers_and_sources(clickhouse_compression .) +if (ENABLE_FUZZING) + include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") + add_headers_and_sources(fuzz_compression .) -# Remove this file, because it has dependencies on DataTypes -list(REMOVE_ITEM ${clickhouse_compression_sources} CompressionFactoryAdditions.cpp) + # Remove this file, because it has dependencies on DataTypes + list(REMOVE_ITEM ${fuzz_compression_sources} CompressionFactoryAdditions.cpp) -add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources}) -target_link_libraries(clickhouse_compression PUBLIC clickhouse_parsers clickhouse_common_io common lz4) + add_library(fuzz_compression ${fuzz_compression_headers} ${fuzz_compression_sources}) + target_link_libraries(fuzz_compression PUBLIC clickhouse_parsers clickhouse_common_io common lz4) +endif() if (ENABLE_EXAMPLES) add_subdirectory(examples) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index b995a425429..5c3da6342ee 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -172,13 +172,15 @@ namespace UInt8 getDeltaBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is not applicable for because the data type is not of fixed size"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is not applicable for {} because the data type is not of fixed size", + column_type->getName()); size_t max_size = column_type->getSizeOfValueInMemory(); if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", + column_type->getName()); } } diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 0818b951036..3f96cfa8ff8 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -434,13 +434,15 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, UInt8 getDataBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec DoubleDelta is not applicable for {} because the data type is not of fixed size"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec DoubleDelta is not applicable for {} because the data type is not of fixed size", + column_type->getName()); size_t max_size = column_type->getSizeOfValueInMemory(); if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", + column_type->getName()); } } diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 6aabc6b8746..1276ac911f1 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -334,13 +334,15 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) UInt8 getDataBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Gorilla is not applicable for because the data type is not of fixed size"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Gorilla is not applicable for {} because the data type is not of fixed size", + column_type->getName()); size_t max_size = column_type->getSizeOfValueInMemory(); if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", + column_type->getName()); } } diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index 302fce833c5..377db915bce 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -728,8 +728,7 @@ void registerCodecT64(CompressionCodecFactory & factory) auto type_idx = typeIdx(type); if (type && type_idx == TypeIndex::Nothing) - throw Exception("T64 codec is not supported for specified type ", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); - + throw Exception("T64 codec is not supported for specified type " + type->getName(), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); return std::make_shared(type_idx, variant); }; diff --git a/src/Compression/examples/CMakeLists.txt b/src/Compression/examples/CMakeLists.txt index 04c17f12b42..8978a3c0aff 100644 --- a/src/Compression/examples/CMakeLists.txt +++ b/src/Compression/examples/CMakeLists.txt @@ -1,5 +1,5 @@ add_executable (compressed_buffer compressed_buffer.cpp) -target_link_libraries (compressed_buffer PRIVATE clickhouse_compression common) +target_link_libraries (compressed_buffer PRIVATE fuzz_compression common) add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) -target_link_libraries (cached_compressed_read_buffer PRIVATE clickhouse_compression common) +target_link_libraries (cached_compressed_read_buffer PRIVATE fuzz_compression common) diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt index e677e158a20..be4848b01ab 100644 --- a/src/Compression/fuzzers/CMakeLists.txt +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -5,16 +5,16 @@ # If you want really small size of the resulted binary, just link with fuzz_compression and clickhouse_common_io add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) -target_link_libraries (compressed_buffer_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (compressed_buffer_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) add_executable (lz4_decompress_fuzzer lz4_decompress_fuzzer.cpp) -target_link_libraries (lz4_decompress_fuzzer PUBLIC clickhouse_compression lz4 ${LIB_FUZZING_ENGINE}) +target_link_libraries (lz4_decompress_fuzzer PUBLIC fuzz_compression lz4 ${LIB_FUZZING_ENGINE}) add_executable (delta_decompress_fuzzer delta_decompress_fuzzer.cpp) -target_link_libraries (delta_decompress_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (delta_decompress_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) add_executable (double_delta_decompress_fuzzer double_delta_decompress_fuzzer.cpp) -target_link_libraries (double_delta_decompress_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (double_delta_decompress_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp) -target_link_libraries (encrypted_decompress_fuzzer PRIVATE clickhouse_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (encrypted_decompress_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) diff --git a/src/Core/CMakeLists.txt b/src/Core/CMakeLists.txt index 52e68e4dd73..85e2008753d 100644 --- a/src/Core/CMakeLists.txt +++ b/src/Core/CMakeLists.txt @@ -4,4 +4,4 @@ endif () if (ENABLE_FUZZING) add_subdirectory(fuzzers) -endif () +endif() diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index 2f48fb0c548..a20dd3567a9 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -9,7 +9,7 @@ if (USE_DEBUG_HELPERS) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") endif () -if (ENABLE_EXAMPLES) +if(ENABLE_EXAMPLES) add_subdirectory(examples) endif() diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index fabea01441f..c05441148df 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -1,7 +1,5 @@ #include -#include - #include #include #include @@ -690,5 +688,4 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N } } - } From 02e097a262bb5cd156d4365bb3c2b70ef42cc6fa Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 24 Aug 2021 13:36:07 +0000 Subject: [PATCH 107/127] Fix tests --- src/Compression/CompressedReadBufferBase.cpp | 1 - src/Compression/LZ4_decompress_faster.cpp | 15 +++++++++++++-- src/Compression/fuzzers/lz4_decompress_fuzzer.cpp | 6 ++++-- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 749f174677c..006b3fab2d8 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -253,4 +253,3 @@ CompressedReadBufferBase::~CompressedReadBufferBase() = default; /// Proper d } - diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index bbe3e899620..e3e20e26bbc 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -474,8 +474,19 @@ bool NO_INLINE decompressImpl( if (unlikely(copy_end > output_end)) return false; - if (unlikely(ip + std::max(copy_amount, static_cast(std::ceil(static_cast(length) / copy_amount) * copy_amount)) >= input_end)) - return false; + std::cout << "ip " << ip << std::endl; + std::cout << "length " << length << std::endl; + std::cout << "diff " << input_end - ip << std::endl; + std::cout << "input_end " << input_end << std::endl; + + // Due to implementation specifics the copy length is always a multiple of copy_amount + const size_t real_length = std::max(copy_amount, static_cast(std::ceil(static_cast(length) / copy_amount) * copy_amount)); + + if (unlikely(ip + real_length >= input_end + ADDITIONAL_BYTES_AT_END_OF_BUFFER)) + return false; + + // if (unlikely(ip + 1 >= input_end)) + // return false; wildCopy(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer. diff --git a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp index f13c776cda9..85c4c9bd329 100644 --- a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp @@ -3,6 +3,7 @@ #include #include +#include namespace DB { @@ -19,7 +20,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) try { - if (size < sizeof(AuxiliaryRandomData)) + if (size < sizeof(AuxiliaryRandomData) + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER) return 0; const auto * p = reinterpret_cast(data); @@ -27,13 +28,14 @@ try size_t output_buffer_size = p->decompressed_size % 65536; size -= sizeof(AuxiliaryRandomData); + size -= LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t); // std::string input = std::string(reinterpret_cast(data), size); // fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size); DB::Memory<> memory; - memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); + memory.resize(output_buffer_size + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); From b333e7c744e5fcf477e5f06e0ef52b15e456165d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 24 Aug 2021 13:40:59 +0000 Subject: [PATCH 108/127] Better --- src/Compression/LZ4_decompress_faster.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index e3e20e26bbc..52884e1e778 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -474,20 +474,12 @@ bool NO_INLINE decompressImpl( if (unlikely(copy_end > output_end)) return false; - std::cout << "ip " << ip << std::endl; - std::cout << "length " << length << std::endl; - std::cout << "diff " << input_end - ip << std::endl; - std::cout << "input_end " << input_end << std::endl; - // Due to implementation specifics the copy length is always a multiple of copy_amount const size_t real_length = std::max(copy_amount, static_cast(std::ceil(static_cast(length) / copy_amount) * copy_amount)); if (unlikely(ip + real_length >= input_end + ADDITIONAL_BYTES_AT_END_OF_BUFFER)) return false; - // if (unlikely(ip + 1 >= input_end)) - // return false; - wildCopy(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer. if (copy_end == output_end) From de8aa64e5639136f807b2f6b420f93472f04a1ea Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 24 Aug 2021 13:52:51 +0000 Subject: [PATCH 109/127] Deleted file --- .../CompressionFactoryAdditions.cpp | 214 ------------------ 1 file changed, 214 deletions(-) delete mode 100644 src/Compression/CompressionFactoryAdditions.cpp diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp deleted file mode 100644 index bff294d1ea5..00000000000 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ /dev/null @@ -1,214 +0,0 @@ -/** - * This file contains a part of CompressionCodecFactory methods definitions and - * is needed only because they have dependencies on DataTypes. - * They are not useful for fuzzers, so we leave them in other translation unit. - */ - -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int UNKNOWN_CODEC; - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; -} - - -void CompressionCodecFactory::validateCodec( - const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const -{ - if (family_name.empty()) - throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS); - - if (level) - { - auto literal = std::make_shared(static_cast(*level)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), - {}, sanity_check, allow_experimental_codecs); - } - else - { - auto identifier = std::make_shared(Poco::toUpper(family_name)); - validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), - {}, sanity_check, allow_experimental_codecs); - } -} - - -ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( - const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const -{ - if (const auto * func = ast->as()) - { - ASTPtr codecs_descriptions = std::make_shared(); - - bool is_compression = false; - bool has_none = false; - std::optional generic_compression_codec_pos; - std::set post_processing_codecs; - - bool can_substitute_codec_arguments = true; - for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) - { - const auto & inner_codec_ast = func->arguments->children[i]; - String codec_family_name; - ASTPtr codec_arguments; - if (const auto * family_name = inner_codec_ast->as()) - { - codec_family_name = family_name->name(); - codec_arguments = {}; - } - else if (const auto * ast_func = inner_codec_ast->as()) - { - codec_family_name = ast_func->name; - codec_arguments = ast_func->arguments; - } - else - throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - /// Default codec replaced with current default codec which may depend on different - /// settings (and properties of data) in runtime. - CompressionCodecPtr result_codec; - if (codec_family_name == DEFAULT_CODEC_NAME) - { - if (codec_arguments != nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME); - - result_codec = default_codec; - codecs_descriptions->children.emplace_back(std::make_shared(DEFAULT_CODEC_NAME)); - } - else - { - if (column_type) - { - CompressionCodecPtr prev_codec; - IDataType::StreamCallbackWithType callback = [&]( - const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) - { - if (ISerialization::isSpecialCompressionAllowed(substream_path)) - { - result_codec = getImpl(codec_family_name, codec_arguments, &substream_type); - - /// Case for column Tuple, which compressed with codec which depends on data type, like Delta. - /// We cannot substitute parameters for such codecs. - if (prev_codec && prev_codec->getHash() != result_codec->getHash()) - can_substitute_codec_arguments = false; - prev_codec = result_codec; - } - }; - - ISerialization::SubstreamPath stream_path; - column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path); - - if (!result_codec) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); - } - else - { - result_codec = getImpl(codec_family_name, codec_arguments, nullptr); - } - - if (!allow_experimental_codecs && result_codec->isExperimental()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Codec {} is experimental and not meant to be used in production." - " You can enable it with the 'allow_experimental_codecs' setting.", - codec_family_name); - - codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); - } - - is_compression |= result_codec->isCompression(); - has_none |= result_codec->isNone(); - - if (!generic_compression_codec_pos && result_codec->isGenericCompression()) - generic_compression_codec_pos = i; - - if (result_codec->isPostProcessing()) - post_processing_codecs.insert(i); - } - - String codec_description = queryToString(codecs_descriptions); - - if (sanity_check) - { - if (codecs_descriptions->children.size() > 1 && has_none) - throw Exception( - "It does not make sense to have codec NONE along with other compression codecs: " + codec_description - + ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", - ErrorCodes::BAD_ARGUMENTS); - - /// Allow to explicitly specify single NONE codec if user don't want any compression. - /// But applying other transformations solely without compression (e.g. Delta) does not make sense. - /// It's okay to apply post-processing codecs solely without anything else. - if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size()) - throw Exception( - "Compression codec " + codec_description - + " does not compress anything." - " You may want to add generic compression algorithm after other transformations, like: " - + codec_description - + ", LZ4." - " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", - ErrorCodes::BAD_ARGUMENTS); - - /// It does not make sense to apply any non-post-processing codecs - /// after post-processing one. - if (!post_processing_codecs.empty() && - *post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size()) - throw Exception("The combination of compression codecs " + codec_description + " is meaningless," - " because it does not make sense to apply any non-post-processing codecs after" - " post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'" - " to skip this check).", ErrorCodes::BAD_ARGUMENTS); - - /// It does not make sense to apply any transformations after generic compression algorithm - /// So, generic compression can be only one and only at the end. - if (generic_compression_codec_pos && - *generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size()) - throw Exception("The combination of compression codecs " + codec_description + " is meaningless," - " because it does not make sense to apply any transformations after generic compression algorithm." - " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); - - } - - /// For columns with nested types like Tuple(UInt32, UInt64) we - /// obviously cannot substitute parameters for codecs which depend on - /// data type, because for the first column Delta(4) is suitable and - /// Delta(8) for the second. So we should leave codec description as is - /// and deduce them in get method for each subtype separately. For all - /// other types it's better to substitute parameters, for better - /// readability and backward compatibility. - if (can_substitute_codec_arguments) - { - std::shared_ptr result = std::make_shared(); - result->name = "CODEC"; - result->arguments = codecs_descriptions; - return result; - } - else - { - return ast; - } - } - - throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); -} - - -} From c5ca63c6067fd08937b61e9e1c3f1a338033ad68 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 00:04:29 +0000 Subject: [PATCH 110/127] Fix performance (probably) --- src/Compression/LZ4_decompress_faster.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 52884e1e778..4a11d457f9a 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -475,7 +475,15 @@ bool NO_INLINE decompressImpl( return false; // Due to implementation specifics the copy length is always a multiple of copy_amount - const size_t real_length = std::max(copy_amount, static_cast(std::ceil(static_cast(length) / copy_amount) * copy_amount)); + size_t real_length = 0; + if constexpr (copy_amount == 8) + real_length = (((length >> 3) + 1) * 8); + else if constexpr (copy_amount == 16) + real_length = (((length >> 4) + 1) * 16); + else if constexpr (copy_amount == 32) + real_length = (((length >> 5) + 1) * 32); + else + throw std::runtime_error("Compile error!"); if (unlikely(ip + real_length >= input_end + ADDITIONAL_BYTES_AT_END_OF_BUFFER)) return false; From 923d6759c0413fbc1e7bec406df91fa5f898e144 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 11:24:41 +0000 Subject: [PATCH 111/127] Fix style --- src/Compression/CompressionCodecDelta.cpp | 1 - ...oryAdditions.cpp => CompressionFactoryAdditions.cpp} | 10 +++++++++- src/Compression/examples/CMakeLists.txt | 4 ++-- src/Compression/fuzzers/CMakeLists.txt | 10 +++++----- 4 files changed, 16 insertions(+), 9 deletions(-) rename src/Compression/{СompressionFactoryAdditions.cpp => CompressionFactoryAdditions.cpp} (96%) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 5c3da6342ee..e8c5b4f878d 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -41,7 +41,6 @@ namespace ErrorCodes extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int ILLEGAL_CODEC_PARAMETER; extern const int BAD_ARGUMENTS; - extern const int CANNOT_DECOMPRESS; } CompressionCodecDelta::CompressionCodecDelta(UInt8 delta_bytes_size_) diff --git a/src/Compression/СompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp similarity index 96% rename from src/Compression/СompressionFactoryAdditions.cpp rename to src/Compression/CompressionFactoryAdditions.cpp index 90ad45d10d6..bff294d1ea5 100644 --- a/src/Compression/СompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -1,3 +1,9 @@ +/** + * This file contains a part of CompressionCodecFactory methods definitions and + * is needed only because they have dependencies on DataTypes. + * They are not useful for fuzzers, so we leave them in other translation unit. + */ + #include #include @@ -20,6 +26,8 @@ namespace ErrorCodes { extern const int UNEXPECTED_AST_STRUCTURE; extern const int UNKNOWN_CODEC; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } @@ -43,6 +51,7 @@ void CompressionCodecFactory::validateCodec( } } + ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const { @@ -202,5 +211,4 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( } - } diff --git a/src/Compression/examples/CMakeLists.txt b/src/Compression/examples/CMakeLists.txt index 8978a3c0aff..3cfc0ccb7dc 100644 --- a/src/Compression/examples/CMakeLists.txt +++ b/src/Compression/examples/CMakeLists.txt @@ -1,5 +1,5 @@ add_executable (compressed_buffer compressed_buffer.cpp) -target_link_libraries (compressed_buffer PRIVATE fuzz_compression common) +target_link_libraries (compressed_buffer PRIVATE dbms) add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) -target_link_libraries (cached_compressed_read_buffer PRIVATE fuzz_compression common) +target_link_libraries (cached_compressed_read_buffer PRIVATE dbms) diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt index be4848b01ab..189aea66a92 100644 --- a/src/Compression/fuzzers/CMakeLists.txt +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -5,16 +5,16 @@ # If you want really small size of the resulted binary, just link with fuzz_compression and clickhouse_common_io add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) -target_link_libraries (compressed_buffer_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) add_executable (lz4_decompress_fuzzer lz4_decompress_fuzzer.cpp) -target_link_libraries (lz4_decompress_fuzzer PUBLIC fuzz_compression lz4 ${LIB_FUZZING_ENGINE}) +target_link_libraries (lz4_decompress_fuzzer PUBLIC dbms lz4 ${LIB_FUZZING_ENGINE}) add_executable (delta_decompress_fuzzer delta_decompress_fuzzer.cpp) -target_link_libraries (delta_decompress_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (delta_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) add_executable (double_delta_decompress_fuzzer double_delta_decompress_fuzzer.cpp) -target_link_libraries (double_delta_decompress_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (double_delta_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp) -target_link_libraries (encrypted_decompress_fuzzer PRIVATE fuzz_compression ${LIB_FUZZING_ENGINE}) +target_link_libraries (encrypted_decompress_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) From ea5fbbedec58317a16affdd99caa84e2d0814c61 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Aug 2021 19:15:56 +0300 Subject: [PATCH 112/127] Fix clickhouse keeper jepsen tests --- programs/keeper/Keeper.cpp | 4 ++-- .../src/jepsen/clickhouse_keeper/db.clj | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 4d01a523853..a67a305e23b 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -300,9 +300,9 @@ int Keeper::main(const std::vector & /*args*/) if (config().has("keeper_server.storage_path")) path = config().getString("keeper_server.storage_path"); else if (config().has("keeper_server.log_storage_path")) - path = config().getString("keeper_server.log_storage_path"); + path = std::filesystem::path(config().getString("keeper_server.log_storage_path")).parent_path(); else if (config().has("keeper_server.snapshot_storage_path")) - path = config().getString("keeper_server.snapshot_storage_path"); + path = std::filesystem::path(config().getString("keeper_server.snapshot_storage_path")).parent_path(); else path = std::filesystem::path{KEEPER_DEFAULT_PATH}; diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj index 30c2c0eaf4f..745d88e97f7 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj @@ -68,6 +68,7 @@ (do (c/exec :mkdir :-p common-prefix) (c/exec :mkdir :-p data-dir) + (c/exec :mkdir :-p coordination-data-dir) (c/exec :mkdir :-p logs-dir) (c/exec :mkdir :-p configs-dir) (c/exec :mkdir :-p sub-configs-dir) From f1be822a1a11f8e70926a17dcc390bceaea56ba2 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Wed, 25 Aug 2021 20:14:04 +0300 Subject: [PATCH 113/127] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index ea9a0fc4c27..dea07e8848e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2240,8 +2240,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. -!!! warning "Warning" - In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). And then the server will argue that the constant column has different values. + In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). **Syntax** From 5a50a1e9d989d2943596c09963e139421cd5cfdc Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Wed, 25 Aug 2021 20:15:28 +0300 Subject: [PATCH 114/127] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 741af425dee..cea4c66e23c 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2190,8 +2190,7 @@ defaultRoles() Возвращает идентификатор текущего запроса. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `query_id`. -!!! warning "Предупреждение" - В отличие от [initialQueryID](#initial-query-id), функция `queryID` может возвращать различные значения для разных шардов (см. пример), что неверно для столбца констант. + В отличие от [initialQueryID](#initial-query-id), функция `queryID` может возвращать различные значения для разных шардов (см. пример). **Синтаксис** From e8e60e1a608228844b9841dbe7dd354981b42597 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Wed, 25 Aug 2021 20:15:57 +0300 Subject: [PATCH 115/127] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index dea07e8848e..d169e7034c4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2259,6 +2259,8 @@ Type: [String](../../sql-reference/data-types/string.md) Query: ``` sql +CREATE TABLE tmp (str String) ENGINE = Log; +INSERT INTO tmp (*) VALUES ('a'); SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); ``` From 9ebe99b4b02fb8ef42fadd6dfd4e2501f04a13ba Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Wed, 25 Aug 2021 20:16:07 +0300 Subject: [PATCH 116/127] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index cea4c66e23c..299fd0c568b 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2209,6 +2209,8 @@ queryID() Запрос: ``` sql +CREATE TABLE tmp (str String) ENGINE = Log; +INSERT INTO tmp (*) VALUES ('a'); SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); ``` From 7295bf95b3440b32878055b3ae8699b2048b3533 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Wed, 25 Aug 2021 20:16:16 +0300 Subject: [PATCH 117/127] Update docs/en/sql-reference/functions/other-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/en/sql-reference/functions/other-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index d169e7034c4..01bc2eab309 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2295,6 +2295,8 @@ Type: [String](../../sql-reference/data-types/string.md) Query: ``` sql +CREATE TABLE tmp (str String) ENGINE = Log; +INSERT INTO tmp (*) VALUES ('a'); SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); ``` From 0d39cd2faeb6730bd26cae7ddb8193e030c67e37 Mon Sep 17 00:00:00 2001 From: Dmitriy <72220289+sevirov@users.noreply.github.com> Date: Wed, 25 Aug 2021 20:16:23 +0300 Subject: [PATCH 118/127] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: Alexey Boykov <33257111+mathalex@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 299fd0c568b..0f460f84875 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2245,6 +2245,8 @@ initialQueryID() Запрос: ``` sql +CREATE TABLE tmp (str String) ENGINE = Log; +INSERT INTO tmp (*) VALUES ('a'); SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); ``` From 80b6097dae42a802cb106f8eb390330c6e896d2a Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 25 Aug 2021 20:33:00 +0300 Subject: [PATCH 119/127] Fix typo MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Убрал лишние отступы. --- docs/en/sql-reference/functions/other-functions.md | 2 +- docs/ru/sql-reference/functions/other-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 01bc2eab309..cf77444b17f 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2240,7 +2240,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. - In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). +In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). **Syntax** diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 0f460f84875..5aae0eee9f9 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2190,7 +2190,7 @@ defaultRoles() Возвращает идентификатор текущего запроса. Другие параметры запроса могут быть извлечены из системной таблицы [system.query_log](../../operations/system-tables/query_log.md) через `query_id`. - В отличие от [initialQueryID](#initial-query-id), функция `queryID` может возвращать различные значения для разных шардов (см. пример). +В отличие от [initialQueryID](#initial-query-id), функция `queryID` может возвращать различные значения для разных шардов (см. пример). **Синтаксис** From 28d70dae43dac5c1d51fcbb3bb9f1265d06120d2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 20:58:53 +0000 Subject: [PATCH 120/127] Deleted printing exception message --- src/Compression/fuzzers/compressed_buffer_fuzzer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp b/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp index a87046eff5c..1f669696fb9 100644 --- a/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp +++ b/src/Compression/fuzzers/compressed_buffer_fuzzer.cpp @@ -17,6 +17,5 @@ try } catch (...) { - std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; return 1; } From f99d7d89b875085d0701a8f8170ac91eb61e6bb1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 21:11:45 +0000 Subject: [PATCH 121/127] Added static assert --- src/Compression/LZ4_decompress_faster.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 4a11d457f9a..1aeae6b1b9a 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -476,14 +476,14 @@ bool NO_INLINE decompressImpl( // Due to implementation specifics the copy length is always a multiple of copy_amount size_t real_length = 0; + + static_assert(copy_amount == 8 || copy_amount == 16 || copy_amount == 32); if constexpr (copy_amount == 8) real_length = (((length >> 3) + 1) * 8); else if constexpr (copy_amount == 16) real_length = (((length >> 4) + 1) * 16); else if constexpr (copy_amount == 32) real_length = (((length >> 5) + 1) * 32); - else - throw std::runtime_error("Compile error!"); if (unlikely(ip + real_length >= input_end + ADDITIONAL_BYTES_AT_END_OF_BUFFER)) return false; From ac50251a6fc758a9db2da9d562140988bb4dafb3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 25 Aug 2021 21:21:54 +0000 Subject: [PATCH 122/127] Updated ya.make files --- src/Core/ya.make | 4 ++++ src/Storages/ya.make | 1 + src/TableFunctions/ya.make | 1 + 3 files changed, 6 insertions(+) diff --git a/src/Core/ya.make b/src/Core/ya.make index d1e352ee846..6946d7a47bb 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -31,6 +31,10 @@ SRCS( MySQL/PacketsProtocolText.cpp MySQL/PacketsReplication.cpp NamesAndTypes.cpp + PostgreSQL/Connection.cpp + PostgreSQL/PoolWithFailover.cpp + PostgreSQL/Utils.cpp + PostgreSQL/insertPostgreSQLValue.cpp PostgreSQLProtocol.cpp QueryProcessingStage.cpp Settings.cpp diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 04311b83d09..c001d933558 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -211,6 +211,7 @@ SRCS( System/StorageSystemTables.cpp System/StorageSystemUserDirectories.cpp System/StorageSystemUsers.cpp + System/StorageSystemWarnings.cpp System/StorageSystemZeros.cpp System/StorageSystemZooKeeper.cpp System/attachSystemTables.cpp diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index f50e345f2d8..e957c923423 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -22,6 +22,7 @@ SRCS( TableFunctionNull.cpp TableFunctionNumbers.cpp TableFunctionRemote.cpp + TableFunctionSQLite.cpp TableFunctionURL.cpp TableFunctionValues.cpp TableFunctionView.cpp From 0b66581de2ae0ed714786e962bf250c66f72893a Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 25 Aug 2021 21:51:43 +0000 Subject: [PATCH 123/127] Fix build --- .../odbc-bridge/ODBCBlockOutputStream.cpp | 2 +- src/Common/getInsertQuery.cpp | 2 +- src/Parsers/getInsertQuery.cpp | 28 +++++++++++++++++++ src/{Common => Parsers}/getInsertQuery.h | 0 src/Storages/StoragePostgreSQL.cpp | 4 +-- 5 files changed, 32 insertions(+), 4 deletions(-) create mode 100644 src/Parsers/getInsertQuery.cpp rename src/{Common => Parsers}/getInsertQuery.h (100%) diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index 5b3bb0f42b9..8a4387c2389 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Common/getInsertQuery.cpp b/src/Common/getInsertQuery.cpp index 830374b6060..6f52056dfe2 100644 --- a/src/Common/getInsertQuery.cpp +++ b/src/Common/getInsertQuery.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Parsers/getInsertQuery.cpp b/src/Parsers/getInsertQuery.cpp new file mode 100644 index 00000000000..6f52056dfe2 --- /dev/null +++ b/src/Parsers/getInsertQuery.cpp @@ -0,0 +1,28 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ +std::string getInsertQuery(const std::string & db_name, const std::string & table_name, const ColumnsWithTypeAndName & columns, IdentifierQuotingStyle quoting) +{ + ASTInsertQuery query; + query.table_id.database_name = db_name; + query.table_id.table_name = table_name; + query.columns = std::make_shared(','); + query.children.push_back(query.columns); + for (const auto & column : columns) + query.columns->children.emplace_back(std::make_shared(column.name)); + + WriteBufferFromOwnString buf; + IAST::FormatSettings settings(buf, true); + settings.always_quote_identifiers = true; + settings.identifier_quoting_style = quoting; + query.IAST::format(settings); + return buf.str(); +} +} diff --git a/src/Common/getInsertQuery.h b/src/Parsers/getInsertQuery.h similarity index 100% rename from src/Common/getInsertQuery.h rename to src/Parsers/getInsertQuery.h diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 7726008c160..3617e964734 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -29,7 +29,7 @@ #include #include #include -#include +#include #include @@ -292,7 +292,7 @@ private: pqxx::connection & connection; pqxx::work tx; - Inserter(pqxx::connection & connection_) + explicit Inserter(pqxx::connection & connection_) : connection(connection_) , tx(connection) {} From aea22ec6a1b35172cc409f78bb0ca1dd76c86b40 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 26 Aug 2021 08:44:50 +0300 Subject: [PATCH 124/127] Delete getInsertQuery.cpp --- src/Common/getInsertQuery.cpp | 28 ---------------------------- 1 file changed, 28 deletions(-) delete mode 100644 src/Common/getInsertQuery.cpp diff --git a/src/Common/getInsertQuery.cpp b/src/Common/getInsertQuery.cpp deleted file mode 100644 index 6f52056dfe2..00000000000 --- a/src/Common/getInsertQuery.cpp +++ /dev/null @@ -1,28 +0,0 @@ -#include - -#include -#include -#include -#include - - -namespace DB -{ -std::string getInsertQuery(const std::string & db_name, const std::string & table_name, const ColumnsWithTypeAndName & columns, IdentifierQuotingStyle quoting) -{ - ASTInsertQuery query; - query.table_id.database_name = db_name; - query.table_id.table_name = table_name; - query.columns = std::make_shared(','); - query.children.push_back(query.columns); - for (const auto & column : columns) - query.columns->children.emplace_back(std::make_shared(column.name)); - - WriteBufferFromOwnString buf; - IAST::FormatSettings settings(buf, true); - settings.always_quote_identifiers = true; - settings.identifier_quoting_style = quoting; - query.IAST::format(settings); - return buf.str(); -} -} From b13808f13ad814617ab14e3aca7df7ad8134b25f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 26 Aug 2021 12:08:36 +0300 Subject: [PATCH 125/127] Remove past event --- README.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/README.md b/README.md index 178547ea523..496a6357f44 100644 --- a/README.md +++ b/README.md @@ -13,6 +13,3 @@ ClickHouse® is an open-source column-oriented database management system that a * [Code Browser](https://clickhouse.tech/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation. * [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person. - -## Upcoming Events -* [SF Bay Area ClickHouse August Community Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/279109379/) on 25 August 2021. From b8296416f7f918ca67f526d306a0e3e7eebdbdce Mon Sep 17 00:00:00 2001 From: lehasm Date: Thu, 26 Aug 2021 13:09:02 +0300 Subject: [PATCH 126/127] DOCSUP-13276: document print_query_id (#28106) * EN description * Example description updated * Update docs/en/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/en/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Translated to ru * Minor fixes * Some Ru grammar and formatting fixed. * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/interfaces/cli.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/interfaces/cli.md | 30 ++++++++++++++++++++-- docs/ru/interfaces/cli.md | 54 +++++++++++++++++++++++++++++---------- 2 files changed, 68 insertions(+), 16 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 8457ea41857..70b7d59b037 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -141,7 +141,7 @@ Since version 20.5, `clickhouse-client` has automatic syntax highlighting (alway Example of a config file: -``` xml +```xml username password @@ -149,4 +149,30 @@ Example of a config file: ``` -[Original article](https://clickhouse.tech/docs/en/interfaces/cli/) +### Query ID Format {#query-id-format} + +In interactive mode `clickhouse-client` shows query ID for every query. By default, the ID is formatted like this: + +```sql +Query id: 927f137d-00f1-4175-8914-0dd066365e96 +``` + +A custom format may be specified in a configuration file inside a `query_id_formats` tag. `{query_id}` placeholder in the format string is replaced with the ID of a query. Several format strings are allowed inside the tag. +This feature can be used to generate URLs to facilitate profiling of queries. + +**Example** + +```xml + + + http://speedscope-host/#profileURL=qp%3Fid%3D{query_id} + + +``` + +If the configuration above is applied, the ID of a query is shown in the following format: + +``` text +speedscope:http://speedscope-host/#profileURL=qp%3Fid%3Dc8ecc783-e753-4b38-97f1-42cddfb98b7d +``` + diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 277b73a6d36..bbb66b70371 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -26,7 +26,7 @@ Connected to ClickHouse server version 20.13.1 revision 54442. Клиент может быть использован в интерактивном и не интерактивном (batch) режиме. Чтобы использовать batch режим, укажите параметр query, или отправьте данные в stdin (проверяется, что stdin - не терминал), или и то, и другое. -Аналогично HTTP интерфейсу, при использовании одновременно параметра query и отправке данных в stdin, запрос составляется из конкатенации параметра query, перевода строки, и данных в stdin. Это удобно для больших INSERT запросов. +Аналогично HTTP интерфейсу, при использовании одновременно параметра query и отправке данных в stdin, запрос составляется из конкатенации параметра query, перевода строки и данных в stdin. Это удобно для больших `INSERT` запросов. Примеры использования клиента для вставки данных: @@ -41,17 +41,17 @@ _EOF $ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV"; ``` -В batch режиме в качестве формата данных по умолчанию используется формат TabSeparated. Формат может быть указан в секции FORMAT запроса. +В batch режиме в качестве формата данных по умолчанию используется формат `TabSeparated`. Формат может быть указан в запросе в секции `FORMAT`. -По умолчанию, в batch режиме вы можете выполнить только один запрос. Чтобы выполнить несколько запросов из «скрипта», используйте параметр –multiquery. Это работает для всех запросов кроме INSERT. Результаты запросов выводятся подряд без дополнительных разделителей. -Также, при необходимости выполнить много запросов, вы можете запускать clickhouse-client на каждый запрос. Заметим, что запуск программы clickhouse-client может занимать десятки миллисекунд. +По умолчанию в batch режиме вы можете выполнить только один запрос. Чтобы выполнить несколько запросов из «скрипта», используйте параметр `–-multiquery`. Это работает для всех запросов кроме `INSERT`. Результаты запросов выводятся подряд без дополнительных разделителей. +Если нужно выполнить много запросов, вы можете запускать clickhouse-client отдельно на каждый запрос. Заметим, что запуск программы clickhouse-client может занимать десятки миллисекунд. -В интерактивном режиме, вы получите командную строку, в которую можно вводить запросы. +В интерактивном режиме вы получаете командную строку, в которую можно вводить запросы. Если не указано multiline (по умолчанию): -Чтобы выполнить запрос, нажмите Enter. Точка с запятой на конце запроса не обязательна. Чтобы ввести запрос, состоящий из нескольких строк, перед переводом строки, введите символ обратного слеша: `\` - тогда после нажатия Enter, вам предложат ввести следующую строку запроса. +Чтобы выполнить запрос, нажмите Enter. Точка с запятой на конце запроса необязательна. Чтобы ввести запрос, состоящий из нескольких строк, в конце строки поставьте символ обратного слеша `\`, тогда после нажатия Enter вы сможете ввести следующую строку запроса. -Если указано multiline (многострочный режим): +Если указан параметр `--multiline` (многострочный режим): Чтобы выполнить запрос, завершите его точкой с запятой и нажмите Enter. Если в конце введённой строки не было точки с запятой, то вам предложат ввести следующую строчку запроса. Исполняется только один запрос, поэтому всё, что введено после точки с запятой, игнорируется. @@ -61,20 +61,20 @@ $ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FOR Командная строка сделана на основе readline (и history) (или libedit, или без какой-либо библиотеки, в зависимости от сборки) - то есть, в ней работают привычные сочетания клавиш, а также присутствует история. История пишется в `~/.clickhouse-client-history`. -По умолчанию, в качестве формата, используется формат PrettyCompact (красивые таблички). Вы можете изменить формат с помощью секции FORMAT запроса, или с помощью указания `\G` на конце запроса, с помощью аргумента командной строки `--format` или `--vertical`, или с помощью конфигурационного файла клиента. +По умолчанию используется формат вывода `PrettyCompact` (он поддерживает красивый вывод таблиц). Вы можете изменить формат вывода результатов запроса следующими способами: с помощью секции `FORMAT` в запросе, указав символ `\G` в конце запроса, используя аргументы командной строки `--format` или `--vertical` или с помощью конфигурационного файла клиента. -Чтобы выйти из клиента, нажмите Ctrl+D, или наберите вместо запроса одно из: «exit», «quit», «logout», «учше», «йгше», «дщпщге», «exit;», «quit;», «logout;», «учшеж», «йгшеж», «дщпщгеж», «q», «й», «q», «Q», «:q», «й», «Й», «Жй» +Чтобы выйти из клиента, нажмите Ctrl+D или наберите вместо запроса одно из: «exit», «quit», «logout», «учше», «йгше», «дщпщге», «exit;», «quit;», «logout;», «учшеж», «йгшеж», «дщпщгеж», «q», «й», «q», «Q», «:q», «й», «Й», «Жй». -При выполнении запроса, клиент показывает: +При выполнении запроса клиент показывает: -1. Прогресс выполнение запроса, который обновляется не чаще, чем 10 раз в секунду (по умолчанию). При быстрых запросах, прогресс может не успеть отобразиться. +1. Прогресс выполнение запроса, который обновляется не чаще, чем 10 раз в секунду (по умолчанию). При быстрых запросах прогресс может не успеть отобразиться. 2. Отформатированный запрос после его парсинга - для отладки. 3. Результат в заданном формате. 4. Количество строк результата, прошедшее время, а также среднюю скорость выполнения запроса. -Вы можете прервать длинный запрос, нажав Ctrl+C. При этом вам всё равно придётся чуть-чуть подождать, пока сервер остановит запрос. На некоторых стадиях выполнения, запрос невозможно прервать. Если вы не дождётесь и нажмёте Ctrl+C второй раз, то клиент будет завершён. +Вы можете прервать длинный запрос, нажав Ctrl+C. При этом вам всё равно придётся чуть-чуть подождать, пока сервер остановит запрос. На некоторых стадиях выполнения запрос невозможно прервать. Если вы не дождётесь и нажмёте Ctrl+C второй раз, то клиент будет завершён. -Клиент командной строки позволяет передать внешние данные (внешние временные таблицы) для использования запроса. Подробнее смотрите раздел «Внешние данные для обработки запроса» +Клиент командной строки позволяет передать внешние данные (внешние временные таблицы) для выполнения запроса. Подробнее смотрите раздел «Внешние данные для обработки запроса». ### Запросы с параметрами {#cli-queries-with-parameters} @@ -84,7 +84,7 @@ $ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FOR clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}" ``` -#### Cинтаксис запроса {#cli-queries-with-parameters-syntax} +#### Синтаксис запроса {#cli-queries-with-parameters-syntax} Отформатируйте запрос обычным способом. Представьте значения, которые вы хотите передать из параметров приложения в запрос в следующем формате: @@ -155,3 +155,29 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe ``` +### Формат ID запроса {#query-id-format} + +В интерактивном режиме `clickhouse-client` показывает ID для каждого запроса. По умолчанию ID выводится в таком виде: + +```sql +Query id: 927f137d-00f1-4175-8914-0dd066365e96 +``` + +Произвольный формат ID можно задать в конфигурационном файле внутри тега `query_id_formats`. ID подставляется вместо `{query_id}` в строке формата. В теге может быть перечислено несколько строк формата. +Эта возможность может быть полезна для генерации URL, с помощью которых выполняется профилирование запросов. + +**Пример** + +```xml + + + http://speedscope-host/#profileURL=qp%3Fid%3D{query_id} + + +``` + +Если применить приведённую выше конфигурацию, то ID запроса будет выводиться в следующем виде: + +``` text +speedscope:http://speedscope-host/#profileURL=qp%3Fid%3Dc8ecc783-e753-4b38-97f1-42cddfb98b7d +``` From d9308749de2e593ad9141d0ee52da48a75c9502e Mon Sep 17 00:00:00 2001 From: sdk2 <469656392@qq.com> Date: Thu, 26 Aug 2021 18:52:17 +0800 Subject: [PATCH 127/127] Update src/Functions/GatherUtils/Sources.h --- src/Functions/GatherUtils/Sources.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index 79f627fb64c..86b590646dc 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -325,7 +325,7 @@ struct StringSource }; -/// Differs to StringSource by having 'offest' and 'length' in code points instead of bytes in getSlice* methods. +/// Differs to StringSource by having 'offset' and 'length' in code points instead of bytes in getSlice* methods. /** NOTE: The behaviour of substring and substringUTF8 is inconsistent when negative offset is greater than string size: * substring: * hello