From f8f67a788e4c8dc41b59d6f22631172fb4a431df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jun 2020 19:55:45 +0300 Subject: [PATCH 001/535] allow to turn on fsync on inserts and merges --- src/Disks/DiskLocal.cpp | 17 +++++++- src/Disks/DiskLocal.h | 2 + src/Disks/DiskMemory.cpp | 5 +++ src/Disks/DiskMemory.h | 2 + src/Disks/IDisk.h | 3 ++ src/Disks/S3/DiskS3.cpp | 5 +++ src/Disks/S3/DiskS3.h | 2 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 11 +++-- .../MergeTree/IMergeTreeDataPartWriter.h | 6 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 41 +++++++++++++------ .../MergeTree/MergeTreeDataMergerMutator.h | 6 ++- .../MergeTreeDataPartWriterCompact.cpp | 4 +- .../MergeTreeDataPartWriterCompact.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 7 +++- src/Storages/MergeTree/MergeTreeSettings.h | 3 ++ .../MergeTree/MergedBlockOutputStream.cpp | 7 ++-- .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 9 ++-- .../MergeTree/MergedColumnOnlyOutputStream.h | 2 +- 21 files changed, 108 insertions(+), 33 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 68f5ee99a7a..c67bac7ffe2 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,7 +8,7 @@ #include #include - +#include namespace DB { @@ -19,6 +19,9 @@ namespace ErrorCodes extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int PATH_ACCESS_DENIED; extern const int INCORRECT_DISK_INDEX; + extern const int FILE_DOESNT_EXIST; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_FSYNC; } std::mutex DiskLocal::reservation_mutex; @@ -188,6 +191,18 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } +void DiskLocal::sync(const String & path) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), O_RDONLY); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + + if (-1 == fsync(fd)) + throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); +} + DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 61a3994b655..743ba2ceb10 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,6 +59,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 3e43d159ba5..5b3350e40f7 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,6 +261,11 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +void DiskMemory::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index b0c1d30c61d..8a3ddf05aa7 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,6 +52,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 011c75402f4..8de77a560d1 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,6 +105,9 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; + /// Do fsync on directory. + virtual void sync(const String & path) const = 0; + /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 71b5991f770..292f6567df4 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -466,6 +466,11 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } +void DiskS3::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 5fa8e8358a6..09132367ae8 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,6 +58,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 73ac7fc0064..03ae2166504 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -308,7 +308,8 @@ void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices( skip_index_data_mark = skip_index_current_data_mark; } -void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) +void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization( + MergeTreeData::DataPart::Checksums & checksums, bool sync) { bool write_final_mark = (with_final_mark && data_written); if (write_final_mark && compute_granularity) @@ -330,12 +331,14 @@ void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::Da index_stream->next(); checksums.files["primary.idx"].file_size = index_stream->count(); checksums.files["primary.idx"].file_hash = index_stream->getHash(); - index_stream = nullptr; + if (sync) + index_stream->sync(); + index_stream.reset(); } } void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, bool sync) { for (size_t i = 0; i < skip_indices.size(); ++i) { @@ -348,6 +351,8 @@ void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( { stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); } skip_indices_streams.clear(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 2f849e7c895..eebdb880a66 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -102,9 +102,9 @@ public: void initSkipIndices(); void initPrimaryIndex(); - virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0; - void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums); - void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums); + virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0; + void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync); + void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksum, bool sync); protected: /// Count index_granularity for block and store in `index_granularity` diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 00830dd78c2..ccd7f234925 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -576,6 +576,13 @@ public: } }; +static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) +{ + return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) + || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); +} + + /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, @@ -648,6 +655,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } size_t sum_input_rows_upper_bound = merge_entry->total_rows_count; + size_t sum_compressed_bytes_upper_bound = merge_entry->total_size_bytes_compressed; MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values); LOG_DEBUG(log, "Selected MergeAlgorithm: {}", ((merge_alg == MergeAlgorithm::Vertical) ? "Vertical" : "Horizontal")); @@ -803,7 +811,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values) merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, force_ttl); - if (metadata_snapshot->hasSecondaryIndices()) { const auto & indices = metadata_snapshot->getSecondaryIndices(); @@ -863,6 +870,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values && ttl_merges_blocker.isCancelled()) throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); + bool need_sync = needSyncPart(sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); MergeTreeData::DataPart::Checksums checksums_gathered_columns; /// Gather ordinary columns @@ -942,7 +950,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); column_gathered_stream.readSuffix(); - auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns); + auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns, need_sync); checksums_gathered_columns.add(std::move(changed_checksums)); if (rows_written != column_elems_written) @@ -979,9 +987,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } if (merge_alg != MergeAlgorithm::Vertical) - to.writeSuffixAndFinalizePart(new_data_part); + to.writeSuffixAndFinalizePart(new_data_part, need_sync); else - to.writeSuffixAndFinalizePart(new_data_part, &storage_columns, &checksums_gathered_columns); + to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); + + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); return new_data_part; } @@ -1081,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// Don't change granularity type while mutating subset of columns 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; if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) @@ -1099,7 +1110,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); /// no finalization required, because mutateAllPartColumns use /// MergedBlockOutputStream which finilaze all part fields itself @@ -1154,7 +1166,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); } for (const auto & [rename_from, rename_to] : files_to_rename) @@ -1174,6 +1187,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); + return new_data_part; } @@ -1599,7 +1615,8 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1637,7 +1654,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( new_data_part->minmax_idx = std::move(minmax_idx); mutating_stream->readSuffix(); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, need_sync); } void MergeTreeDataMergerMutator::mutateSomePartColumns( @@ -1650,7 +1667,8 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1684,10 +1702,9 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( mutating_stream->readSuffix(); - auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums); + auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums, need_sync); new_data_part->checksums.add(std::move(changed_checksums)); - } void MergeTreeDataMergerMutator::finalizeMutatedPart( diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 121cc770d51..23b8d7f681b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -189,7 +189,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Mutate some columns of source part with mutation_stream void mutateSomePartColumns( @@ -202,7 +203,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Initialize and write to disk new part fields like checksums, columns, /// etc. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index f7a3ad75cf5..79800204a3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -141,7 +141,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy column.type->serializeBinaryBulkStateSuffix(serialize_settings, state); } -void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { if (columns_buffer.size() != 0) writeBlock(header.cloneWithColumns(columns_buffer.releaseColumns())); @@ -158,6 +158,8 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); stream.reset(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8183c038c4c..dde7deafc58 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -20,7 +20,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; protected: void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index e71ea4d4b94..fcd0249b10c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -264,7 +264,7 @@ void MergeTreeDataPartWriterWide::writeColumn( next_index_offset = current_row - total_rows; } -void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { const auto & global_settings = storage.global_context.getSettingsRef(); IDataType::SerializeBinaryBulkSettings serialize_settings; @@ -295,6 +295,8 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch { stream.second->finalize(); stream.second->addToChecksums(checksums); + if (sync) + stream.second->sync(); } column_streams.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index f5a9d17f63c..4286065a3ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -23,7 +23,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 099480aca2f..cf8860b7f04 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -303,10 +303,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); + bool sync_on_insert = data.getSettings()->sync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); + + /// Sync part directory. + if (sync_on_insert) + new_data_part->volume->getDisk()->sync(full_path); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f2d2a7cc3d4..da2c9ee49ee 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ + M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e776a35f21f..5e15084aa7d 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -85,6 +85,7 @@ void MergedBlockOutputStream::writeSuffix() void MergedBlockOutputStream::writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync, const NamesAndTypesList * total_columns_list, MergeTreeData::DataPart::Checksums * additional_column_checksums) { @@ -95,9 +96,9 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( checksums = std::move(*additional_column_checksums); /// Finish columns serialization. - writer->finishDataSerialization(checksums); - writer->finishPrimaryIndexSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishPrimaryIndexSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); NamesAndTypesList part_columns; if (!total_columns_list) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 1a8bf9da822..002ef78a9af 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -46,6 +46,7 @@ public: /// Finilize writing part and fill inner structures void writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync = false, const NamesAndTypesList * total_columns_list = nullptr, MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 1faadd0d720..e767fb3f155 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -63,12 +63,15 @@ void MergedColumnOnlyOutputStream::writeSuffix() } MergeTreeData::DataPart::Checksums -MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums) +MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( + MergeTreeData::MutableDataPartPtr & new_part, + MergeTreeData::DataPart::Checksums & all_checksums, + bool sync) { /// Finish columns serialization. MergeTreeData::DataPart::Checksums checksums; - writer->finishDataSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); auto columns = new_part->getColumns(); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 902138ced9d..507a964ede0 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -27,7 +27,7 @@ public: void write(const Block & block) override; void writeSuffix() override; MergeTreeData::DataPart::Checksums - writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums); + writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums, bool sync = false); private: Block header; From b2aa565a37076230af2ceaa32ee21fa351d37931 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 27 Jun 2020 00:55:48 +0300 Subject: [PATCH 002/535] allow to turn on fsync on inserts, merges and fetches --- src/Common/FileSyncGuard.h | 41 +++++++++++++++++++ src/Disks/DiskLocal.cpp | 35 ++++++++++------ src/Disks/DiskLocal.h | 6 ++- src/Disks/DiskMemory.cpp | 20 ++++++--- src/Disks/DiskMemory.h | 6 ++- src/Disks/IDisk.h | 12 ++++-- src/Disks/S3/DiskS3.cpp | 21 +++++++--- src/Disks/S3/DiskS3.h | 6 ++- src/Storages/MergeTree/DataPartsExchange.cpp | 16 +++++++- src/Storages/MergeTree/DataPartsExchange.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 15 ++++--- .../MergeTree/MergeTreeDataWriter.cpp | 12 +++--- src/Storages/MergeTree/MergeTreeSettings.h | 2 + 14 files changed, 154 insertions(+), 44 deletions(-) create mode 100644 src/Common/FileSyncGuard.h diff --git a/src/Common/FileSyncGuard.h b/src/Common/FileSyncGuard.h new file mode 100644 index 00000000000..5ec9b1d0c98 --- /dev/null +++ b/src/Common/FileSyncGuard.h @@ -0,0 +1,41 @@ +#pragma once + +#include + +namespace DB +{ + +/// Helper class, that recieves file descriptor and does fsync for it in destructor. +/// It's used to keep descriptor open, while doing some operations with it, and do fsync at the end. +/// Guaranties of sequence 'close-reopen-fsync' may depend on kernel version. +/// Source: linux-fsdevel mailing-list https://marc.info/?l=linux-fsdevel&m=152535409207496 +class FileSyncGuard +{ +public: + /// NOTE: If you have already opened descriptor, it's preffered to use + /// this constructor instead of construnctor with path. + FileSyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} + + FileSyncGuard(const DiskPtr & disk_, const String & path) + : disk(disk_), fd(disk_->open(path, O_RDONLY)) {} + + ~FileSyncGuard() + { + try + { + disk->sync(fd); + disk->close(fd); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + +private: + DiskPtr disk; + int fd = -1; +}; + +} + diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index c67bac7ffe2..f85b69baf5e 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_FSYNC; + extern const int CANNOT_CLOSE_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -191,18 +192,6 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } -void DiskLocal::sync(const String & path) const -{ - String full_path = disk_path + path; - int fd = ::open(full_path.c_str(), O_RDONLY); - if (-1 == fd) - throwFromErrnoWithPath("Cannot open file " + full_path, full_path, - errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); - - if (-1 == fsync(fd)) - throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); -} - DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); @@ -299,6 +288,28 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } +int DiskLocal::open(const String & path, mode_t mode) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), mode); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + return fd; +} + +void DiskLocal::close(int fd) const +{ + if (-1 == ::close(fd)) + throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); +} + +void DiskLocal::sync(int fd) const +{ + if (-1 == ::fsync(fd)) + throw Exception("Cannot fsync", ErrorCodes::CANNOT_FSYNC); +} + DiskPtr DiskLocalReservation::getDisk(size_t i) const { if (i != 0) diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 743ba2ceb10..d70ac06c18b 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,8 +59,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -101,6 +99,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 5b3350e40f7..a7f1df04e1f 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,11 +261,6 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } -void DiskMemory::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); -} - DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); @@ -413,6 +408,21 @@ void DiskMemory::setReadOnly(const String &) throw Exception("Method setReadOnly is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +int DiskMemory::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + using DiskMemoryPtr = std::shared_ptr; diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index 8a3ddf05aa7..7f111fe5e7d 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,8 +52,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -92,6 +90,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: void createDirectoriesImpl(const String & path); void replaceFileImpl(const String & from_path, const String & to_path); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 8de77a560d1..bc5c9381643 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,9 +105,6 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; - /// Do fsync on directory. - virtual void sync(const String & path) const = 0; - /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; @@ -174,6 +171,15 @@ public: /// Create hardlink from `src_path` to `dst_path`. virtual void createHardLink(const String & src_path, const String & dst_path) = 0; + + /// Wrapper for POSIX open + virtual int open(const String & path, mode_t mode) const = 0; + + /// Wrapper for POSIX close + virtual void close(int fd) const = 0; + + /// Wrapper for POSIX fsync + virtual void sync(int fd) const = 0; }; using DiskPtr = std::shared_ptr; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 292f6567df4..3e0fb05ed6f 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int CANNOT_SEEK_THROUGH_FILE; extern const int UNKNOWN_FORMAT; extern const int INCORRECT_DISK_INDEX; + extern const int NOT_IMPLEMENTED; } namespace @@ -466,11 +467,6 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } -void DiskS3::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); -} - void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) @@ -669,6 +665,21 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } +int DiskS3::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskS3Reservation::~DiskS3Reservation() { try diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 09132367ae8..cbf161da561 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,8 +58,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; @@ -98,6 +96,10 @@ public: void setReadOnly(const String & path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 6796e630ff2..e7bb8206cd9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -224,9 +225,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( int server_protocol_version = parse(in.getResponseCookie("server_protocol_version", "0")); ReservationPtr reservation; + size_t sum_files_size = 0; if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) { - size_t sum_files_size; readBinary(sum_files_size, in); if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS) { @@ -247,7 +248,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); + bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + + return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( @@ -255,6 +259,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in) { @@ -276,6 +281,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, part_download_path); + MergeTreeData::DataPart::Checksums checksums; for (size_t i = 0; i < files; ++i) { @@ -316,6 +325,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( if (file_name != "checksums.txt" && file_name != "columns.txt") checksums.addFile(file_name, file_size, expected_hash); + + if (sync) + hashing_out.sync(); } assertEOF(in); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index c1aff6bdba5..e983d6deecf 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -71,6 +71,7 @@ private: const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 61dfeed6b7c..ab9bb7879aa 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -664,6 +665,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String from = getFullRelativePath(); String to = storage.relative_data_path + new_relative_path + "/"; + std::optional sync_guard; + if (storage.getSettings()->sync_part_directory) + sync_guard.emplace(volume->getDisk(), to); + if (!volume->getDisk()->exists(from)) throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is logical error.", ErrorCodes::FILE_DOESNT_EXIST); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ccd7f234925..9c8c4e3c1d5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -695,6 +696,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor gathering_column_names.clear(); } + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ @@ -991,9 +996,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor else to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } @@ -1089,6 +1091,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /// Don't change granularity type while mutating subset of columns auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); @@ -1187,9 +1193,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index cf8860b7f04..01f0b086cea 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents @@ -259,7 +260,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - new_data_part->volume->getDisk()->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); + + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) @@ -309,10 +315,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa out.writeWithPermutation(block, perm_ptr); out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); - /// Sync part directory. - if (sync_on_insert) - new_data_part->volume->getDisk()->sync(full_path); - ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterCompressedBytes, new_data_part->getBytesOnDisk()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index da2c9ee49ee..c559ce2804e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -45,7 +45,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From ca346ea13cd0ad0f02a29d59302584c826b52298 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 Jul 2020 02:41:37 +0300 Subject: [PATCH 003/535] rename fsync-related settings --- src/Storages/MergeTree/DataPartsExchange.cpp | 6 +++--- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 10 +++++----- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index e7bb8206cd9..72b478cf587 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -248,8 +248,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch - && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + bool sync = (data_settings->min_compressed_bytes_to_fsync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_fsync_after_fetch); return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } @@ -282,7 +282,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, part_download_path); MergeTreeData::DataPart::Checksums checksums; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ab9bb7879aa..3d8cb6b7fc5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -666,7 +666,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String to = storage.relative_data_path + new_relative_path + "/"; std::optional sync_guard; - if (storage.getSettings()->sync_part_directory) + if (storage.getSettings()->fsync_part_directory) sync_guard.emplace(volume->getDisk(), to); if (!volume->getDisk()->exists(from)) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9c8c4e3c1d5..c39d1981031 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -579,8 +579,8 @@ public: static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) { - return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) - || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); + return ((settings.min_rows_to_fsync_after_merge && input_rows >= settings.min_rows_to_fsync_after_merge) + || (settings.min_compressed_bytes_to_fsync_after_merge && input_bytes >= settings.min_compressed_bytes_to_fsync_after_merge)); } @@ -697,7 +697,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /** Read from all parts, merge and write into a new one. @@ -1092,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /// Don't change granularity type while mutating subset of columns diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 01f0b086cea..23210fc604e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -264,7 +264,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa disk->createDirectories(full_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. @@ -309,7 +309,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); - bool sync_on_insert = data.getSettings()->sync_after_insert; + bool sync_on_insert = data.getSettings()->fsync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c559ce2804e..eeee0c4b1e1 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,11 +43,11 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ - M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ - M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ - M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(SettingUInt64, min_rows_to_fsync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ + M(SettingBool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From d6434f61dc7b08072862d4d10ea6fa9da781b6c1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Jul 2020 03:15:02 +0300 Subject: [PATCH 004/535] support codecs in compact parts --- .../MergeTreeDataPartWriterCompact.cpp | 80 +++++++++++++------ .../MergeTreeDataPartWriterCompact.h | 21 ++++- .../01375_compact_parts_codecs.reference | 3 + .../01375_compact_parts_codecs.sql | 31 +++++++ 4 files changed, 109 insertions(+), 26 deletions(-) create mode 100644 tests/queries/0_stateless/01375_compact_parts_codecs.reference create mode 100644 tests/queries/0_stateless/01375_compact_parts_codecs.sql diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index f7a3ad75cf5..696197aa4ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -14,19 +14,23 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter( data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) + , plain_file(data_part->volume->getDisk()->writeFile( + part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, + settings.max_compress_block_size, + WriteMode::Rewrite, + settings.estimated_size, + settings.aio_threshold)) + , plain_hashing(*plain_file) + , marks_file(data_part->volume->getDisk()->writeFile( + part_path + MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension_, + 4096, + WriteMode::Rewrite)) + , marks(*marks_file) { - using DataPart = MergeTreeDataPartCompact; - String data_file_name = DataPart::DATA_FILE_NAME; - - stream = std::make_unique( - data_file_name, - data_part->volume->getDisk(), - part_path + data_file_name, DataPart::DATA_FILE_EXTENSION, - part_path + data_file_name, marks_file_extension, - default_codec, - settings.max_compress_block_size, - settings.estimated_size, - settings.aio_threshold); + const auto & storage_columns = metadata_snapshot->getColumns(); + for (const auto & column : columns_list) + compressed_streams[column.name] = std::make_unique( + plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); } void MergeTreeDataPartWriterCompact::write( @@ -98,14 +102,13 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) for (const auto & column : columns_list) { - /// There could already be enough data to compress into the new block. - if (stream->compressed.offset() >= settings.min_compress_block_size) - stream->compressed.next(); + auto & stream = compressed_streams[column.name]; - writeIntBinary(stream->plain_hashing.count(), stream->marks); - writeIntBinary(stream->compressed.offset(), stream->marks); + writeIntBinary(plain_hashing.count(), marks); + writeIntBinary(UInt64(0), marks); writeColumnSingleGranule(block.getByName(column.name), current_row, rows_to_write); + stream->hashing_buf.next(); } ++from_mark; @@ -120,7 +123,7 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) index_granularity.appendMark(rows_written); } - writeIntBinary(rows_to_write, stream->marks); + writeIntBinary(rows_to_write, marks); } next_index_offset = 0; @@ -132,7 +135,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy IDataType::SerializeBinaryBulkStatePtr state; IDataType::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.getter = [this](IDataType::SubstreamPath) -> WriteBuffer * { return &stream->compressed; }; + serialize_settings.getter = [this, &column](IDataType::SubstreamPath) -> WriteBuffer * { return &compressed_streams.at(column.name)->hashing_buf; }; serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; @@ -150,15 +153,15 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: { for (size_t i = 0; i < columns_list.size(); ++i) { - writeIntBinary(stream->plain_hashing.count(), stream->marks); - writeIntBinary(stream->compressed.offset(), stream->marks); + writeIntBinary(plain_hashing.count(), marks); + writeIntBinary(UInt64(0), marks); } - writeIntBinary(0ULL, stream->marks); + writeIntBinary(UInt64(0), marks); } - stream->finalize(); - stream->addToChecksums(checksums); - stream.reset(); + plain_file->next(); + marks.next(); + addToChecksums(checksums); } static void fillIndexGranularityImpl( @@ -199,6 +202,33 @@ void MergeTreeDataPartWriterCompact::fillIndexGranularity(size_t index_granulari rows_in_block); } +void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & checksums) +{ + using uint128 = CityHash_v1_0_2::uint128; + + String data_file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; + String marks_file_name = MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension; + + checksums.files[data_file_name].is_compressed = true; + size_t uncompressed_size = 0; + uint128 uncompressed_hash{0, 0}; + + for (const auto & [_, stream] : compressed_streams) + { + uncompressed_size += stream->hashing_buf.count(); + uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( + reinterpret_cast(&uncompressed_hash), sizeof(uncompressed_hash), uncompressed_hash); + } + + checksums.files[data_file_name].uncompressed_size = uncompressed_size; + checksums.files[data_file_name].uncompressed_hash = uncompressed_hash; + checksums.files[data_file_name].file_size = plain_hashing.count(); + checksums.files[data_file_name].file_hash = plain_hashing.getHash(); + + checksums.files[marks_file_name].file_size = marks.count(); + checksums.files[marks_file_name].file_hash = marks.getHash(); +} + void MergeTreeDataPartWriterCompact::ColumnsBuffer::add(MutableColumns && columns) { if (accumulated_columns.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8183c038c4c..a5bfd8a16cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -34,7 +34,7 @@ private: void writeBlock(const Block & block); - StreamPtr stream; + void addToChecksums(MergeTreeDataPartChecksums & checksumns); Block header; @@ -53,6 +53,25 @@ private: }; ColumnsBuffer columns_buffer; + + /// compressed -> compressed_buf -> plain_hashing -> plain_file + std::unique_ptr plain_file; + HashingWriteBuffer plain_hashing; + + struct CompressedStream + { + CompressedWriteBuffer compressed_buf; + HashingWriteBuffer hashing_buf; + + CompressedStream(WriteBuffer & buf, const CompressionCodecPtr & codec) + : compressed_buf(buf, codec), hashing_buf(compressed_buf) {} + }; + + std::unordered_map> compressed_streams; + + /// marks -> marks_file + std::unique_ptr marks_file; + HashingWriteBuffer marks; }; } diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.reference b/tests/queries/0_stateless/01375_compact_parts_codecs.reference new file mode 100644 index 00000000000..982c45a26e3 --- /dev/null +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.reference @@ -0,0 +1,3 @@ +12000 11890 +11965 11890 +5858 11890 diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.sql b/tests/queries/0_stateless/01375_compact_parts_codecs.sql new file mode 100644 index 00000000000..467745c6fa2 --- /dev/null +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS codecs; + +CREATE TABLE codecs (id UInt32, val UInt32, s String) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; + +CREATE TABLE codecs (id UInt32 CODEC(NONE), val UInt32 CODEC(NONE), s String CODEC(NONE)) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; + +CREATE TABLE codecs (id UInt32, val UInt32 CODEC(Delta, ZSTD), s String CODEC(ZSTD)) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; From 80a62977f1aa430144a6bdfae0b7e37605eb5b20 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 9 Jul 2020 21:26:54 +0300 Subject: [PATCH 005/535] fix hashing in DataPartWriterCompact --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 69e581a6299..ac697e1b212 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -217,8 +217,9 @@ void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & for (const auto & [_, stream] : compressed_streams) { uncompressed_size += stream->hashing_buf.count(); + auto stream_hash = stream->hashing_buf.getHash(); uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( - reinterpret_cast(&uncompressed_hash), sizeof(uncompressed_hash), uncompressed_hash); + reinterpret_cast(&stream_hash), sizeof(stream_hash), uncompressed_hash); } checksums.files[data_file_name].uncompressed_size = uncompressed_size; From 24f627e52c5f6f461cd1bc42b2306725ad0491b8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 10 Jul 2020 18:57:10 +0300 Subject: [PATCH 006/535] fix reading from compact parts with different codecs --- .../CachedCompressedReadBuffer.cpp | 14 +++- src/Compression/CachedCompressedReadBuffer.h | 4 +- .../CompressedReadBufferFromFile.cpp | 6 ++ .../CompressedReadBufferFromFile.h | 1 + .../MergeTree/MergeTreeReaderCompact.cpp | 80 ++++++++++--------- .../MergeTree/MergeTreeReaderCompact.h | 21 ++++- 6 files changed, 81 insertions(+), 45 deletions(-) diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index 1b083c004c0..beb13d15f01 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -19,8 +20,9 @@ void CachedCompressedReadBuffer::initInput() { if (!file_in) { - file_in = file_in_creator(); - compressed_in = file_in.get(); + file_in_holder = file_in_creator(); + file_in = file_in_holder.get(); + compressed_in = file_in; if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); @@ -71,6 +73,14 @@ bool CachedCompressedReadBuffer::nextImpl() return true; } +CachedCompressedReadBuffer::CachedCompressedReadBuffer( + const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_) + : ReadBuffer(nullptr, 0), file_in(file_in_), cache(cache_), path(path_), file_pos(0) +{ + if (file_in == nullptr) + throw Exception("Neither file_in nor file_in_creator is initialized in CachedCompressedReadBuffer", ErrorCodes::LOGICAL_ERROR); +} + CachedCompressedReadBuffer::CachedCompressedReadBuffer( const std::string & path_, std::function()> file_in_creator_, UncompressedCache * cache_) : ReadBuffer(nullptr, 0), file_in_creator(std::move(file_in_creator_)), cache(cache_), path(path_), file_pos(0) diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 88bcec8197d..2c5aa4920bd 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -22,7 +22,8 @@ class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadB private: std::function()> file_in_creator; UncompressedCache * cache; - std::unique_ptr file_in; + std::unique_ptr file_in_holder; + ReadBufferFromFileBase * file_in; const std::string path; size_t file_pos; @@ -38,6 +39,7 @@ private: clockid_t clock_type {}; public: + CachedCompressedReadBuffer(const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_); CachedCompressedReadBuffer(const std::string & path, std::function()> file_in_creator, UncompressedCache * cache_); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index ddd8bba686f..2927ee1b399 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -37,6 +37,12 @@ bool CompressedReadBufferFromFile::nextImpl() return true; } +CompressedReadBufferFromFile::CompressedReadBufferFromFile(ReadBufferFromFileBase & file_in_) + : BufferWithOwnMemory(0), file_in(file_in_) +{ + compressed_in = &file_in; +} + CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf) : BufferWithOwnMemory(0), p_file_in(std::move(buf)), file_in(*p_file_in) { diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index 1729490f606..1de28062e41 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -28,6 +28,7 @@ private: bool nextImpl() override; public: + CompressedReadBufferFromFile(ReadBufferFromFileBase & buf); CompressedReadBufferFromFile(std::unique_ptr buf); CompressedReadBufferFromFile( diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 4357ee66a6e..920f171d7f9 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -45,40 +45,31 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { size_t buffer_size = settings.max_read_buffer_size; const String full_data_path = data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; + file_in = data_part->volume->getDisk()->readFile( + full_data_path, buffer_size, 0, + settings.min_bytes_to_use_direct_io, + settings.min_bytes_to_use_mmap_io); - if (uncompressed_cache) + auto full_path = fullPath(data_part->volume->getDisk(), full_data_path); + for (const auto & column : columns) { - auto buffer = std::make_unique( - fullPath(data_part->volume->getDisk(), full_data_path), - [this, full_data_path, buffer_size]() - { - return data_part->volume->getDisk()->readFile( - full_data_path, - buffer_size, - 0, - settings.min_bytes_to_use_direct_io, - settings.min_bytes_to_use_mmap_io); - }, - uncompressed_cache); + + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; + if (uncompressed_cache) + { + cached_buffer = std::make_unique(full_path, file_in.get(), uncompressed_cache); + if (profile_callback_) + cached_buffer->setProfileCallback(profile_callback_, clock_type_); + } + else + { + non_cached_buffer = std::make_unique(*file_in); + if (profile_callback_) + non_cached_buffer->setProfileCallback(profile_callback_, clock_type_); + } - if (profile_callback_) - buffer->setProfileCallback(profile_callback_, clock_type_); - - cached_buffer = std::move(buffer); - data_buffer = cached_buffer.get(); - } - else - { - auto buffer = - std::make_unique( - data_part->volume->getDisk()->readFile( - full_data_path, buffer_size, 0, settings.min_bytes_to_use_direct_io, settings.min_bytes_to_use_mmap_io)); - - if (profile_callback_) - buffer->setProfileCallback(profile_callback_, clock_type_); - - non_cached_buffer = std::move(buffer); - data_buffer = non_cached_buffer.get(); + column_streams[column.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; } size_t columns_num = columns.size(); @@ -181,15 +172,16 @@ void MergeTreeReaderCompact::readData( const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) { + auto & stream = column_streams[name]; if (!isContinuousReading(from_mark, column_position)) - seekToMark(from_mark, column_position); + seekToMark(stream, from_mark, column_position); auto buffer_getter = [&](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != IDataType::Substream::ArraySizes)) return nullptr; - return data_buffer; + return stream.data_buffer; }; IDataType::DeserializeBinaryBulkSettings deserialize_settings; @@ -209,15 +201,15 @@ void MergeTreeReaderCompact::readData( } -void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index) +void MergeTreeReaderCompact::seekToMark(ColumnStream & stream, size_t row_index, size_t column_index) { MarkInCompressedFile mark = marks_loader.getMark(row_index, column_index); try { - if (cached_buffer) - cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); - if (non_cached_buffer) - non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (stream.cached_buffer) + stream.cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (stream.non_cached_buffer) + stream.non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); } catch (Exception & e) { @@ -239,4 +231,16 @@ bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_posi || (mark == last_mark + 1 && column_position == 0 && last_column == data_part->getColumns().size() - 1); } +MergeTreeReaderCompact::ColumnStream::ColumnStream( + std::unique_ptr cached_buffer_, + std::unique_ptr non_cached_buffer_) + : cached_buffer(std::move(cached_buffer_)) + , non_cached_buffer(std::move(non_cached_buffer_)) +{ + if (cached_buffer) + data_buffer = cached_buffer.get(); + else + data_buffer = non_cached_buffer.get(); +} + } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 0457b4b6a50..41682f8b0bd 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -35,9 +36,21 @@ public: private: bool isContinuousReading(size_t mark, size_t column_position); - ReadBuffer * data_buffer; - std::unique_ptr cached_buffer; - std::unique_ptr non_cached_buffer; + std::unique_ptr file_in; + + struct ColumnStream + { + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; + ReadBuffer * data_buffer; + + ColumnStream() = default; + ColumnStream( + std::unique_ptr cached_buffer_, + std::unique_ptr non_cached_buffer_); + }; + + std::unordered_map column_streams; MergeTreeMarksLoader marks_loader; @@ -49,7 +62,7 @@ private: size_t next_mark = 0; std::optional> last_read_granule; - void seekToMark(size_t row_index, size_t column_index); + void seekToMark(ColumnStream & stream, size_t row_index, size_t column_index); void readData(const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets = false); From fbec940e0fa7246ca7b42e056de7d0cea50640d0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 11 Jul 2020 02:33:36 +0300 Subject: [PATCH 007/535] fix reading and check query for compact parts with different codecs --- .../CachedCompressedReadBuffer.cpp | 2 ++ src/Compression/CachedCompressedReadBuffer.h | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 4 ++-- .../MergeTree/MergeTreeReaderCompact.cpp | 5 +++-- src/Storages/MergeTree/checkDataPart.cpp | 22 ++++++++++++++----- .../01390_check_table_codec.reference | 2 ++ .../0_stateless/01390_check_table_codec.sql | 15 +++++++++++++ 7 files changed, 41 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/01390_check_table_codec.reference create mode 100644 tests/queries/0_stateless/01390_check_table_codec.sql diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index beb13d15f01..218925f8eae 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -79,6 +79,8 @@ CachedCompressedReadBuffer::CachedCompressedReadBuffer( { if (file_in == nullptr) throw Exception("Neither file_in nor file_in_creator is initialized in CachedCompressedReadBuffer", ErrorCodes::LOGICAL_ERROR); + + compressed_in = file_in; } CachedCompressedReadBuffer::CachedCompressedReadBuffer( diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 2c5aa4920bd..89bf66a3e2c 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -23,7 +23,7 @@ private: std::function()> file_in_creator; UncompressedCache * cache; std::unique_ptr file_in_holder; - ReadBufferFromFileBase * file_in; + ReadBufferFromFileBase * file_in = nullptr; const std::string path; size_t file_pos; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index ac697e1b212..d15bba232d6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -17,7 +17,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( default_codec_, settings_, index_granularity_) , plain_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, - settings.max_compress_block_size, + settings.max_compress_block_size, WriteMode::Rewrite, settings.estimated_size, settings.aio_threshold)) @@ -31,7 +31,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const auto & storage_columns = metadata_snapshot->getColumns(); for (const auto & column : columns_list) compressed_streams[column.name] = std::make_unique( - plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); + plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); } void MergeTreeDataPartWriterCompact::write( diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 920f171d7f9..89ca8b96dba 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -53,7 +53,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( auto full_path = fullPath(data_part->volume->getDisk(), full_data_path); for (const auto & column : columns) { - + std::unique_ptr cached_buffer; std::unique_ptr non_cached_buffer; if (uncompressed_cache) @@ -69,7 +69,8 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( non_cached_buffer->setProfileCallback(profile_callback_, clock_type_); } - column_streams[column.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; + auto column_from_part = getColumnFromPart(column); + column_streams[column_from_part.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; } size_t columns_num = columns.size(); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 793bddc88c0..790a250d831 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -96,11 +96,24 @@ IMergeTreeDataPart::Checksums checkDataPart( }; }; + /// This function calculates only checksum of file content (compressed or uncompressed). + auto checksum_file = [](const DiskPtr & disk_, const String & file_path) + { + auto file_buf = disk_->readFile(file_path); + HashingReadBuffer hashing_buf(*file_buf); + hashing_buf.tryIgnore(std::numeric_limits::max()); + return IMergeTreeDataPart::Checksums::Checksum{hashing_buf.count(), hashing_buf.getHash()}; + }; + + bool check_uncompressed = true; /// First calculate checksums for columns data if (part_type == MergeTreeDataPartType::COMPACT) { const auto & file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; - checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name); + checksums_data.files[file_name] = checksum_file(disk, path + file_name); + /// Uncompressed checksums in compact parts are computed in a complex way. + /// We check only checksum of compressed file. + check_uncompressed = false; } else if (part_type == MergeTreeDataPartType::WIDE) { @@ -141,10 +154,7 @@ IMergeTreeDataPart::Checksums checkDataPart( if (txt_checksum_it == checksum_files_txt.end() || txt_checksum_it->second.uncompressed_size == 0) { /// The file is not compressed. - auto file_buf = disk->readFile(it->path()); - HashingReadBuffer hashing_buf(*file_buf); - hashing_buf.tryIgnore(std::numeric_limits::max()); - checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); + checksums_data.files[file_name] = checksum_file(disk, it->path()); } else /// If we have both compressed and uncompressed in txt, than calculate them { @@ -157,7 +167,7 @@ IMergeTreeDataPart::Checksums checkDataPart( return {}; if (require_checksums || !checksums_txt.files.empty()) - checksums_txt.checkEqual(checksums_data, true); + checksums_txt.checkEqual(checksums_data, check_uncompressed); return checksums_data; } diff --git a/tests/queries/0_stateless/01390_check_table_codec.reference b/tests/queries/0_stateless/01390_check_table_codec.reference new file mode 100644 index 00000000000..3025e6463d8 --- /dev/null +++ b/tests/queries/0_stateless/01390_check_table_codec.reference @@ -0,0 +1,2 @@ +all_1_1_0 1 +all_1_1_0 1 diff --git a/tests/queries/0_stateless/01390_check_table_codec.sql b/tests/queries/0_stateless/01390_check_table_codec.sql new file mode 100644 index 00000000000..639d5bea6e4 --- /dev/null +++ b/tests/queries/0_stateless/01390_check_table_codec.sql @@ -0,0 +1,15 @@ +SET check_query_single_value_result = 0; + +DROP TABLE IF EXISTS check_codec; + +CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); +CHECK TABLE check_codec; + +DROP TABLE check_codec; + +CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = '10M'; +INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); +CHECK TABLE check_codec; + +DROP TABLE check_codec; From 9384b6950b6c5311202788c8b38ed84dd53a13e8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Jul 2020 15:10:20 +0300 Subject: [PATCH 008/535] Add some tests configs --- tests/ci/build_config.json | 153 +++++++++++++++++++++++ tests/ci/tests_config.json | 242 +++++++++++++++++++++++++++++++++++++ 2 files changed, 395 insertions(+) create mode 100644 tests/ci/build_config.json create mode 100644 tests/ci/tests_config.json diff --git a/tests/ci/build_config.json b/tests/ci/build_config.json new file mode 100644 index 00000000000..e4b9c1d6b75 --- /dev/null +++ b/tests/ci/build_config.json @@ -0,0 +1,153 @@ +[ + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "alien_pkgs": true, + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "performance", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "address", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "undefined", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "thread", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "memory", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "debug", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "unbundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "splitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-darwin", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-aarch64", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-freebsd", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + } +] diff --git a/tests/ci/tests_config.json b/tests/ci/tests_config.json new file mode 100644 index 00000000000..481de51d08b --- /dev/null +++ b/tests/ci/tests_config.json @@ -0,0 +1,242 @@ +{ + "Functional stateful tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (unbundled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "unbundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, polymorphic parts enabled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (undefined)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + } +} From a88e391bd2f6812bd7d39db267f2b5bc77b0daa3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Jul 2020 06:21:18 +0000 Subject: [PATCH 009/535] Fix bug with insert, simplify exchanges logic --- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 3 + .../ReadBufferFromRabbitMQConsumer.cpp | 246 +++++++----------- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 9 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 38 ++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 5 +- .../WriteBufferToRabbitMQProducer.cpp | 52 ++-- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 15 +- .../integration/test_storage_rabbitmq/test.py | 130 +++++---- 8 files changed, 236 insertions(+), 262 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 1a03fc4969e..2559b31c44a 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -56,6 +56,9 @@ void RabbitMQBlockOutputStream::write(const Block & block) void RabbitMQBlockOutputStream::writeSuffix() { child->writeSuffix(); + + if (buffer) + buffer->finilizeProducer(); } } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 6bd5c36e757..d59e9c9eade 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,20 +14,9 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - namespace ExchangeType { - /// Note that default here means default by implementation and not by rabbitmq settings - static const String DEFAULT = "default"; - static const String FANOUT = "fanout"; - static const String DIRECT = "direct"; - static const String TOPIC = "topic"; - static const String HASH = "consistent_hash"; - static const String HEADERS = "headers"; + static const String HASH_SUF = "_hash"; } static const auto QUEUE_SIZE = 50000; /// Equals capacity of a single rabbitmq queue @@ -36,34 +25,31 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, HandlerPtr event_handler_, const String & exchange_name_, + const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, Poco::Logger * log_, char row_delimiter_, - bool bind_by_id_, + bool hash_exchange_, size_t num_queues_, - const String & exchange_type_, const String & local_exchange_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer_channel(std::move(consumer_channel_)) , event_handler(event_handler_) , exchange_name(exchange_name_) + , exchange_type(exchange_type_) , routing_keys(routing_keys_) , channel_id(channel_id_) - , bind_by_id(bind_by_id_) + , hash_exchange(hash_exchange_) , num_queues(num_queues_) - , exchange_type(exchange_type_) , local_exchange(local_exchange_) - , local_default_exchange(local_exchange + "_" + ExchangeType::DIRECT) - , local_hash_exchange(local_exchange + "_" + ExchangeType::HASH) + , local_hash_exchange(local_exchange + ExchangeType::HASH_SUF) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) , messages(QUEUE_SIZE * num_queues) { - exchange_type_set = exchange_type != ExchangeType::DEFAULT; - /* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added. * By default there is one queue per consumer. */ @@ -86,67 +72,24 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() void ReadBufferFromRabbitMQConsumer::initExchange() { - /* This direct-exchange is used for default implemenation and for INSERT query (so it is always declared). If exchange_type - * is not set, then there are only two exchanges - external, defined by the client, and local, unique for each table (default). - * This strict division to external and local exchanges is needed to avoid too much complexity with defining exchange_name - * for INSERT query producer and, in general, it is better to distinguish them into separate ones. - */ - consumer_channel->declareExchange(local_default_exchange, AMQP::direct).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare local direct-exchange. Reason: {}", message); - }); - - if (!exchange_type_set) - { - consumer_channel->declareExchange(exchange_name, AMQP::fanout).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare default fanout-exchange. Reason: {}", message); - }); - - /// With fanout exchange the binding key is ignored - a parameter might be arbitrary. All distribution lies on local_exchange. - consumer_channel->bindExchange(exchange_name, local_default_exchange, routing_keys[0]).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local direct-exchange to fanout-exchange. Reason: {}", message); - }); - - return; - } - - AMQP::ExchangeType type; - if (exchange_type == ExchangeType::FANOUT) type = AMQP::ExchangeType::fanout; - else if (exchange_type == ExchangeType::DIRECT) type = AMQP::ExchangeType::direct; - else if (exchange_type == ExchangeType::TOPIC) type = AMQP::ExchangeType::topic; - else if (exchange_type == ExchangeType::HASH) type = AMQP::ExchangeType::consistent_hash; - else if (exchange_type == ExchangeType::HEADERS) type = AMQP::ExchangeType::headers; - else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); - /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which - * will evenly distribute messages between all consumers. (This enables better scaling as without hash-exchange - the only - * option to avoid getting the same messages more than once - is having only one consumer with one queue) + * will evenly distribute messages between all consumers. */ - consumer_channel->declareExchange(exchange_name, type).onError([&](const char * message) + consumer_channel->declareExchange(exchange_name, exchange_type).onError([&](const char * message) { local_exchange_declared = false; LOG_ERROR(log, "Failed to declare client's {} exchange. Reason: {}", exchange_type, message); }); /// No need for declaring hash-exchange if there is only one consumer with one queue or exchange type is already hash - if (!bind_by_id) + if (!hash_exchange || exchange_type == AMQP::ExchangeType::consistent_hash) return; - hash_exchange = true; - - if (exchange_type == ExchangeType::HASH) - return; - - /* By default hash exchange distributes messages based on a hash value of a routing key, which must be a string integer. But - * in current case we use hash exchange for binding to another exchange of some other type, which needs its own routing keys - * of other types: headers, patterns and string-keys. This means that hash property must be changed. - */ { + /* By default hash exchange distributes messages based on a hash value of a routing key, which must be a string integer. But + * in current case we use hash exchange for binding to another exchange of some other type, which needs its own routing keys + * of other types: headers, patterns and string-keys. This means that hash property must be changed. + */ AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; @@ -161,7 +104,7 @@ void ReadBufferFromRabbitMQConsumer::initExchange() /// Then bind client's exchange to sharding exchange (by keys, specified by the client): - if (exchange_type == ExchangeType::HEADERS) + if (exchange_type == AMQP::ExchangeType::headers) { AMQP::Table binding_arguments; std::vector matching; @@ -181,6 +124,14 @@ void ReadBufferFromRabbitMQConsumer::initExchange() LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); }); } + else if (exchange_type == AMQP::ExchangeType::fanout) + { + consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_keys[0]).onError([&](const char * message) + { + local_exchange_declared = false; + LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); + }); + } else { for (const auto & routing_key : routing_keys) @@ -198,30 +149,31 @@ void ReadBufferFromRabbitMQConsumer::initExchange() void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { /// These variables might be updated later from a separate thread in onError callbacks. - if (!local_exchange_declared || (exchange_type_set && !local_hash_exchange_declared)) + if (!local_exchange_declared || (hash_exchange && !local_hash_exchange_declared)) { initExchange(); local_exchange_declared = true; local_hash_exchange_declared = true; } - bool default_bindings_created = false, default_bindings_error = false; bool bindings_created = false, bindings_error = false; consumer_channel->declareQueue(AMQP::exclusive) .onSuccess([&](const std::string & queue_name_, int /* msgcount */, int /* consumercount */) { queues.emplace_back(queue_name_); + LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); + subscribed_queue[queue_name_] = false; - - String binding_key = routing_keys[0]; - - /* Every consumer has at least one unique queue. Bind the queues to exchange based on the consumer_channel_id - * in case there is one queue per consumer and bind by queue_id in case there is more than 1 queue per consumer. - * (queue_id is based on channel_id) + /* Subscription can probably be moved back to readPrefix(), but not sure whether it is better in regard to speed, because + * if moved there, it must(!) be wrapped inside a channel->onSuccess callback or any other, otherwise + * consumer might fail to subscribe and no resubscription will help. */ - if (bind_by_id || hash_exchange) + subscribe(queues.back()); + + if (hash_exchange) { + String binding_key; if (queues.size() == 1) { binding_key = std::to_string(channel_id); @@ -230,39 +182,67 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { binding_key = std::to_string(channel_id + queue_id); } + /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor + * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. + */ + String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_hash_exchange; + + /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. + consumer_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) + .onSuccess([&] + { + bindings_created = true; + }) + .onError([&](const char * message) + { + bindings_error = true; + LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); + }); } - - /// Bind queue to exchange that is used for INSERT query and also for default implementation. - consumer_channel->bindQueue(local_default_exchange, queue_name_, binding_key) - .onSuccess([&] + else if (exchange_type == AMQP::ExchangeType::fanout) { - default_bindings_created = true; - }) - .onError([&](const char * message) - { - default_bindings_error = true; - LOG_ERROR(log, "Failed to bind to key {}. Reason: {}", binding_key, message); - }); - - /* Subscription can probably be moved back to readPrefix(), but not sure whether it is better in regard to speed, because - * if moved there, it must(!) be wrapped inside a channel->onSuccess callback or any other, otherwise - * consumer might fail to subscribe and no resubscription will help. - */ - subscribe(queues.back()); - - LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); - - if (exchange_type_set) - { - if (hash_exchange) + consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) + .onSuccess([&] { - /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor - * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. - */ - String current_hash_exchange = exchange_type == ExchangeType::HASH ? exchange_name : local_hash_exchange; + bindings_created = true; + }) + .onError([&](const char * message) + { + bindings_error = true; + LOG_ERROR(log, "Failed to bind to key. Reason: {}", message); + }); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + AMQP::Table binding_arguments; + std::vector matching; - /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. - consumer_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) + /// It is not parsed for the second time - if it was parsed above, then it would never end up here. + for (const auto & header : routing_keys) + { + boost::split(matching, header, [](char c){ return c == '='; }); + binding_arguments[matching[0]] = matching[1]; + matching.clear(); + } + + consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) + .onSuccess([&] + { + bindings_created = true; + }) + .onError([&](const char * message) + { + bindings_error = true; + LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); + }); + } + else + { + /// Means there is only one queue with one consumer - no even distribution needed - no hash-exchange. + for (const auto & routing_key : routing_keys) + { + /// Binding directly to exchange, specified by the client. + consumer_channel->bindQueue(exchange_name, queue_name_, routing_key) .onSuccess([&] { bindings_created = true; @@ -270,56 +250,14 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) .onError([&](const char * message) { bindings_error = true; - LOG_ERROR(log, "Failed to create queue binding to key {}. Reason: {}", binding_key, message); + LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); }); } - else if (exchange_type == ExchangeType::HEADERS) - { - AMQP::Table binding_arguments; - std::vector matching; - - /// It is not parsed for the second time - if it was parsed above, then it would never end up here. - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - binding_arguments[matching[0]] = matching[1]; - matching.clear(); - } - - consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue to key. Reason: {}", message); - }); - } - else - { - /// Means there is only one queue with one consumer - no even distribution needed - no hash-exchange. - for (const auto & routing_key : routing_keys) - { - /// Binding directly to exchange, specified by the client. - consumer_channel->bindQueue(exchange_name, queue_name_, routing_key) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue to key. Reason: {}", message); - }); - } - } } }) .onError([&](const char * message) { - default_bindings_error = true; + bindings_error = true; LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); @@ -327,7 +265,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) * It is important at this moment to make sure that queue bindings are created before any publishing can happen because * otherwise messages will be routed nowhere. */ - while ((!default_bindings_created && !default_bindings_error) || (exchange_type_set && !bindings_created && !bindings_error)) + while (!bindings_created && !bindings_error) { iterateEventLoop(); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 51ef8ceba3e..82dc3f55248 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -26,13 +26,13 @@ public: ChannelPtr consumer_channel_, HandlerPtr event_handler_, const String & exchange_name_, + const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, Poco::Logger * log_, char row_delimiter_, - bool bind_by_id_, + bool hash_exchange_, size_t num_queues_, - const String & exchange_type_, const String & local_exchange_, const std::atomic & stopped_); @@ -48,12 +48,12 @@ private: HandlerPtr event_handler; const String exchange_name; + const AMQP::ExchangeType exchange_type; const Names routing_keys; const size_t channel_id; - const bool bind_by_id; + const bool hash_exchange; const size_t num_queues; - const String exchange_type; const String local_exchange; const String local_default_exchange; const String local_hash_exchange; @@ -65,7 +65,6 @@ private: String default_local_exchange; bool local_exchange_declared = false, local_hash_exchange_declared = false; - bool exchange_type_set = false, hash_exchange = false; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index a3d16087e34..0d6cf95f39c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -49,6 +49,16 @@ namespace ErrorCodes extern const int CANNOT_CONNECT_RABBITMQ; } +namespace ExchangeType +{ + /// Note that default here means default by implementation and not by rabbitmq settings + static const String DEFAULT = "default"; + static const String FANOUT = "fanout"; + static const String DIRECT = "direct"; + static const String TOPIC = "topic"; + static const String HASH = "consistent_hash"; + static const String HEADERS = "headers"; +} StorageRabbitMQ::StorageRabbitMQ( const StorageID & table_id_, @@ -72,7 +82,6 @@ StorageRabbitMQ::StorageRabbitMQ( , row_delimiter(row_delimiter_) , num_consumers(num_consumers_) , num_queues(num_queues_) - , exchange_type(exchange_type_) , use_transactional_channel(use_transactional_channel_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) @@ -107,7 +116,22 @@ StorageRabbitMQ::StorageRabbitMQ( heartbeat_task = global_context.getSchedulePool().createTask("RabbitMQHeartbeatTask", [this]{ heartbeatFunc(); }); heartbeat_task->deactivate(); - bind_by_id = num_consumers > 1 || num_queues > 1; + hash_exchange = num_consumers > 1 || num_queues > 1; + + exchange_type_set = exchange_type_ != ExchangeType::DEFAULT; + if (exchange_type_set) + { + if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; + else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct; + else if (exchange_type_ == ExchangeType::TOPIC) exchange_type = AMQP::ExchangeType::topic; + else if (exchange_type_ == ExchangeType::HASH) exchange_type = AMQP::ExchangeType::consistent_hash; + else if (exchange_type_ == ExchangeType::HEADERS) exchange_type = AMQP::ExchangeType::headers; + else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); + } + else + { + exchange_type = AMQP::ExchangeType::fanout; + } auto table_id = getStorageID(); String table_name = table_id.table_name; @@ -264,17 +288,17 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, event_handler, exchange_name, routing_keys, - next_channel_id, log, row_delimiter, bind_by_id, num_queues, - exchange_type, local_exchange_name, stream_cancelled); + consumer_channel, event_handler, exchange_name, exchange_type, routing_keys, + next_channel_id, log, row_delimiter, hash_exchange, num_queues, + local_exchange_name, stream_cancelled); } ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( - parsed_address, global_context, login_password, routing_keys[0], local_exchange_name, - log, num_consumers * num_queues, bind_by_id, use_transactional_channel, + parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, + log, num_consumers * num_queues, use_transactional_channel, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index e1c8b33c91e..4457c5ff8c9 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -76,15 +76,15 @@ private: Names routing_keys; const String exchange_name; + AMQP::ExchangeType exchange_type; String local_exchange_name; const String format_name; char row_delimiter; size_t num_consumers; size_t num_created_consumers = 0; - bool bind_by_id; + bool hash_exchange; size_t num_queues; - const String exchange_type; const bool use_transactional_channel; Poco::Logger * log; @@ -99,6 +99,7 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers + bool exchange_type_set = false; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; std::atomic loop_started = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 57ef2405255..11b13714448 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CONNECT_RABBITMQ; + extern const int LOGICAL_ERROR; } static const auto QUEUE_SIZE = 50000; @@ -27,20 +28,20 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address, Context & global_context, const std::pair & login_password_, - const String & routing_key_, - const String & exchange_, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool bind_by_id_, bool use_transactional_channel_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_) : WriteBuffer(nullptr, 0) , login_password(login_password_) - , routing_key(routing_key_) - , exchange_name(exchange_ + "_direct") - , bind_by_id(bind_by_id_) + , routing_keys(routing_keys_) + , exchange_name(exchange_name_) + , exchange_type(exchange_type_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) , payloads(QUEUE_SIZE * num_queues) @@ -73,7 +74,6 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( } producer_channel = std::make_shared(connection.get()); - checkExchange(); /// If publishing should be wrapped in transactions if (use_transactional_channel) @@ -83,6 +83,17 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); + + if (exchange_type == AMQP::ExchangeType::headers) + { + std::vector matching; + for (const auto & header : routing_keys) + { + boost::split(matching, header, [](char c){ return c == '='; }); + key_arguments[matching[0]] = matching[1]; + matching.clear(); + } + } } @@ -90,7 +101,7 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() { stop_loop.store(true); writing_task->deactivate(); - checkExchange(); + initExchange(); connection->close(); assert(rows == 0 && chunks.empty()); @@ -133,28 +144,34 @@ void WriteBufferToRabbitMQProducer::writingFunc() while (!payloads.empty()) { payloads.pop(payload); - next_queue = next_queue % num_queues + 1; - if (bind_by_id) + if (exchange_type == AMQP::ExchangeType::consistent_hash) { + next_queue = next_queue % num_queues + 1; producer_channel->publish(exchange_name, std::to_string(next_queue), payload); } + else if (exchange_type == AMQP::ExchangeType::headers) + { + AMQP::Envelope envelope(payload.data(), payload.size()); + envelope.setHeaders(key_arguments); + producer_channel->publish(exchange_name, "", envelope, key_arguments); + } else { - producer_channel->publish(exchange_name, routing_key, payload); + producer_channel->publish(exchange_name, routing_keys[0], payload); } } + iterateEventLoop(); } } -void WriteBufferToRabbitMQProducer::checkExchange() +void WriteBufferToRabbitMQProducer::initExchange() { std::atomic exchange_declared = false, exchange_error = false; - /// The AMQP::passive flag indicates that it should only be checked if there is a valid exchange with the given name. - producer_channel->declareExchange(exchange_name, AMQP::direct, AMQP::passive) + producer_channel->declareExchange(exchange_name, exchange_type) .onSuccess([&]() { exchange_declared = true; @@ -162,10 +179,10 @@ void WriteBufferToRabbitMQProducer::checkExchange() .onError([&](const char * message) { exchange_error = true; - LOG_ERROR(log, "Exchange for INSERT query was not declared. Reason: {}", message); + LOG_ERROR(log, "Exchange error: {}", message); }); - /// These variables are updated in a separate thread and starting the loop blocks current thread + /// These variables are updated in a separate thread. while (!exchange_declared && !exchange_error) { iterateEventLoop(); @@ -175,9 +192,6 @@ void WriteBufferToRabbitMQProducer::checkExchange() void WriteBufferToRabbitMQProducer::finilizeProducer() { - /// This will make sure everything is published - checkExchange(); - if (use_transactional_channel) { std::atomic answer_received = false, wait_rollback = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 8dc5a32b7d7..20b133b6930 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -23,11 +23,11 @@ public: std::pair & parsed_address, Context & global_context, const std::pair & login_password_, - const String & routing_key_, - const String & exchange_, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool bind_by_id_, bool use_transactional_channel_, std::optional delimiter, size_t rows_per_message, @@ -38,21 +38,22 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } + void finilizeProducer(); private: void nextImpl() override; - void checkExchange(); + void initExchange(); void iterateEventLoop(); void writingFunc(); - void finilizeProducer(); const std::pair login_password; - const String routing_key; + const Names routing_keys; const String exchange_name; - const bool bind_by_id; + AMQP::ExchangeType exchange_type; const size_t num_queues; const bool use_transactional_channel; + AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; std::atomic stop_loop = false; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 42b7101f9c6..a044eba805c 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -485,7 +485,6 @@ def test_rabbitmq_big_message(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.view') - print("Result", result, "Expected", batch_messages * rabbitmq_messages) if int(result) == batch_messages * rabbitmq_messages: break @@ -552,7 +551,6 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.view') time.sleep(1) - print("Result", result, "Expected", messages_num * threads_num) if int(result) == messages_num * threads_num: break @@ -778,6 +776,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'insert', + rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert1', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -788,10 +787,64 @@ def test_rabbitmq_insert(rabbitmq_cluster): consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() - consumer.exchange_declare(exchange='insert_rabbitmq_direct', exchange_type='direct') + consumer.exchange_declare(exchange='insert', exchange_type='direct') result = consumer.queue_declare(queue='') queue_name = result.method.queue - consumer.queue_bind(exchange='insert_rabbitmq_direct', queue=queue_name, routing_key='insert1') + consumer.queue_bind(exchange='insert', queue=queue_name, routing_key='insert1') + + values = [] + for i in range(50): + values.append("({i}, {i})".format(i=i)) + values = ','.join(values) + + while True: + try: + instance.query("INSERT INTO test.rabbitmq VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if 'Local: Timed out.' in str(e): + continue + else: + raise + + insert_messages = [] + def onReceived(channel, method, properties, body): + i = 0 + insert_messages.append(body.decode()) + if (len(insert_messages) == 50): + channel.stop_consuming() + + consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() + consumer_connection.close() + + result = '\n'.join(insert_messages) + rabbitmq_check_result(result, True) + + +@pytest.mark.timeout(240) +def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'insert_headers', + rabbitmq_exchange_type = 'headers', + rabbitmq_routing_key_list = 'test=insert,topic=headers', + rabbitmq_format = 'TSV', + rabbitmq_row_delimiter = '\\n'; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + consumer_connection = pika.BlockingConnection(parameters) + + consumer = consumer_connection.channel() + consumer.exchange_declare(exchange='insert_headers', exchange_type='headers') + result = consumer.queue_declare(queue='') + queue_name = result.method.queue + consumer.queue_bind(exchange='insert_headers', queue=queue_name, routing_key="", + arguments={'x-match':'all', 'test':'insert', 'topic':'headers'}) values = [] for i in range(50): @@ -815,7 +868,6 @@ def test_rabbitmq_insert(rabbitmq_cluster): if (len(insert_messages) == 50): channel.stop_consuming() - consumer.basic_qos(prefetch_count=50) consumer.basic_consume(onReceived, queue_name) consumer.start_consuming() consumer_connection.close() @@ -833,6 +885,8 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): CREATE TABLE test.rabbitmq_many (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'many_inserts', + rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert2', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -887,69 +941,6 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) -@pytest.mark.timeout(240) -def test_rabbitmq_sharding_between_channels_and_queues_insert(rabbitmq_cluster): - instance.query(''' - DROP TABLE IF EXISTS test.view_sharding; - DROP TABLE IF EXISTS test.consumer_sharding; - CREATE TABLE test.rabbitmq_sharding (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, - rabbitmq_num_queues = 2, - rabbitmq_format = 'TSV', - rabbitmq_row_delimiter = '\\n'; - CREATE TABLE test.view_sharding (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; - CREATE MATERIALIZED VIEW test.consumer_sharding TO test.view_sharding AS - SELECT * FROM test.rabbitmq_sharding; - ''') - - messages_num = 10000 - def insert(): - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ','.join(values) - - while True: - try: - instance.query("INSERT INTO test.rabbitmq_sharding VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if 'Local: Timed out.' in str(e): - continue - else: - raise - - threads = [] - threads_num = 20 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = instance.query('SELECT count() FROM test.view_sharding') - time.sleep(1) - if int(result) == messages_num * threads_num: - break - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_sharding; - DROP TABLE IF EXISTS test.consumer_sharding; - DROP TABLE IF EXISTS test.view_sharding; - ''') - - for thread in threads: - thread.join() - - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - @pytest.mark.timeout(420) def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query(''' @@ -958,6 +949,9 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): CREATE TABLE test.rabbitmq_overload (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'over', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'over', rabbitmq_num_consumers = 10, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; From ac448db4918ad03cafed1ec2616af0bbb759fbcd Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Jul 2020 10:05:00 +0000 Subject: [PATCH 010/535] Add virtuals --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 9 +- .../ReadBufferFromRabbitMQConsumer.cpp | 17 +-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 15 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 5 +- .../integration/test_storage_rabbitmq/test.py | 134 ++++++++++++++++++ 5 files changed, 168 insertions(+), 12 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 83e3a02b478..7b1cdd11317 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -21,7 +21,8 @@ RabbitMQBlockInputStream::RabbitMQBlockInputStream( , context(context_) , column_names(columns) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) - , virtual_header(metadata_snapshot->getSampleBlockForColumns({"_exchange"}, storage.getVirtuals(), storage.getStorageID())) + , virtual_header(metadata_snapshot->getSampleBlockForColumns( + {"_exchange_name", "_consumer_tag", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) { } @@ -124,10 +125,16 @@ Block RabbitMQBlockInputStream::readImpl() auto new_rows = read_rabbitmq_message(); auto exchange_name = buffer->getExchange(); + auto consumer_tag = buffer->getConsumerTag(); + auto delivery_tag = buffer->getDeliveryTag(); + auto redelivered = buffer->getRedelivered(); for (size_t i = 0; i < new_rows; ++i) { virtual_columns[0]->insert(exchange_name); + virtual_columns[1]->insert(consumer_tag); + virtual_columns[2]->insert(delivery_tag); + virtual_columns[3]->insert(redelivered); } total_rows = total_rows + new_rows; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index d59e9c9eade..0aff21f8a8e 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -48,7 +48,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) - , messages(QUEUE_SIZE * num_queues) + , received(QUEUE_SIZE * num_queues) { /* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added. * By default there is one queue per consumer. @@ -65,7 +65,7 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { consumer_channel->close(); - messages.clear(); + received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -278,15 +278,16 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) return; consumer_channel->consume(queue_name, AMQP::noack) - .onSuccess([&](const std::string & /* consumer */) + .onSuccess([&](const std::string & consumer) { subscribed_queue[queue_name] = true; consumer_error = false; ++count_subscribed; + consumer_tag = consumer; LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); }) - .onReceived([&](const AMQP::Message & message, uint64_t /* deliveryTag */, bool /* redelivered */) + .onReceived([&](const AMQP::Message & message, uint64_t deliveryTag, bool redelivered) { size_t message_size = message.bodySize(); if (message_size && message.body() != nullptr) @@ -297,7 +298,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) message_received += row_delimiter; } - messages.push(message_received); + received.push({deliveryTag, message_received, redelivered}); } }) .onError([&](const char * message) @@ -346,10 +347,10 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() if (stopped || !allowed) return false; - if (messages.tryPop(current)) + if (received.tryPop(current)) { - auto * new_position = const_cast(current.data()); - BufferBase::set(new_position, current.size(), 0); + auto * new_position = const_cast(current.message.data()); + BufferBase::set(new_position, current.message.size(), 0); allowed = false; return true; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 82dc3f55248..85644562d0c 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -38,10 +38,20 @@ public: ~ReadBufferFromRabbitMQConsumer() override; + struct MessageData + { + UInt64 delivery_tag; + String message; + bool redelivered; + }; + void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); auto getExchange() const { return exchange_name; } + auto getConsumerTag() const { return consumer_tag; } + auto getDeliveryTag() const { return current.delivery_tag; } + auto getRedelivered() const { return current.redelivered; } private: ChannelPtr consumer_channel; @@ -69,8 +79,9 @@ private: std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; - ConcurrentBoundedQueue messages; - String current; + String consumer_tag; + ConcurrentBoundedQueue received; + MessageData current; std::vector queues; std::unordered_map subscribed_queue; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0d6cf95f39c..66af7dc3f56 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -597,7 +597,10 @@ void registerStorageRabbitMQ(StorageFactory & factory) NamesAndTypesList StorageRabbitMQ::getVirtuals() const { return NamesAndTypesList{ - {"_exchange", std::make_shared()} + {"_exchange_name", std::make_shared()}, + {"_consumer_tag", std::make_shared()}, + {"_delivery_tag", std::make_shared()}, + {"_redelivered", std::make_shared()} }; } diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index a044eba805c..eaaa8613b5f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1520,6 +1520,140 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): assert int(result) == messages_num * num_tables_to_receive, 'ClickHouse lost some messages: {}'.format(result) +@pytest.mark.timeout(420) +def test_rabbitmq_virtual_columns(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view; + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'virtuals', + rabbitmq_format = 'JSONEachRow'; + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + channel.exchange_declare(exchange='virtuals', exchange_type='fanout') + + message_num = 10 + i = [0] + messages = [] + for _ in range(message_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + + for message in messages: + channel.basic_publish(exchange='virtuals', routing_key='', body=message) + + while True: + result = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result) == message_num: + break + + connection.close() + + result = instance.query("SELECT count(DISTINCT _delivery_tag) FROM test.view") + assert int(result) == 10 + + result = instance.query("SELECT count(DISTINCT _consumer_tag) FROM test.view") + assert int(result) == 1 + + result = instance.query(''' + SELECT key, value, _exchange_name, SUBSTRING(_consumer_tag, 1, 8), _delivery_tag, _redelivered + FROM test.view + ORDER BY key + ''') + + expected = '''\ +0 0 virtuals amq.ctag 1 0 +1 1 virtuals amq.ctag 2 0 +2 2 virtuals amq.ctag 3 0 +3 3 virtuals amq.ctag 4 0 +4 4 virtuals amq.ctag 5 0 +5 5 virtuals amq.ctag 6 0 +6 6 virtuals amq.ctag 7 0 +7 7 virtuals amq.ctag 8 0 +8 8 virtuals amq.ctag 9 0 +9 9 virtuals amq.ctag 10 0 +''' + assert TSV(result) == TSV(expected) + + +@pytest.mark.timeout(420) +def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'virtuals_mv', + rabbitmq_format = 'JSONEachRow'; + CREATE TABLE test.view (key UInt64, value UInt64, + exchange_name String, consumer_tag String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _exchange_name as exchange_name, _consumer_tag as consumer_tag, _delivery_tag as delivery_tag, _redelivered as redelivered + FROM test.rabbitmq; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + channel.exchange_declare(exchange='virtuals_mv', exchange_type='fanout') + + message_num = 10 + i = [0] + messages = [] + for _ in range(message_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + + for message in messages: + channel.basic_publish(exchange='virtuals_mv', routing_key='', body=message) + + while True: + result = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result) == message_num: + break + + connection.close() + + result = instance.query("SELECT count(DISTINCT delivery_tag) FROM test.view") + assert int(result) == 10 + + result = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + assert int(result) == 1 + + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view") + expected = '''\ +0 0 virtuals_mv amq.ctag 1 0 +1 1 virtuals_mv amq.ctag 2 0 +2 2 virtuals_mv amq.ctag 3 0 +3 3 virtuals_mv amq.ctag 4 0 +4 4 virtuals_mv amq.ctag 5 0 +5 5 virtuals_mv amq.ctag 6 0 +6 6 virtuals_mv amq.ctag 7 0 +7 7 virtuals_mv amq.ctag 8 0 +8 8 virtuals_mv amq.ctag 9 0 +9 9 virtuals_mv amq.ctag 10 0 +''' + + instance.query(''' + DROP TABLE test.consumer; + DROP TABLE test.view; + ''') + + assert TSV(result) == TSV(expected) + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From f0f6111655852d1c5cc8c50db7e0efcdaed41192 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Jul 2020 15:47:39 +0000 Subject: [PATCH 011/535] Move exchange init, add bridge-exchange --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 2 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 2 + src/Storages/RabbitMQ/RabbitMQHandler.cpp | 2 + src/Storages/RabbitMQ/RabbitMQHandler.h | 3 +- .../ReadBufferFromRabbitMQConsumer.cpp | 125 ++------------ .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 13 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 152 +++++++++++++++++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 13 +- .../WriteBufferToRabbitMQProducer.cpp | 5 +- .../integration/test_storage_rabbitmq/test.py | 129 ++++++++------- 10 files changed, 253 insertions(+), 193 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 7b1cdd11317..6e8e153392c 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -124,7 +124,7 @@ Block RabbitMQBlockInputStream::readImpl() auto new_rows = read_rabbitmq_message(); - auto exchange_name = buffer->getExchange(); + auto exchange_name = storage.getExchange(); auto consumer_tag = buffer->getConsumerTag(); auto delivery_tag = buffer->getDeliveryTag(); auto redelivered = buffer->getRedelivered(); diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 2559b31c44a..87a17d3e1ed 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -33,6 +33,8 @@ Block RabbitMQBlockOutputStream::getHeader() const void RabbitMQBlockOutputStream::writePrefix() { + if (storage.checkBridge()) + storage.unbindExchange(); buffer = storage.createWriteBuffer(); if (!buffer) throw Exception("Failed to create RabbitMQ producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index 5d17ff23b64..f01b1e60eab 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -31,9 +31,11 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); + loop_started.store(true); /// stop_loop variable is updated in a separate thread while (!stop_loop.load()) uv_run(loop, UV_RUN_NOWAIT); + loop_started.store(false); } void RabbitMQHandler::iterateLoop() diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 5893ace1d2f..b1b84e1d07a 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -21,12 +21,13 @@ public: void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); + bool checkLoop() const { return loop_started.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false; + std::atomic stop_loop = false, loop_started = false; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 0aff21f8a8e..8c272e04691 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,15 +14,11 @@ namespace DB { -namespace ExchangeType -{ - static const String HASH_SUF = "_hash"; -} - static const auto QUEUE_SIZE = 50000; /// Equals capacity of a single rabbitmq queue ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, + ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, const AMQP::ExchangeType & exchange_type_, @@ -36,6 +32,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer_channel(std::move(consumer_channel_)) + , setup_channel(setup_channel_) , event_handler(event_handler_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) @@ -43,21 +40,14 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , channel_id(channel_id_) , hash_exchange(hash_exchange_) , num_queues(num_queues_) - , local_exchange(local_exchange_) - , local_hash_exchange(local_exchange + ExchangeType::HASH_SUF) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) + , local_exchange(local_exchange_) , received(QUEUE_SIZE * num_queues) { - /* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added. - * By default there is one queue per consumer. - */ for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) - { - /// Queue bingings must be declared before any publishing => it must be done here and not in readPrefix() initQueueBindings(queue_id); - } } @@ -70,125 +60,34 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() } -void ReadBufferFromRabbitMQConsumer::initExchange() -{ - /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which - * will evenly distribute messages between all consumers. - */ - consumer_channel->declareExchange(exchange_name, exchange_type).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare client's {} exchange. Reason: {}", exchange_type, message); - }); - - /// No need for declaring hash-exchange if there is only one consumer with one queue or exchange type is already hash - if (!hash_exchange || exchange_type == AMQP::ExchangeType::consistent_hash) - return; - - { - /* By default hash exchange distributes messages based on a hash value of a routing key, which must be a string integer. But - * in current case we use hash exchange for binding to another exchange of some other type, which needs its own routing keys - * of other types: headers, patterns and string-keys. This means that hash property must be changed. - */ - AMQP::Table binding_arguments; - binding_arguments["hash-property"] = "message_id"; - - /// Declare exchange for sharding. - consumer_channel->declareExchange(local_hash_exchange, AMQP::consistent_hash, binding_arguments) - .onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare {} exchange: {}", exchange_type, message); - }); - } - - /// Then bind client's exchange to sharding exchange (by keys, specified by the client): - - if (exchange_type == AMQP::ExchangeType::headers) - { - AMQP::Table binding_arguments; - std::vector matching; - - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - binding_arguments[matching[0]] = matching[1]; - matching.clear(); - } - - /// Routing key can be arbitrary here. - consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_keys[0], binding_arguments) - .onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); - }); - } - else if (exchange_type == AMQP::ExchangeType::fanout) - { - consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_keys[0]).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); - }); - } - else - { - for (const auto & routing_key : routing_keys) - { - consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_key).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); - }); - } - } -} - - void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { - /// These variables might be updated later from a separate thread in onError callbacks. - if (!local_exchange_declared || (hash_exchange && !local_hash_exchange_declared)) - { - initExchange(); - local_exchange_declared = true; - local_hash_exchange_declared = true; - } - bool bindings_created = false, bindings_error = false; - consumer_channel->declareQueue(AMQP::exclusive) + setup_channel->declareQueue(AMQP::exclusive) .onSuccess([&](const std::string & queue_name_, int /* msgcount */, int /* consumercount */) { queues.emplace_back(queue_name_); LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); subscribed_queue[queue_name_] = false; - /* Subscription can probably be moved back to readPrefix(), but not sure whether it is better in regard to speed, because - * if moved there, it must(!) be wrapped inside a channel->onSuccess callback or any other, otherwise - * consumer might fail to subscribe and no resubscription will help. - */ subscribe(queues.back()); if (hash_exchange) { String binding_key; if (queues.size() == 1) - { binding_key = std::to_string(channel_id); - } else - { binding_key = std::to_string(channel_id + queue_id); - } + /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. */ - String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_hash_exchange; + String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_exchange; /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. - consumer_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) + setup_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) .onSuccess([&] { bindings_created = true; @@ -201,7 +100,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) } else if (exchange_type == AMQP::ExchangeType::fanout) { - consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) + setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) .onSuccess([&] { bindings_created = true; @@ -225,7 +124,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) matching.clear(); } - consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) + setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) .onSuccess([&] { bindings_created = true; @@ -242,7 +141,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) for (const auto & routing_key : routing_keys) { /// Binding directly to exchange, specified by the client. - consumer_channel->bindQueue(exchange_name, queue_name_, routing_key) + setup_channel->bindQueue(exchange_name, queue_name_, routing_key) .onSuccess([&] { bindings_created = true; @@ -261,10 +160,6 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); - /* Run event loop (which updates local variables in a separate thread) until bindings are created or failed to be created. - * It is important at this moment to make sure that queue bindings are created before any publishing can happen because - * otherwise messages will be routed nowhere. - */ while (!bindings_created && !bindings_error) { iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 85644562d0c..6896dd7f4b0 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -24,6 +24,7 @@ class ReadBufferFromRabbitMQConsumer : public ReadBuffer public: ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, + ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, const AMQP::ExchangeType & exchange_type_, @@ -48,13 +49,13 @@ public: void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); - auto getExchange() const { return exchange_name; } auto getConsumerTag() const { return consumer_tag; } auto getDeliveryTag() const { return current.delivery_tag; } auto getRedelivered() const { return current.redelivered; } private: ChannelPtr consumer_channel; + ChannelPtr setup_channel; HandlerPtr event_handler; const String exchange_name; @@ -64,18 +65,12 @@ private: const bool hash_exchange; const size_t num_queues; - const String local_exchange; - const String local_default_exchange; - const String local_hash_exchange; - Poco::Logger * log; char row_delimiter; bool allowed = true; const std::atomic & stopped; - String default_local_exchange; - bool local_exchange_declared = false, local_hash_exchange_declared = false; - + const String local_exchange; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; @@ -87,7 +82,7 @@ private: bool nextImpl() override; - void initExchange(); + void connectAlternateExchange(); void initQueueBindings(const size_t queue_id); void subscribe(const String & queue_name); void iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 66af7dc3f56..d56a46c4f55 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -118,8 +118,7 @@ StorageRabbitMQ::StorageRabbitMQ( hash_exchange = num_consumers > 1 || num_queues > 1; - exchange_type_set = exchange_type_ != ExchangeType::DEFAULT; - if (exchange_type_set) + if (exchange_type_ != ExchangeType::DEFAULT) { if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct; @@ -133,11 +132,23 @@ StorageRabbitMQ::StorageRabbitMQ( exchange_type = AMQP::ExchangeType::fanout; } + if (exchange_type == AMQP::ExchangeType::headers) + { + std::vector matching; + for (const auto & header : routing_keys) + { + boost::split(matching, header, [](char c){ return c == '='; }); + bind_headers[matching[0]] = matching[1]; + matching.clear(); + } + } + auto table_id = getStorageID(); String table_name = table_id.table_name; /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name - local_exchange_name = exchange_name + "_" + table_name; + local_exchange = exchange_name + "_" + table_name; + bridge_exchange = local_exchange + "_bridge"; /// One looping task for all consumers as they share the same connection == the same handler == the same event loop looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); @@ -163,6 +174,133 @@ void StorageRabbitMQ::loopingFunc() } +void StorageRabbitMQ::initExchange() +{ + /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which + * will evenly distribute messages between all consumers. + */ + setup_channel->declareExchange(exchange_name, exchange_type, AMQP::durable) + .onError([&](const char * message) + { + throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " + + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + /// Bridge exchange is needed to easily disconnect consumer queues. + setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) + .onError([&](const char * message) + { + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + if (!hash_exchange) + { + consumer_exchange = bridge_exchange; + return; + } + + /// Declare exchange for sharding. + AMQP::Table binding_arguments; + binding_arguments["hash-property"] = "message_id"; + + setup_channel->declareExchange(local_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) + .onError([&](const char * message) + { + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + setup_channel->bindExchange(bridge_exchange, local_exchange, routing_keys[0]) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + consumer_exchange = local_exchange; +} + + +void StorageRabbitMQ::bindExchange() +{ + std::atomic binding_created = false; + + /// Bridge exchange connects client's exchange with consumers' queues. + if (exchange_type == AMQP::ExchangeType::headers) + { + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) + .onSuccess([&]() + { + binding_created = true; + }) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + } + else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) + { + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0]) + .onSuccess([&]() + { + binding_created = true; + }) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + } + else + { + for (const auto & routing_key : routing_keys) + { + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_key) + .onSuccess([&]() + { + binding_created = true; + }) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + } + } + + while (!binding_created) + { + event_handler->iterateLoop(); + } +} + + +void StorageRabbitMQ::unbindExchange() +{ + if (bridge.try_lock()) + { + if (exchange_removed.load()) + return; + + setup_channel->removeExchange(bridge_exchange) + .onSuccess([&]() + { + exchange_removed.store(true); + }) + .onError([&](const char * message) + { + throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + while (!exchange_removed) + { + event_handler->iterateLoop(); + } + + event_handler->stop(); + looping_task->deactivate(); + + bridge.unlock(); + } +} + + Pipes StorageRabbitMQ::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -207,6 +345,10 @@ BlockOutputStreamPtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadat void StorageRabbitMQ::startup() { + setup_channel = std::make_shared(connection.get()); + initExchange(); + bindExchange(); + for (size_t i = 0; i < num_consumers; ++i) { try @@ -288,9 +430,9 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, event_handler, exchange_name, exchange_type, routing_keys, + consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, next_channel_id, log, row_delimiter, hash_exchange, num_queues, - local_exchange_name, stream_cancelled); + local_exchange, stream_cancelled); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 4457c5ff8c9..07b24e8ca1d 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -54,6 +54,9 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; + const String getExchange() const { return exchange_name; } + bool checkBridge() const { return !exchange_removed.load(); } + void unbindExchange(); protected: StorageRabbitMQ( @@ -77,7 +80,6 @@ private: Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - String local_exchange_name; const String format_name; char row_delimiter; @@ -99,10 +101,13 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers - bool exchange_type_set = false; + String local_exchange, bridge_exchange, consumer_exchange; + std::mutex bridge; + AMQP::Table bind_headers; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; - std::atomic loop_started = false; + std::atomic loop_started = false, exchange_removed = false; + ChannelPtr setup_channel; BackgroundSchedulePool::TaskHolder streaming_task; BackgroundSchedulePool::TaskHolder heartbeat_task; @@ -115,6 +120,8 @@ private: void threadFunc(); void heartbeatFunc(); void loopingFunc(); + void initExchange(); + void bindExchange(); void pingConnection() { connection->heartbeat(); } bool streamToViews(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 11b13714448..27e4a7b8a03 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -171,15 +171,14 @@ void WriteBufferToRabbitMQProducer::initExchange() { std::atomic exchange_declared = false, exchange_error = false; - producer_channel->declareExchange(exchange_name, exchange_type) + producer_channel->declareExchange(exchange_name, exchange_type, AMQP::durable + AMQP::passive) .onSuccess([&]() { exchange_declared = true; }) - .onError([&](const char * message) + .onError([&](const char * /* message */) { exchange_error = true; - LOG_ERROR(log, "Exchange error: {}", message); }); /// These variables are updated in a separate thread. diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index eaaa8613b5f..104ffa4e5cb 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -121,7 +121,7 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'new', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'new', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') @@ -130,20 +130,19 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(25): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='new', body=message) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='new', body=message) connection.close() @@ -160,21 +159,20 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): def test_rabbitmq_select_from_old_syntax_table(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ('rabbitmq1:5672', 'old', 'clickhouse-exchange', 'JSONEachRow', '\\n'); + ENGINE = RabbitMQ('rabbitmq1:5672', 'old', 'old', 'JSONEachRow', '\\n'); ''') credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='old', body=message) + channel.basic_publish(exchange='old', routing_key='old', body=message) connection.close() @@ -208,7 +206,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'json', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'delim1', rabbitmq_format = 'JSONEachRow' ''') @@ -216,7 +214,6 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = '' for i in range(25): @@ -224,14 +221,14 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='json', body=message) + channel.basic_publish(exchange='delim1', routing_key='json', body=message) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='json', body=message) + channel.basic_publish(exchange='delim1', routing_key='json', body=message) result = '' while True: @@ -250,7 +247,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'csv', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'delim2', rabbitmq_format = 'CSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -259,14 +256,13 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='csv', body=message) + channel.basic_publish(exchange='delim2', routing_key='csv', body=message) result = '' while True: @@ -286,7 +282,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'tsv', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'delim3', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -295,14 +291,13 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(50): messages.append('{i}\t{i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='tsv', body=message) + channel.basic_publish(exchange='delim3', routing_key='tsv', body=message) result = '' while True: @@ -322,6 +317,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'mv', rabbitmq_routing_key_list = 'mv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -341,7 +337,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='mv', body=message) + channel.basic_publish(exchange='mv', routing_key='mv', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -365,6 +361,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'mvsq', rabbitmq_routing_key_list = 'mvsq', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -384,7 +381,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='mvsq', body=message) + channel.basic_publish(exchange='mvsq', routing_key='mvsq', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -410,6 +407,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'mmv', rabbitmq_routing_key_list = 'mmv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -434,7 +432,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='mmv', body=message) + channel.basic_publish(exchange='mmv', routing_key='mmv', body=message) while True: result1 = instance.query('SELECT * FROM test.view1') @@ -471,6 +469,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value String) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'big', rabbitmq_routing_key_list = 'big', rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value String) @@ -481,7 +480,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): ''') for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='big', body=message) + channel.basic_publish(exchange='big', routing_key='big', body=message) while True: result = instance.query('SELECT count() FROM test.view') @@ -506,6 +505,7 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'test_channels_sharding', rabbitmq_num_consumers = 5, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -528,7 +528,6 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -536,7 +535,7 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): i[0] += 1 key = str(randrange(1, NUM_CHANNELS)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='test_channels_sharding', routing_key=key, body=message) connection.close() threads = [] @@ -569,6 +568,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'test_queues_sharding', rabbitmq_num_queues = 4, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -591,7 +591,6 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -599,7 +598,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): i[0] += 1 key = str(randrange(1, NUM_QUEUES)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='test_queues_sharding', routing_key=key, body=message) connection.close() threads = [] @@ -633,6 +632,7 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'test_sharding', rabbitmq_num_queues = 2, rabbitmq_num_consumers = 10, rabbitmq_format = 'JSONEachRow', @@ -657,7 +657,6 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -665,7 +664,7 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) i[0] += 1 key = str(randrange(1, NUM_QUEUES * NUM_CONSUMERS)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='test_sharding', routing_key=key, body=message) connection.close() threads = [] @@ -699,6 +698,7 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'combo', rabbitmq_num_consumers = 4, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -728,7 +728,6 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -736,7 +735,7 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): i[0] += 1 key = str(randrange(1, NUM_CONSUMERS)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='combo', routing_key=key, body=message) connection.close() threads = [] @@ -787,7 +786,6 @@ def test_rabbitmq_insert(rabbitmq_cluster): consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() - consumer.exchange_declare(exchange='insert', exchange_type='direct') result = consumer.queue_declare(queue='') queue_name = result.method.queue consumer.queue_bind(exchange='insert', queue=queue_name, routing_key='insert1') @@ -840,7 +838,6 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() - consumer.exchange_declare(exchange='insert_headers', exchange_type='headers') result = consumer.queue_declare(queue='') queue_name = result.method.queue consumer.queue_bind(exchange='insert_headers', queue=queue_name, routing_key="", @@ -890,12 +887,20 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): rabbitmq_routing_key_list = 'insert2', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; + CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'many_inserts', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'insert2', + rabbitmq_format = 'TSV', + rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view_many (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS - SELECT * FROM test.rabbitmq_many; + SELECT * FROM test.rabbitmq_consume; ''') messages_num = 1000 @@ -933,6 +938,7 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): DROP TABLE IF EXISTS test.rabbitmq_many; DROP TABLE IF EXISTS test.consumer_many; DROP TABLE IF EXISTS test.view_many; + DROP TABLE IF EXISTS test.view_consume; ''') for thread in threads: @@ -946,13 +952,21 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view_overload; DROP TABLE IF EXISTS test.consumer_overload; + CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'over', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'over', + rabbitmq_num_consumers = 6, + rabbitmq_format = 'TSV', + rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.rabbitmq_overload (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'over', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'over', - rabbitmq_num_consumers = 10, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view_overload (key UInt64, value UInt64) @@ -960,7 +974,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer_overload TO test.view_overload AS - SELECT * FROM test.rabbitmq_overload; + SELECT * FROM test.rabbitmq_consume; ''') messages_num = 100000 @@ -999,6 +1013,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): DROP TABLE IF EXISTS test.rabbitmq_overload; DROP TABLE IF EXISTS test.consumer_overload; DROP TABLE IF EXISTS test.view_overload; + DROP TABLE IF EXISTS test.view_consume; ''') for thread in threads: @@ -1044,7 +1059,6 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='direct_exchange_testing', exchange_type='direct') messages = [] for _ in range(messages_num): @@ -1118,7 +1132,6 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='fanout_exchange_testing', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -1207,7 +1220,6 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='topic_exchange_testing', exchange_type='topic') messages = [] for _ in range(messages_num): @@ -1253,8 +1265,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): def test_rabbitmq_hash_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) ENGINE = MergeTree() ORDER BY key; ''') @@ -1275,7 +1286,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.{0}; + SELECT key, value, _consumer_tag AS consumer_tag FROM test.{0}; '''.format(table_name)) i = [0] @@ -1288,14 +1299,16 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): # init connection here because otherwise python rabbitmq client might fail connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='hash_exchange_testing', exchange_type='x-consistent-hash') messages = [] for _ in range(messages_num): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 + current = 0 for message in messages: - key = str(randrange(10)) - channel.basic_publish(exchange='hash_exchange_testing', routing_key=key, body=message) + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='hash_exchange_testing', routing_key=mes_id, + properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -1307,11 +1320,13 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): time.sleep(random.uniform(0, 1)) thread.start() + result1 = '' while True: - result = instance.query('SELECT count() FROM test.destination') + result1 = instance.query('SELECT count() FROM test.destination') time.sleep(1) - if int(result) == messages_num * threads_num: + if int(result1) == messages_num * threads_num: break + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") for consumer_id in range(num_tables): table_name = 'rabbitmq_consumer{}'.format(consumer_id) @@ -1327,7 +1342,9 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): for thread in threads: thread.join() - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result2) >= 30 + @pytest.mark.timeout(420) @@ -1383,7 +1400,6 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): # init connection here because otherwise python rabbitmq client might fail connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='multiple_bindings_testing', exchange_type='direct') messages = [] for _ in range(messages_num): @@ -1481,7 +1497,6 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='headers_exchange_testing', exchange_type='headers') messages = [] for _ in range(messages_num): @@ -1524,20 +1539,19 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): def test_rabbitmq_virtual_columns(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view; - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + CREATE TABLE test.rabbitmq_virtuals (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals', rabbitmq_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq; + SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; ''') credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='virtuals', exchange_type='fanout') message_num = 10 i = [0] @@ -1581,6 +1595,9 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): 8 8 virtuals amq.ctag 9 0 9 9 virtuals amq.ctag 10 0 ''' + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + ''') assert TSV(result) == TSV(expected) @@ -1589,7 +1606,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + CREATE TABLE test.rabbitmq_virtuals_mv (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals_mv', @@ -1599,14 +1616,13 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT *, _exchange_name as exchange_name, _consumer_tag as consumer_tag, _delivery_tag as delivery_tag, _redelivered as redelivered - FROM test.rabbitmq; + FROM test.rabbitmq_virtuals_mv; ''') credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='virtuals_mv', exchange_type='fanout') message_num = 10 i = [0] @@ -1647,8 +1663,9 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ''' instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv ''') assert TSV(result) == TSV(expected) From f9a4bf9e6156820646cd008863af0f8a4f193c5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Jul 2020 11:45:01 +0000 Subject: [PATCH 012/535] Add queue resume read --- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 1 + src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 26 +++-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 3 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 22 ++++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 +- .../integration/test_storage_rabbitmq/test.py | 94 ++++++++++++++++++- 7 files changed, 137 insertions(+), 14 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 87a17d3e1ed..c2eae19cb86 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -35,6 +35,7 @@ void RabbitMQBlockOutputStream::writePrefix() { if (storage.checkBridge()) storage.unbindExchange(); + buffer = storage.createWriteBuffer(); if (!buffer) throw Exception("Failed to create RabbitMQ producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 5cd52ed9ef7..488fc59e562 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -19,6 +19,7 @@ namespace DB M(SettingUInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(SettingUInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ + M(SettingString, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 8c272e04691..5abdb4fe7c2 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -24,6 +24,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, + const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, @@ -38,6 +39,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , exchange_type(exchange_type_) , routing_keys(routing_keys_) , channel_id(channel_id_) + , queue_base(queue_base_) , hash_exchange(hash_exchange_) , num_queues(num_queues_) , log(log_) @@ -54,7 +56,6 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { consumer_channel->close(); - received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -64,12 +65,14 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { bool bindings_created = false, bindings_error = false; - setup_channel->declareQueue(AMQP::exclusive) - .onSuccess([&](const std::string & queue_name_, int /* msgcount */, int /* consumercount */) + auto success_callback = [&](const std::string & queue_name_, int msgcount, int /* consumercount */) { queues.emplace_back(queue_name_); LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); + if (msgcount) + LOG_TRACE(log, "Queue " + queue_name_ + " is non-empty. Non-consumed messaged will also be delivered."); + subscribed_queue[queue_name_] = false; subscribe(queues.back()); @@ -86,7 +89,6 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) */ String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_exchange; - /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. setup_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) .onSuccess([&] { @@ -116,7 +118,6 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) AMQP::Table binding_arguments; std::vector matching; - /// It is not parsed for the second time - if it was parsed above, then it would never end up here. for (const auto & header : routing_keys) { boost::split(matching, header, [](char c){ return c == '='; }); @@ -153,13 +154,24 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) }); } } - }) - .onError([&](const char * message) + }; + + auto error_callback([&](const char * message) { bindings_error = true; LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); + if (!queue_base.empty()) + { + const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); + setup_channel->declareQueue(queue_name, AMQP::durable).onSuccess(success_callback).onError(error_callback); + } + else + { + setup_channel->declareQueue(AMQP::durable).onSuccess(success_callback).onError(error_callback); + } + while (!bindings_created && !bindings_error) { iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 6896dd7f4b0..f4978e54229 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -30,6 +30,7 @@ public: const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, + const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, @@ -62,6 +63,7 @@ private: const AMQP::ExchangeType exchange_type; const Names routing_keys; const size_t channel_id; + const String queue_base; const bool hash_exchange; const size_t num_queues; @@ -82,7 +84,6 @@ private: bool nextImpl() override; - void connectAlternateExchange(); void initQueueBindings(const size_t queue_id); void subscribe(const String & queue_name); void iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index d56a46c4f55..db4f1c7b338 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -72,7 +72,8 @@ StorageRabbitMQ::StorageRabbitMQ( const String & exchange_type_, size_t num_consumers_, size_t num_queues_, - const bool use_transactional_channel_) + const bool use_transactional_channel_, + const String & queue_base_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) , rabbitmq_context(Context(global_context)) @@ -83,6 +84,7 @@ StorageRabbitMQ::StorageRabbitMQ( , num_consumers(num_consumers_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) + , queue_base(queue_base_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) , login_password(std::make_pair( @@ -288,7 +290,7 @@ void StorageRabbitMQ::unbindExchange() throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - while (!exchange_removed) + while (!exchange_removed.load()) { event_handler->iterateLoop(); } @@ -431,7 +433,7 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() return std::make_shared( consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, - next_channel_id, log, row_delimiter, hash_exchange, num_queues, + next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, local_exchange, stream_cancelled); } @@ -725,10 +727,22 @@ void registerStorageRabbitMQ(StorageFactory & factory) } } + String queue_base = rabbitmq_settings.rabbitmq_queue_base.value; + if (args_count >= 10) + { + engine_args[9] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[9], args.local_context); + + const auto * ast = engine_args[9]->as(); + if (ast && ast->value.getType() == Field::Types::String) + { + queue_base = safeGet(ast->value); + } + } + return StorageRabbitMQ::create( args.table_id, args.context, args.columns, host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, - num_queues, use_transactional_channel); + num_queues, use_transactional_channel, queue_base); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 07b24e8ca1d..d43f2ba27f1 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -71,7 +71,8 @@ protected: const String & exchange_type_, size_t num_consumers_, size_t num_queues_, - const bool use_transactional_channel_); + const bool use_transactional_channel_, + const String & queue_base_); private: Context global_context; @@ -88,6 +89,7 @@ private: bool hash_exchange; size_t num_queues; const bool use_transactional_channel; + const String queue_base; Poco::Logger * log; std::pair parsed_address; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 104ffa4e5cb..655dee7a816 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -877,6 +877,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): def test_rabbitmq_many_inserts(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_many; + DROP TABLE IF EXISTS test.rabbitmq_consume; DROP TABLE IF EXISTS test.view_many; DROP TABLE IF EXISTS test.consumer_many; CREATE TABLE test.rabbitmq_many (key UInt64, value UInt64) @@ -935,10 +936,10 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): break instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_consume; DROP TABLE IF EXISTS test.rabbitmq_many; DROP TABLE IF EXISTS test.consumer_many; DROP TABLE IF EXISTS test.view_many; - DROP TABLE IF EXISTS test.view_consume; ''') for thread in threads: @@ -952,6 +953,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view_overload; DROP TABLE IF EXISTS test.consumer_overload; + DROP TABLE IF EXISTS test.rabbitmq_consume; CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -1671,6 +1673,96 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): assert TSV(result) == TSV(expected) +@pytest.mark.timeout(420) +def test_rabbitmq_queue_resume(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_queue_resume; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='queue_resume', routing_key='queue_resume', body=message, + properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 10 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + ''') + + for thread in threads: + thread.join() + + collected = int(instance.query('SELECT count() FROM test.view')) + + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + while True: + result1 = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result1) > collected: + break + + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + ''') + + assert int(result1) > collected, 'ClickHouse lost some messages: {}'.format(result) + assert int(result2) == 2 + + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 2b57857afc7e220f7844f98ef26fb5aff24ed6c4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Jul 2020 12:33:07 +0000 Subject: [PATCH 013/535] Add dl-exchange, commits --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 11 + .../RabbitMQ/RabbitMQBlockInputStream.h | 2 + src/Storages/RabbitMQ/RabbitMQHandler.cpp | 2 - src/Storages/RabbitMQ/RabbitMQHandler.h | 3 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 50 ++- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 8 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 25 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 +- .../integration/test_storage_rabbitmq/test.py | 353 ++++++++++-------- 10 files changed, 296 insertions(+), 163 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 6e8e153392c..630581b13dc 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -137,6 +137,8 @@ Block RabbitMQBlockInputStream::readImpl() virtual_columns[3]->insert(redelivered); } + last_inserted_delivery_tag = delivery_tag; + total_rows = total_rows + new_rows; buffer->allowNext(); @@ -158,4 +160,13 @@ Block RabbitMQBlockInputStream::readImpl() return result_block; } + +void RabbitMQBlockInputStream::readSuffixImpl() +{ + if (!buffer) + return; + + buffer->ackMessages(last_inserted_delivery_tag); +} + } diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h index f4ab76f72cf..09cda6ff94f 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h @@ -26,6 +26,7 @@ public: void readPrefixImpl() override; Block readImpl() override; + void readSuffixImpl() override; private: StorageRabbitMQ & storage; @@ -38,6 +39,7 @@ private: const Block virtual_header; ConsumerBufferPtr buffer; + UInt64 last_inserted_delivery_tag; }; } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index f01b1e60eab..5d17ff23b64 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -31,11 +31,9 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); - loop_started.store(true); /// stop_loop variable is updated in a separate thread while (!stop_loop.load()) uv_run(loop, UV_RUN_NOWAIT); - loop_started.store(false); } void RabbitMQHandler::iterateLoop() diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index b1b84e1d07a..5893ace1d2f 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -21,13 +21,12 @@ public: void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); - bool checkLoop() const { return loop_started.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false, loop_started = false; + std::atomic stop_loop = false; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 488fc59e562..cd7e7de9622 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -20,6 +20,7 @@ namespace DB M(SettingUInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ M(SettingString, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ + M(SettingString, rabbitmq_deadletter_exchange, "", "Exchange name to be passed as a dead-letter-exchange name.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 5abdb4fe7c2..705aae7ec61 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -30,6 +30,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( bool hash_exchange_, size_t num_queues_, const String & local_exchange_, + const String & deadletter_exchange_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer_channel(std::move(consumer_channel_)) @@ -46,6 +47,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , row_delimiter(row_delimiter_) , stopped(stopped_) , local_exchange(local_exchange_) + , deadletter_exchange(deadletter_exchange_) , received(QUEUE_SIZE * num_queues) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) @@ -55,6 +57,12 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { + if (ack.load() && consumer_channel) + { + consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. + LOG_TRACE(log, "Acknowledged messages with deliveryTags up to {}", prev_tag); + } + consumer_channel->close(); received.clear(); BufferBase::set(nullptr, 0, 0); @@ -162,14 +170,20 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); + AMQP::Table queue_settings; + if (!deadletter_exchange.empty()) + { + queue_settings["x-dead-letter-exchange"] = deadletter_exchange; + } + if (!queue_base.empty()) { const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); - setup_channel->declareQueue(queue_name, AMQP::durable).onSuccess(success_callback).onError(error_callback); + setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); } else { - setup_channel->declareQueue(AMQP::durable).onSuccess(success_callback).onError(error_callback); + setup_channel->declareQueue(AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); } while (!bindings_created && !bindings_error) @@ -184,15 +198,20 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) if (subscribed_queue[queue_name]) return; - consumer_channel->consume(queue_name, AMQP::noack) + consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { subscribed_queue[queue_name] = true; - consumer_error = false; ++count_subscribed; + LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + + consumer_error = false; consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + }); }) .onReceived([&](const AMQP::Message & message, uint64_t deliveryTag, bool redelivered) { @@ -201,11 +220,16 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) { String message_received = std::string(message.body(), message.body() + message_size); if (row_delimiter != '\0') - { message_received += row_delimiter; - } received.push({deliveryTag, message_received, redelivered}); + + std::lock_guard lock(wait_ack); + if (ack.exchange(false) && prev_tag < max_tag && consumer_channel) + { + consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. + LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); + } } }) .onError([&](const char * message) @@ -243,6 +267,17 @@ void ReadBufferFromRabbitMQConsumer::checkSubscription() } +void ReadBufferFromRabbitMQConsumer::ackMessages(UInt64 last_inserted_delivery_tag) +{ + if (last_inserted_delivery_tag > prev_tag) + { + std::lock_guard lock(wait_ack); + prev_tag = last_inserted_delivery_tag; + ack.store(true); + } +} + + void ReadBufferFromRabbitMQConsumer::iterateEventLoop() { event_handler->iterateLoop(); @@ -259,6 +294,7 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() auto * new_position = const_cast(current.message.data()); BufferBase::set(new_position, current.message.size(), 0); allowed = false; + max_tag = current.delivery_tag; return true; } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index f4978e54229..8033f537e8c 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -36,6 +36,7 @@ public: bool hash_exchange_, size_t num_queues_, const String & local_exchange_, + const String & deadletter_exchange_, const std::atomic & stopped_); ~ReadBufferFromRabbitMQConsumer() override; @@ -49,6 +50,7 @@ public: void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); + void ackMessages(UInt64 last_inserted_delivery_tag); auto getConsumerTag() const { return consumer_tag; } auto getDeliveryTag() const { return current.delivery_tag; } @@ -72,15 +74,19 @@ private: bool allowed = true; const std::atomic & stopped; - const String local_exchange; + const String local_exchange, deadletter_exchange; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; String consumer_tag; ConcurrentBoundedQueue received; + UInt64 prev_tag = 0; MessageData current; std::vector queues; std::unordered_map subscribed_queue; + std::atomic ack = false; + std::mutex wait_ack; + UInt64 max_tag = 0; bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index db4f1c7b338..52a07026c24 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -73,7 +73,8 @@ StorageRabbitMQ::StorageRabbitMQ( size_t num_consumers_, size_t num_queues_, const bool use_transactional_channel_, - const String & queue_base_) + const String & queue_base_, + const String & deadletter_exchange_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) , rabbitmq_context(Context(global_context)) @@ -85,6 +86,7 @@ StorageRabbitMQ::StorageRabbitMQ( , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) , queue_base(queue_base_) + , deadletter_exchange(deadletter_exchange_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) , login_password(std::make_pair( @@ -224,6 +226,7 @@ void StorageRabbitMQ::initExchange() void StorageRabbitMQ::bindExchange() { std::atomic binding_created = false; + size_t bound_keys = 0; /// Bridge exchange connects client's exchange with consumers' queues. if (exchange_type == AMQP::ExchangeType::headers) @@ -257,7 +260,9 @@ void StorageRabbitMQ::bindExchange() setup_channel->bindExchange(exchange_name, bridge_exchange, routing_key) .onSuccess([&]() { - binding_created = true; + ++bound_keys; + if (bound_keys == routing_keys.size()) + binding_created = true; }) .onError([&](const char * message) { @@ -434,7 +439,7 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() return std::make_shared( consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, - local_exchange, stream_cancelled); + local_exchange, deadletter_exchange, stream_cancelled); } @@ -739,10 +744,22 @@ void registerStorageRabbitMQ(StorageFactory & factory) } } + String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; + if (args_count >= 11) + { + engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); + + const auto * ast = engine_args[9]->as(); + if (ast && ast->value.getType() == Field::Types::String) + { + deadletter_exchange = safeGet(ast->value); + } + } + return StorageRabbitMQ::create( args.table_id, args.context, args.columns, host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, - num_queues, use_transactional_channel, queue_base); + num_queues, use_transactional_channel, queue_base, deadletter_exchange); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index d43f2ba27f1..7e2d6c6b35e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -72,7 +72,8 @@ protected: size_t num_consumers_, size_t num_queues_, const bool use_transactional_channel_, - const String & queue_base_); + const String & queue_base_, + const String & deadletter_exchange); private: Context global_context; @@ -90,6 +91,7 @@ private: size_t num_queues; const bool use_transactional_channel; const String queue_base; + const String deadletter_exchange; Poco::Logger * log; std::pair parsed_address; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 655dee7a816..3a2b6cd6be3 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -496,135 +496,9 @@ def test_rabbitmq_big_message(rabbitmq_cluster): assert int(result) == rabbitmq_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result) -@pytest.mark.timeout(420) -def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): - - NUM_CHANNELS = 5 - - instance.query(''' - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'test_channels_sharding', - rabbitmq_num_consumers = 5, - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - ''') - - time.sleep(1) - - i = [0] - messages_num = 10000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - key = str(randrange(1, NUM_CHANNELS)) - for message in messages: - channel.basic_publish(exchange='test_channels_sharding', routing_key=key, body=message) - connection.close() - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = instance.query('SELECT count() FROM test.view') - time.sleep(1) - if int(result) == messages_num * threads_num: - break - - for thread in threads: - thread.join() - - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - @pytest.mark.timeout(420) def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): - NUM_QUEUES = 4 - - instance.query(''' - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'test_queues_sharding', - rabbitmq_num_queues = 4, - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - ''') - - time.sleep(1) - - i = [0] - messages_num = 10000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - key = str(randrange(1, NUM_QUEUES)) - for message in messages: - channel.basic_publish(exchange='test_queues_sharding', routing_key=key, body=message) - connection.close() - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = instance.query('SELECT count() FROM test.view') - time.sleep(1) - if int(result) == messages_num * threads_num: - break - - for thread in threads: - thread.join() - - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - -@pytest.mark.timeout(420) -def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster): - NUM_CONSUMERS = 10 NUM_QUEUES = 2 @@ -639,12 +513,12 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) rabbitmq_row_delimiter = '\\n'; DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) + CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) ENGINE = MergeTree ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; + SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq; ''') time.sleep(1) @@ -662,9 +536,12 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) for _ in range(messages_num): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - key = str(randrange(1, NUM_QUEUES * NUM_CONSUMERS)) + current = 0 for message in messages: - channel.basic_publish(exchange='test_sharding', routing_key=key, body=message) + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='test_sharding', routing_key='', + properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -676,16 +553,20 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) time.sleep(random.uniform(0, 1)) thread.start() + result1 = '' while True: - result = instance.query('SELECT count() FROM test.view') + result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) - if int(result) == messages_num * threads_num: + if int(result1) == messages_num * threads_num: break + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + for thread in threads: thread.join() - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result2) == 10 @pytest.mark.timeout(420) @@ -734,8 +615,12 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 key = str(randrange(1, NUM_CONSUMERS)) + current = 0 for message in messages: - channel.basic_publish(exchange='combo', routing_key=key, body=message) + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='combo', routing_key=key, + properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -1140,11 +1025,11 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - key_num = 0 + current = 0 for message in messages: - mes_id = str(randrange(10)) - channel.basic_publish( - exchange='fanout_exchange_testing', routing_key='', + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='fanout_exchange_testing', routing_key='', properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() @@ -1236,10 +1121,11 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): channel.basic_publish(exchange='topic_exchange_testing', routing_key=key, body=message) key = "random.logs" + current = 0 for message in messages: - mes_id = str(randrange(10)) - channel.basic_publish( - exchange='topic_exchange_testing', routing_key=key, + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='topic_exchange_testing', routing_key=key, properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() @@ -1411,8 +1297,10 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): keys = ['key1', 'key2', 'key3', 'key4', 'key5'] for key in keys: + current = 0 for message in messages: - mes_id = str(randrange(10)) + current += 1 + mes_id = str(current) channel.basic_publish(exchange='multiple_bindings_testing', routing_key=key, properties=pika.BasicProperties(message_id=mes_id), body=message) @@ -1510,9 +1398,10 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): fields['type']='report' fields['year']='2020' - key_num = 0 + current = 0 for message in messages: - mes_id = str(randrange(10)) + current += 1 + mes_id = str(current) channel.basic_publish(exchange='headers_exchange_testing', routing_key='', properties=pika.BasicProperties(headers=fields, message_id=mes_id), body=message) @@ -1674,7 +1563,91 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_queue_resume(rabbitmq_cluster): +def test_rabbitmq_queue_resume_1(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_num_consumers = '2', + rabbitmq_num_queues = '2', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='queue_resume', routing_key='queue_resume', body=message, + properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 10 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + for thread in threads: + thread.join() + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + ''') + + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_num_consumers = '2', + rabbitmq_num_queues = '2', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.rabbitmq_queue_resume; + ''') + + while True: + result1 = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result1) == messages_num * threads_num: + break + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + ''') + + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + + +@pytest.mark.timeout(420) +def test_rabbitmq_queue_resume_2(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1699,7 +1672,6 @@ def test_rabbitmq_queue_resume(rabbitmq_cluster): credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1762,6 +1734,95 @@ def test_rabbitmq_queue_resume(rabbitmq_cluster): assert int(result2) == 2 +@pytest.mark.timeout(420) +def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'consumer_acks', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'consumer_acks', + rabbitmq_queue_base = 'consumer_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='consumer_acks', routing_key='consumer_acks', body=message, + properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 20 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + for thread in threads: + thread.join() + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_consumer_acks; + ''') + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; + ''') + + collected = int(instance.query('SELECT count() FROM test.view')) + + instance.query(''' + CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_queue_base = 'consumer_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + while True: + result1 = instance.query('SELECT count() FROM test.view') + time.sleep(1) + #print("receiived", result1, "collected", collected) + if int(result1) == messages_num * threads_num: + break + + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + ''') + + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + if collected < result1: + assert int(result2) == 2 + if __name__ == '__main__': cluster.start() From 22b16060f685fbe98edc18aba68b2e517aa774bc Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Jul 2020 11:14:46 +0000 Subject: [PATCH 014/535] More reliable publishing --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 3 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 3 + src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 13 +-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 3 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 30 ++++-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 6 +- .../WriteBufferToRabbitMQProducer.cpp | 93 +++++++++++-------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 11 ++- .../integration/test_storage_rabbitmq/test.py | 15 ++- 10 files changed, 104 insertions(+), 74 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 630581b13dc..0c70acaf1e3 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -137,7 +137,8 @@ Block RabbitMQBlockInputStream::readImpl() virtual_columns[3]->insert(redelivered); } - last_inserted_delivery_tag = delivery_tag; + if (delivery_tag > last_inserted_delivery_tag) + last_inserted_delivery_tag = delivery_tag; total_rows = total_rows + new_rows; buffer->allowNext(); diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index c2eae19cb86..37b39bbaeae 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -61,7 +61,10 @@ void RabbitMQBlockOutputStream::writeSuffix() child->writeSuffix(); if (buffer) + { + buffer->updateMaxWait(); buffer->finilizeProducer(); + } } } diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index cd7e7de9622..0f65fe6553c 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -21,6 +21,7 @@ namespace DB M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ M(SettingString, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ M(SettingString, rabbitmq_deadletter_exchange, "", "Exchange name to be passed as a dead-letter-exchange name.", 0) \ + M(SettingBool, rabbitmq_persistent_mode, false, "If set, delivery mode will be set to 2 (makes messages 'persistent', durable).", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 705aae7ec61..b1e63005126 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -57,13 +57,10 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { - if (ack.load() && consumer_channel) - { - consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Acknowledged messages with deliveryTags up to {}", prev_tag); - } - + if (ack.load() && max_tag && consumer_channel) + consumer_channel->ack(max_tag, AMQP::multiple); consumer_channel->close(); + received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -225,7 +222,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) received.push({deliveryTag, message_received, redelivered}); std::lock_guard lock(wait_ack); - if (ack.exchange(false) && prev_tag < max_tag && consumer_channel) + if (ack.exchange(false) && prev_tag && prev_tag <= max_tag && consumer_channel) { consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); @@ -271,7 +268,7 @@ void ReadBufferFromRabbitMQConsumer::ackMessages(UInt64 last_inserted_delivery_t { if (last_inserted_delivery_tag > prev_tag) { - std::lock_guard lock(wait_ack); + std::lock_guard lock(wait_ack); /// See onReceived() callback. prev_tag = last_inserted_delivery_tag; ack.store(true); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 8033f537e8c..6d2deb0be03 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -80,13 +80,12 @@ private: String consumer_tag; ConcurrentBoundedQueue received; - UInt64 prev_tag = 0; + UInt64 prev_tag = 0, max_tag = 0; MessageData current; std::vector queues; std::unordered_map subscribed_queue; std::atomic ack = false; std::mutex wait_ack; - UInt64 max_tag = 0; bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 52a07026c24..daa17719654 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -74,7 +74,8 @@ StorageRabbitMQ::StorageRabbitMQ( size_t num_queues_, const bool use_transactional_channel_, const String & queue_base_, - const String & deadletter_exchange_) + const String & deadletter_exchange_, + const bool persistent_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) , rabbitmq_context(Context(global_context)) @@ -87,6 +88,7 @@ StorageRabbitMQ::StorageRabbitMQ( , use_transactional_channel(use_transactional_channel_) , queue_base(queue_base_) , deadletter_exchange(deadletter_exchange_) + , persistent(persistent_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) , login_password(std::make_pair( @@ -280,7 +282,7 @@ void StorageRabbitMQ::bindExchange() void StorageRabbitMQ::unbindExchange() { - if (bridge.try_lock()) + std::call_once(flag, [&]() { if (exchange_removed.load()) return; @@ -302,9 +304,7 @@ void StorageRabbitMQ::unbindExchange() event_handler->stop(); looping_task->deactivate(); - - bridge.unlock(); - } + }); } @@ -447,7 +447,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - log, num_consumers * num_queues, use_transactional_channel, + log, num_consumers * num_queues, use_transactional_channel, persistent, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } @@ -749,17 +749,31 @@ void registerStorageRabbitMQ(StorageFactory & factory) { engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); - const auto * ast = engine_args[9]->as(); + const auto * ast = engine_args[10]->as(); if (ast && ast->value.getType() == Field::Types::String) { deadletter_exchange = safeGet(ast->value); } } + bool persistent = static_cast(rabbitmq_settings.rabbitmq_persistent_mode); + if (args_count >= 12) + { + const auto * ast = engine_args[11]->as(); + if (ast && ast->value.getType() == Field::Types::UInt64) + { + persistent = static_cast(safeGet(ast->value)); + } + else + { + throw Exception("Transactional channel parameter is a bool", ErrorCodes::BAD_ARGUMENTS); + } + } + return StorageRabbitMQ::create( args.table_id, args.context, args.columns, host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, - num_queues, use_transactional_channel, queue_base, deadletter_exchange); + num_queues, use_transactional_channel, queue_base, deadletter_exchange, persistent); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 7e2d6c6b35e..9c7df1b1421 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -73,7 +73,8 @@ protected: size_t num_queues_, const bool use_transactional_channel_, const String & queue_base_, - const String & deadletter_exchange); + const String & deadletter_exchange, + const bool persistent_); private: Context global_context; @@ -92,6 +93,7 @@ private: const bool use_transactional_channel; const String queue_base; const String deadletter_exchange; + const bool persistent; Poco::Logger * log; std::pair parsed_address; @@ -106,7 +108,7 @@ private: std::vector buffers; /// available buffers for RabbitMQ consumers String local_exchange, bridge_exchange, consumer_exchange; - std::mutex bridge; + std::once_flag flag; AMQP::Table bind_headers; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 27e4a7b8a03..990f70e0d64 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -16,13 +16,13 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CONNECT_RABBITMQ; - extern const int LOGICAL_ERROR; } static const auto QUEUE_SIZE = 50000; static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 1000; static const auto LOOP_WAIT = 10; +static const auto BATCH = 10000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address, @@ -33,7 +33,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool use_transactional_channel_, + const bool use_transactional_channel_, + const bool persistent_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_) @@ -44,6 +45,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , exchange_type(exchange_type_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) + , persistent(persistent_) , payloads(QUEUE_SIZE * num_queues) , log(log_) , delim(delimiter) @@ -57,10 +59,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( event_handler = std::make_unique(loop.get(), log); connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - /* The reason behind making a separate connection for each concurrent producer is explained here: - * https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086 - publishing from - * different threads (as outputStreams are asynchronous) with the same connection leads to internal library errors. - */ + /// New coonection for each publisher because cannot publish from different threads.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) size_t cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { @@ -74,12 +73,27 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( } producer_channel = std::make_shared(connection.get()); + producer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Prodcuer error: {}", message); + }); - /// If publishing should be wrapped in transactions if (use_transactional_channel) { producer_channel->startTransaction(); } + else + { + producer_channel->confirmSelect() + .onAck([&](uint64_t deliveryTag, bool /* multiple */) + { + if (deliveryTag > last_processed) + last_processed = deliveryTag; + }) + .onNack([&](uint64_t /* deliveryTag */, bool /* multiple */, bool /* requeue */) + { + }); + } writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); @@ -99,10 +113,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() { - stop_loop.store(true); writing_task->deactivate(); - initExchange(); - connection->close(); assert(rows == 0 && chunks.empty()); } @@ -130,6 +141,7 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); + ++delivery_tag; payloads.push(payload); } } @@ -139,52 +151,51 @@ void WriteBufferToRabbitMQProducer::writingFunc() { String payload; - while (!stop_loop || !payloads.empty()) + auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) { - while (!payloads.empty()) + payloads.push(std::string(message.body(), message.size())); + //LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); + }; + + while ((!payloads.empty() || wait_all) && connection->usable()) + { + while (!payloads.empty() && producer_channel->usable()) { payloads.pop(payload); + AMQP::Envelope envelope(payload.data(), payload.size()); + current = wait_num ? ++current % wait_num : ++current; + + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. + if (persistent) + envelope.setDeliveryMode(2); if (exchange_type == AMQP::ExchangeType::consistent_hash) { - next_queue = next_queue % num_queues + 1; - producer_channel->publish(exchange_name, std::to_string(next_queue), payload); + producer_channel->publish(exchange_name, std::to_string(current), envelope).onReturned(returned_callback); } else if (exchange_type == AMQP::ExchangeType::headers) { - AMQP::Envelope envelope(payload.data(), payload.size()); envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope, key_arguments); + producer_channel->publish(exchange_name, "", envelope, key_arguments).onReturned(returned_callback); } else { - producer_channel->publish(exchange_name, routing_keys[0], payload); + producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); } + + if (current % BATCH == 0) + iterateEventLoop(); } - iterateEventLoop(); - } -} - - -void WriteBufferToRabbitMQProducer::initExchange() -{ - std::atomic exchange_declared = false, exchange_error = false; - - producer_channel->declareExchange(exchange_name, exchange_type, AMQP::durable + AMQP::passive) - .onSuccess([&]() - { - exchange_declared = true; - }) - .onError([&](const char * /* message */) - { - exchange_error = true; - }); - - /// These variables are updated in a separate thread. - while (!exchange_declared && !exchange_error) - { - iterateEventLoop(); + if (wait_num.load() && last_processed.load() >= wait_num.load()) + { + wait_all.store(false); + LOG_DEBUG(log, "All messages are successfully published"); + } + else + { + iterateEventLoop(); + } } } @@ -198,11 +209,13 @@ void WriteBufferToRabbitMQProducer::finilizeProducer() .onSuccess([&]() { answer_received = true; + wait_all.store(false); LOG_TRACE(log, "All messages were successfully published"); }) .onError([&](const char * message1) { answer_received = true; + wait_all.store(false); wait_rollback = true; LOG_TRACE(log, "Publishing not successful: {}", message1); producer_channel->rollbackTransaction() diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 20b133b6930..ebeb21075bf 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -28,7 +28,8 @@ public: const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool use_transactional_channel_, + const bool use_transactional_channel_, + const bool persistent_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_ @@ -39,10 +40,10 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } void finilizeProducer(); + void updateMaxWait() { wait_num.store(delivery_tag); } private: void nextImpl() override; - void initExchange(); void iterateEventLoop(); void writingFunc(); @@ -52,10 +53,10 @@ private: AMQP::ExchangeType exchange_type; const size_t num_queues; const bool use_transactional_channel; + const bool persistent; AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; - std::atomic stop_loop = false; std::unique_ptr loop; std::unique_ptr event_handler; @@ -63,7 +64,9 @@ private: ChannelPtr producer_channel; ConcurrentBoundedQueue payloads; - size_t next_queue = 0; + UInt64 delivery_tag = 0, current = 0; + std::atomic wait_all = true; + std::atomic wait_num = 0, last_processed = 0; Poco::Logger * log; const std::optional delim; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 3a2b6cd6be3..abf0a20d18f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -845,7 +845,6 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_name = 'over', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'over', - rabbitmq_num_consumers = 6, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.rabbitmq_overload (key UInt64, value UInt64) @@ -892,7 +891,6 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.view_overload') time.sleep(1) - print("Result", int(result), "Expected", messages_num * threads_num) if int(result) == messages_num * threads_num: break @@ -1539,7 +1537,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): result = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") assert int(result) == 1 - result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view") + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") expected = '''\ 0 0 virtuals_mv amq.ctag 1 0 1 1 virtuals_mv amq.ctag 2 0 @@ -1793,7 +1791,7 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; ''') - collected = int(instance.query('SELECT count() FROM test.view')) + #collected = int(instance.query('SELECT count() FROM test.view')) instance.query(''' CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) @@ -1808,10 +1806,10 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) #print("receiived", result1, "collected", collected) - if int(result1) == messages_num * threads_num: + if int(result1) >= messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + #result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; @@ -1819,9 +1817,8 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): DROP TABLE IF EXISTS test.view; ''') - assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - if collected < result1: - assert int(result2) == 2 + # >= because at-least-once + assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) if __name__ == '__main__': From 92efb847534d5fd088f404153452b46ecc6d7c79 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Jul 2020 15:53:55 +0000 Subject: [PATCH 015/535] Update docs --- .../table-engines/integrations/rabbitmq.md | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 7d09c6f72a5..e870471b4eb 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -44,8 +44,11 @@ Optional parameters: - `rabbitmq_routing_key_list` – A comma-separated list of routing keys. - `rabbitmq_row_delimiter` – Delimiter character, which ends the message. - `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. -- `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. Single queue can contain up to 50K messages at the same time. +- `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. - `rabbitmq_transactional_channel` – Wrap insert queries in transactions. Default: `0`. +- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. This settings should be used to be able to restore reading from declared durable queues in case of some failure when not all messages were successfully consumed. Note: it makes sence only if messages are sent with delivery mode 2 (marked 'persistent', durable). To be able to resume consumption from one specific queue in case of failure - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. +- `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified. +- `persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`. Required configuration: @@ -92,13 +95,13 @@ Exchange type options: - `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`. - `consistent-hash` - Data is evenly distributed between all bound tables (where exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. -If exchange type is not specified, then default is `fanout` and routing keys for data publishing must be randomized in range `[1, num_consumers]` for every message/batch (or in range `[1, num_consumers * num_queues]` if `rabbitmq_num_queues` is set). This table configuration works quicker then any other, especially when `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are set. - -If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are specified along with `rabbitmq_exchange_type`, then: +If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then: - `rabbitmq-consistent-hash-exchange` plugin must be enabled. - `message_id` property of the published messages must be specified (unique for each message/batch). +Do not use the same table for inserts and materialized views. + Example: ``` sql @@ -120,3 +123,10 @@ Example: SELECT key, value FROM daily ORDER BY key; ``` + +## Virtual Columns {#virtual-columns} + +- `_exchange_name` - RabbitMQ exchange name. +- `_consumer_tag` - ConsumerTag of the consumer that received the message. +- `_delivery_tag` - DeliveryTag if the message. Scoped per consumer. +- `_redelivered` - Redelivered flag of the message. From 0ee54c8a4a25c5f7fe16a0a4bb22e9236637089b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Jul 2020 08:22:45 +0000 Subject: [PATCH 016/535] Fix build, async acks -> sync acks, fix tests --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 33 +++-- .../RabbitMQ/RabbitMQBlockInputStream.h | 1 - .../ReadBufferFromRabbitMQConsumer.cpp | 28 ++-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 9 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 3 - .../WriteBufferToRabbitMQProducer.cpp | 1 + .../integration/test_storage_rabbitmq/test.py | 133 +++++++----------- 7 files changed, 83 insertions(+), 125 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 0c70acaf1e3..1a20699d23a 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -124,23 +124,26 @@ Block RabbitMQBlockInputStream::readImpl() auto new_rows = read_rabbitmq_message(); - auto exchange_name = storage.getExchange(); - auto consumer_tag = buffer->getConsumerTag(); - auto delivery_tag = buffer->getDeliveryTag(); - auto redelivered = buffer->getRedelivered(); - - for (size_t i = 0; i < new_rows; ++i) + if (new_rows) { - virtual_columns[0]->insert(exchange_name); - virtual_columns[1]->insert(consumer_tag); - virtual_columns[2]->insert(delivery_tag); - virtual_columns[3]->insert(redelivered); + auto exchange_name = storage.getExchange(); + auto consumer_tag = buffer->getConsumerTag(); + auto delivery_tag = buffer->getDeliveryTag(); + auto redelivered = buffer->getRedelivered(); + + buffer->updateNextDeliveryTag(delivery_tag); + + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(exchange_name); + virtual_columns[1]->insert(consumer_tag); + virtual_columns[2]->insert(delivery_tag); + virtual_columns[3]->insert(redelivered); + } + + total_rows = total_rows + new_rows; } - if (delivery_tag > last_inserted_delivery_tag) - last_inserted_delivery_tag = delivery_tag; - - total_rows = total_rows + new_rows; buffer->allowNext(); if (!new_rows || !checkTimeLimit()) @@ -167,7 +170,7 @@ void RabbitMQBlockInputStream::readSuffixImpl() if (!buffer) return; - buffer->ackMessages(last_inserted_delivery_tag); + buffer->ackMessages(); } } diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h index 09cda6ff94f..f4405ce44df 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h @@ -39,7 +39,6 @@ private: const Block virtual_header; ConsumerBufferPtr buffer; - UInt64 last_inserted_delivery_tag; }; } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index b1e63005126..27bb7c12d3d 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,7 +14,7 @@ namespace DB { -static const auto QUEUE_SIZE = 50000; /// Equals capacity of a single rabbitmq queue +static const auto QUEUE_SIZE = 50000; ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, @@ -57,10 +57,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { - if (ack.load() && max_tag && consumer_channel) - consumer_channel->ack(max_tag, AMQP::multiple); consumer_channel->close(); - received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -68,7 +65,7 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { - bool bindings_created = false, bindings_error = false; + std::atomic bindings_created = false, bindings_error = false; auto success_callback = [&](const std::string & queue_name_, int msgcount, int /* consumercount */) { @@ -220,13 +217,6 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) message_received += row_delimiter; received.push({deliveryTag, message_received, redelivered}); - - std::lock_guard lock(wait_ack); - if (ack.exchange(false) && prev_tag && prev_tag <= max_tag && consumer_channel) - { - consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); - } } }) .onError([&](const char * message) @@ -239,7 +229,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) void ReadBufferFromRabbitMQConsumer::checkSubscription() { - if (count_subscribed == num_queues) + if (count_subscribed == num_queues || !consumer_channel->usable()) return; wait_subscribed = num_queues; @@ -264,13 +254,14 @@ void ReadBufferFromRabbitMQConsumer::checkSubscription() } -void ReadBufferFromRabbitMQConsumer::ackMessages(UInt64 last_inserted_delivery_tag) +void ReadBufferFromRabbitMQConsumer::ackMessages() { - if (last_inserted_delivery_tag > prev_tag) + UInt64 delivery_tag = last_inserted_delivery_tag; + if (delivery_tag && delivery_tag > prev_tag) { - std::lock_guard lock(wait_ack); /// See onReceived() callback. - prev_tag = last_inserted_delivery_tag; - ack.store(true); + prev_tag = delivery_tag; + consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. + LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); } } @@ -291,7 +282,6 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() auto * new_position = const_cast(current.message.data()); BufferBase::set(new_position, current.message.size(), 0); allowed = false; - max_tag = current.delivery_tag; return true; } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 6d2deb0be03..4854858c9b9 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -50,7 +50,8 @@ public: void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); - void ackMessages(UInt64 last_inserted_delivery_tag); + void updateNextDeliveryTag(UInt64 delivery_tag) { last_inserted_delivery_tag = delivery_tag; } + void ackMessages(); auto getConsumerTag() const { return consumer_tag; } auto getDeliveryTag() const { return current.delivery_tag; } @@ -80,18 +81,16 @@ private: String consumer_tag; ConcurrentBoundedQueue received; - UInt64 prev_tag = 0, max_tag = 0; + UInt64 last_inserted_delivery_tag = 0, prev_tag = 0; MessageData current; std::vector queues; std::unordered_map subscribed_queue; - std::atomic ack = false; - std::mutex wait_ack; bool nextImpl() override; void initQueueBindings(const size_t queue_id); void subscribe(const String & queue_name); void iterateEventLoop(); - }; + } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index daa17719654..6a842a69550 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -284,9 +284,6 @@ void StorageRabbitMQ::unbindExchange() { std::call_once(flag, [&]() { - if (exchange_removed.load()) - return; - setup_channel->removeExchange(bridge_exchange) .onSuccess([&]() { diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 990f70e0d64..1b48232aa52 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -150,6 +150,7 @@ void WriteBufferToRabbitMQProducer::countRow() void WriteBufferToRabbitMQProducer::writingFunc() { String payload; + current = 0; auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) { diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index abf0a20d18f..bc4585fb6f2 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -120,7 +120,6 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'new', rabbitmq_exchange_name = 'new', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -136,13 +135,13 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='new', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='', body=message) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='new', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='', body=message) connection.close() @@ -191,7 +190,6 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'empty', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -205,8 +203,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'json', - rabbitmq_exchange_name = 'delim1', + rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' ''') @@ -221,14 +218,14 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='delim1', routing_key='json', body=message) + channel.basic_publish(exchange='json', routing_key='', body=message) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='delim1', routing_key='json', body=message) + channel.basic_publish(exchange='json', routing_key='', body=message) result = '' while True: @@ -246,8 +243,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'csv', - rabbitmq_exchange_name = 'delim2', + rabbitmq_exchange_name = 'csv', rabbitmq_format = 'CSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -262,7 +258,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): messages.append('{i}, {i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='delim2', routing_key='csv', body=message) + channel.basic_publish(exchange='csv', routing_key='', body=message) result = '' while True: @@ -281,8 +277,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'tsv', - rabbitmq_exchange_name = 'delim3', + rabbitmq_exchange_name = 'tsv', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -297,7 +292,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): messages.append('{i}\t{i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='delim3', routing_key='tsv', body=message) + channel.basic_publish(exchange='tsv', routing_key='', body=message) result = '' while True: @@ -318,7 +313,6 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mv', - rabbitmq_routing_key_list = 'mv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -337,7 +331,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='mv', routing_key='mv', body=message) + channel.basic_publish(exchange='mv', routing_key='', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -362,7 +356,6 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mvsq', - rabbitmq_routing_key_list = 'mvsq', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -381,7 +374,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='mvsq', routing_key='mvsq', body=message) + channel.basic_publish(exchange='mvsq', routing_key='', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -408,7 +401,6 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mmv', - rabbitmq_routing_key_list = 'mmv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view1 (key UInt64, value UInt64) @@ -432,7 +424,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='mmv', routing_key='mmv', body=message) + channel.basic_publish(exchange='mmv', routing_key='', body=message) while True: result1 = instance.query('SELECT * FROM test.view1') @@ -470,7 +462,6 @@ def test_rabbitmq_big_message(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'big', - rabbitmq_routing_key_list = 'big', rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value String) ENGINE = MergeTree @@ -480,7 +471,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): ''') for message in messages: - channel.basic_publish(exchange='big', routing_key='big', body=message) + channel.basic_publish(exchange='big', routing_key='', body=message) while True: result = instance.query('SELECT count() FROM test.view') @@ -580,7 +571,8 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'combo', - rabbitmq_num_consumers = 4, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') @@ -614,12 +606,11 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): for _ in range(messages_num): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - key = str(randrange(1, NUM_CONSUMERS)) current = 0 for message in messages: current += 1 mes_id = str(current) - channel.basic_publish(exchange='combo', routing_key=key, + channel.basic_publish(exchange='combo', routing_key='', properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() @@ -911,8 +902,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): def test_rabbitmq_direct_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; @@ -927,14 +917,15 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): CREATE TABLE test.direct_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_exchange_name = 'direct_exchange_testing', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'direct_{0}', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.direct_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.direct_exchange_{0}; + SELECT key, value FROM test.direct_exchange_{0}; '''.format(consumer_id)) i = [0] @@ -985,8 +976,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): def test_rabbitmq_fanout_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') @@ -1000,14 +990,15 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): CREATE TABLE test.fanout_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_routing_key_list = 'key_{0}', rabbitmq_exchange_name = 'fanout_exchange_testing', rabbitmq_exchange_type = 'fanout', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.fanout_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.fanout_exchange_{0}; + SELECT key, value FROM test.fanout_exchange_{0}; '''.format(consumer_id)) i = [0] @@ -1055,8 +1046,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): def test_rabbitmq_topic_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') @@ -1070,14 +1060,15 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): CREATE TABLE test.topic_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.{0}', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.topic_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.topic_exchange_{0}; + SELECT key, value FROM test.topic_exchange_{0}; '''.format(consumer_id)) for consumer_id in range(num_tables): @@ -1088,14 +1079,15 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): CREATE TABLE test.topic_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 4, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.logs', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.topic_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.topic_exchange_{0}; + SELECT key, value FROM test.topic_exchange_{0}; '''.format(num_tables + consumer_id)) i = [0] @@ -1166,7 +1158,8 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): CREATE TABLE test.{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 10, + rabbitmq_num_consumers = 4, + rabbitmq_num_queues = 2, rabbitmq_exchange_type = 'consistent_hash', rabbitmq_exchange_name = 'hash_exchange_testing', rabbitmq_format = 'JSONEachRow', @@ -1229,7 +1222,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): thread.join() assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - assert int(result2) >= 30 + assert int(result2) == 4 * num_tables @@ -1237,34 +1230,15 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): def test_rabbitmq_multiple_bindings(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') instance.query(''' - DROP TABLE IF EXISTS test.bindings_1; - DROP TABLE IF EXISTS test.bindings_1_mv; - CREATE TABLE test.bindings_1 (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, - rabbitmq_num_queues = 2, - rabbitmq_exchange_name = 'multiple_bindings_testing', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.bindings_1_mv TO test.destination AS - SELECT * FROM test.bindings_1; - ''') - - # in case num_consumers and num_queues are not set - multiple bindings are implemented differently, so test them too - instance.query(''' - DROP TABLE IF EXISTS test.bindings_2; - DROP TABLE IF EXISTS test.bindings_2_mv; - CREATE TABLE test.bindings_2 (key UInt64, value UInt64) + DROP TABLE IF EXISTS test.bindings; + DROP TABLE IF EXISTS test.bindings_mv; + CREATE TABLE test.bindings (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'multiple_bindings_testing', @@ -1272,8 +1246,8 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.bindings_2_mv TO test.destination AS - SELECT * FROM test.bindings_2; + CREATE MATERIALIZED VIEW test.bindings_mv TO test.destination AS + SELECT * FROM test.bindings; ''') i = [0] @@ -1295,12 +1269,8 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): keys = ['key1', 'key2', 'key3', 'key4', 'key5'] for key in keys: - current = 0 for message in messages: - current += 1 - mes_id = str(current) - channel.basic_publish(exchange='multiple_bindings_testing', routing_key=key, - properties=pika.BasicProperties(message_id=mes_id), body=message) + channel.basic_publish(exchange='multiple_bindings_testing', routing_key=key, body=message) connection.close() @@ -1316,32 +1286,31 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.destination') time.sleep(1) - if int(result) == messages_num * threads_num * 5 * 2: + if int(result) == messages_num * threads_num * 5: break for thread in threads: thread.join() instance.query(''' - DROP TABLE IF EXISTS test.bindings_1; - DROP TABLE IF EXISTS test.bindings_2; + DROP TABLE IF EXISTS test.bindings; + DROP TABLE IF EXISTS test.bindings_mv; DROP TABLE IF EXISTS test.destination; ''') - assert int(result) == messages_num * threads_num * 5 * 2, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num * threads_num * 5, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) def test_rabbitmq_headers_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') - num_tables_to_receive = 3 + num_tables_to_receive = 2 for consumer_id in range(num_tables_to_receive): print("Setting up table {}".format(consumer_id)) instance.query(''' @@ -1350,14 +1319,14 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): CREATE TABLE test.headers_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 4, + rabbitmq_num_consumers = 2, rabbitmq_exchange_name = 'headers_exchange_testing', rabbitmq_exchange_type = 'headers', rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2020', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.headers_exchange_{0}; + SELECT key, value FROM test.headers_exchange_{0}; '''.format(consumer_id)) num_tables_to_ignore = 2 @@ -1375,7 +1344,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.headers_exchange_{0}; + SELECT key, value FROM test.headers_exchange_{0}; '''.format(consumer_id + num_tables_to_receive)) i = [0] @@ -1683,7 +1652,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): connection.close() threads = [] - threads_num = 10 + threads_num = 20 for _ in range(threads_num): threads.append(threading.Thread(target=produce)) for thread in threads: From 469e46a53904a1d765fb94ab393e60f3b82dc225 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Jul 2020 12:41:58 +0000 Subject: [PATCH 017/535] Fix build --- .../RabbitMQ/WriteBufferToRabbitMQProducer.cpp | 11 +++++++---- src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 1b48232aa52..82cb3f2311d 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -150,7 +150,7 @@ void WriteBufferToRabbitMQProducer::countRow() void WriteBufferToRabbitMQProducer::writingFunc() { String payload; - current = 0; + UInt64 message_id = 0; auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) { @@ -164,7 +164,10 @@ void WriteBufferToRabbitMQProducer::writingFunc() { payloads.pop(payload); AMQP::Envelope envelope(payload.data(), payload.size()); - current = wait_num ? ++current % wait_num : ++current; + + ++message_id; + if (wait_num) + message_id %= wait_num; /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) @@ -172,7 +175,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() if (exchange_type == AMQP::ExchangeType::consistent_hash) { - producer_channel->publish(exchange_name, std::to_string(current), envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, std::to_string(message_id), envelope).onReturned(returned_callback); } else if (exchange_type == AMQP::ExchangeType::headers) { @@ -184,7 +187,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); } - if (current % BATCH == 0) + if (message_id % BATCH == 0) iterateEventLoop(); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index ebeb21075bf..30e647af471 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -64,7 +64,7 @@ private: ChannelPtr producer_channel; ConcurrentBoundedQueue payloads; - UInt64 delivery_tag = 0, current = 0; + UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0, last_processed = 0; From 230938d3a3082fbf241c9d873571231a69a5f450 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 11 Jul 2020 15:12:42 +0800 Subject: [PATCH 018/535] Refactor joinGet and implement multi-key lookup. --- src/Functions/FunctionJoinGet.cpp | 83 +++++++++---------- src/Functions/FunctionJoinGet.h | 11 +-- src/Interpreters/HashJoin.cpp | 69 ++++++++------- src/Interpreters/HashJoin.h | 10 +-- src/Interpreters/misc.h | 2 +- .../0_stateless/01080_join_get_null.reference | 2 +- .../0_stateless/01080_join_get_null.sql | 12 +-- .../01400_join_get_with_multi_keys.reference | 1 + .../01400_join_get_with_multi_keys.sql | 9 ++ 9 files changed, 104 insertions(+), 95 deletions(-) create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.reference create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.sql diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index a33b70684a5..1badc689c6a 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include -#include #include @@ -16,19 +16,35 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +template +void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t) +{ + Block keys; + for (size_t i = 2; i < arguments.size(); ++i) + { + auto key = block.getByPosition(arguments[i]); + keys.insert(std::move(key)); + } + block.getByPosition(result) = join->joinGet(keys, result_block); +} + +template +ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const +{ + return std::make_unique>(join, Block{{return_type->createColumn(), return_type, attr_name}}); +} + static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { - if (arguments.size() != 3) - throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) { join_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); size_t dot = join_name.find('.'); String database_name; @@ -43,10 +59,12 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co ++dot; } String table_name = join_name.substr(dot); + if (table_name.empty()) + throw Exception("joinGet does not allow empty table name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context); auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) - throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception("Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); String attr_name; if (const auto * name_col = checkAndGetColumnConst(arguments[1].column.get())) @@ -54,57 +72,30 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co attr_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[1].type->getName() - + " of second argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[1].type->getName() + " of second argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_pair(storage_join, attr_name); } template FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const { + if (arguments.size() < 3) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be greater or equal to 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto [storage_join, attr_name] = getJoin(arguments, context); auto join = storage_join->getJoin(); - DataTypes data_types(arguments.size()); - + DataTypes data_types(arguments.size() - 2); + for (size_t i = 2; i < arguments.size(); ++i) + data_types[i - 2] = arguments[i].type; + auto return_type = join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto return_type = join->joinGetReturnType(attr_name, or_null); return std::make_unique>(table_lock, storage_join, join, attr_name, data_types, return_type); } -template -DataTypePtr JoinGetOverloadResolver::getReturnType(const ColumnsWithTypeAndName & arguments) const -{ - auto [storage_join, attr_name] = getJoin(arguments, context); - auto join = storage_join->getJoin(); - return join->joinGetReturnType(attr_name, or_null); -} - - -template -void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto ctn = block.getByPosition(arguments[2]); - if (isColumnConst(*ctn.column)) - ctn.column = ctn.column->cloneResized(1); - ctn.name = ""; // make sure the key name never collide with the join columns - Block key_block = {ctn}; - join->joinGet(key_block, attr_name, or_null); - auto & result_ctn = key_block.getByPosition(1); - if (isColumnConst(*ctn.column)) - result_ctn.column = ColumnConst::create(result_ctn.column, input_rows_count); - block.getByPosition(result) = result_ctn; -} - -template -ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const -{ - return std::make_unique>(join, attr_name); -} - void registerFunctionJoinGet(FunctionFactory & factory) { // joinGet diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index a82da589960..6b3b1202f60 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -13,14 +13,14 @@ template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(HashJoinPtr join_, String attr_name_) - : join(std::move(join_)), attr_name(std::move(attr_name_)) {} + ExecutableFunctionJoinGet(HashJoinPtr join_, const Block & result_block_) + : join(std::move(join_)), result_block(result_block_) {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; @@ -28,7 +28,7 @@ public: private: HashJoinPtr join; - const String attr_name; + Block result_block; }; template @@ -77,13 +77,14 @@ public: String getName() const override { return name; } FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override; - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override; + DataTypePtr getReturnType(const ColumnsWithTypeAndName &) const override { return {}; } // Not used bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } private: const Context & context; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 27294a57675..ffc806b9e88 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int SET_SIZE_LIMIT_EXCEEDED; extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace @@ -1109,27 +1110,34 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) block = block.cloneWithColumns(std::move(dst_columns)); } -static void checkTypeOfKey(const Block & block_left, const Block & block_right) -{ - const auto & [c1, left_type_origin, left_name] = block_left.safeGetByPosition(0); - const auto & [c2, right_type_origin, right_name] = block_right.safeGetByPosition(0); - auto left_type = removeNullable(left_type_origin); - auto right_type = removeNullable(right_type_origin); - if (!left_type->equals(*right_type)) - throw Exception("Type mismatch of columns to joinGet by: " - + left_name + " " + left_type->getName() + " at left, " - + right_name + " " + right_type->getName() + " at right", - ErrorCodes::TYPE_MISMATCH); -} - - -DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null) const +DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const { std::shared_lock lock(data->rwlock); + size_t num_keys = data_types.size(); + if (right_table_keys.columns() != num_keys) + throw Exception( + "Number of arguments for function joinGet" + toString(or_null ? "OrNull" : "") + + " doesn't match: passed, should be equal to " + toString(num_keys), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < num_keys; ++i) + { + const auto & left_type_origin = data_types[i]; + const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i); + auto left_type = removeNullable(left_type_origin); + auto right_type = removeNullable(right_type_origin); + if (!left_type->equals(*right_type)) + throw Exception( + "Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is " + + right_type->getName(), + ErrorCodes::TYPE_MISMATCH); + } + if (!sample_block_with_columns_to_add.has(column_name)) throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + auto elem = sample_block_with_columns_to_add.getByName(column_name); if (or_null) elem.type = makeNullable(elem.type); @@ -1138,34 +1146,33 @@ DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null template -void HashJoin::joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const +ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const { - joinBlockImpl( - block, {block.getByPosition(0).name}, block_with_columns_to_add, maps_); + // Assemble the key block with correct names. + Block keys; + for (size_t i = 0; i < block.columns(); ++i) + { + auto key = block.getByPosition(i); + key.name = key_names_right[i]; + keys.insert(std::move(key)); + } + + joinBlockImpl( + keys, key_names_right, block_with_columns_to_add, maps_); + return keys.getByPosition(keys.columns() - 1); } -// TODO: support composite key // TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All -void HashJoin::joinGet(Block & block, const String & column_name, bool or_null) const +ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { std::shared_lock lock(data->rwlock); - if (key_names_right.size() != 1) - throw Exception("joinGet only supports StorageJoin containing exactly one key", ErrorCodes::UNSUPPORTED_JOIN_KEYS); - - checkTypeOfKey(block, right_table_keys); - - auto elem = sample_block_with_columns_to_add.getByName(column_name); - if (or_null) - elem.type = makeNullable(elem.type); - elem.column = elem.type->createColumn(); - if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && kind == ASTTableJoin::Kind::Left) { - joinGetImpl(block, {elem}, std::get(data->maps)); + return joinGetImpl(block, block_with_columns_to_add, std::get(data->maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 67d83d27a6d..025f41ac28f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -162,11 +162,11 @@ public: */ void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; - /// Infer the return type for joinGet function - DataTypePtr joinGetReturnType(const String & column_name, bool or_null) const; + /// Check joinGet arguments and infer the return type. + DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; - /// Used by joinGet function that turns StorageJoin into a dictionary - void joinGet(Block & block, const String & column_name, bool or_null) const; + /// Used by joinGet function that turns StorageJoin into a dictionary. + ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ @@ -383,7 +383,7 @@ private: void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; template - void joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; + ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); }; diff --git a/src/Interpreters/misc.h b/src/Interpreters/misc.h index 094dfbbbb81..cae2691ca1f 100644 --- a/src/Interpreters/misc.h +++ b/src/Interpreters/misc.h @@ -28,7 +28,7 @@ inline bool functionIsLikeOperator(const std::string & name) inline bool functionIsJoinGet(const std::string & name) { - return name == "joinGet" || startsWith(name, "dictGet"); + return startsWith(name, "joinGet"); } inline bool functionIsDictGet(const std::string & name) diff --git a/tests/queries/0_stateless/01080_join_get_null.reference b/tests/queries/0_stateless/01080_join_get_null.reference index bfde072a796..0cfbf08886f 100644 --- a/tests/queries/0_stateless/01080_join_get_null.reference +++ b/tests/queries/0_stateless/01080_join_get_null.reference @@ -1 +1 @@ -2 2 +2 diff --git a/tests/queries/0_stateless/01080_join_get_null.sql b/tests/queries/0_stateless/01080_join_get_null.sql index 71e7ddf8e75..9f782452d34 100644 --- a/tests/queries/0_stateless/01080_join_get_null.sql +++ b/tests/queries/0_stateless/01080_join_get_null.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS test_joinGet; -DROP TABLE IF EXISTS test_join_joinGet; -CREATE TABLE test_joinGet(id Int32, user_id Nullable(Int32)) Engine = Memory(); -CREATE TABLE test_join_joinGet(user_id Int32, name String) Engine = Join(ANY, LEFT, user_id); +CREATE TABLE test_joinGet(user_id Nullable(Int32), name String) Engine = Join(ANY, LEFT, user_id); -INSERT INTO test_join_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'); +INSERT INTO test_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'), (null, 'd'); -SELECT 2 id, toNullable(toInt32(2)) user_id WHERE joinGet(test_join_joinGet, 'name', user_id) != ''; +SELECT toNullable(toInt32(2)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; + +-- If the JOIN keys are Nullable fields, the rows where at least one of the keys has the value NULL are not joined. +SELECT cast(null AS Nullable(Int32)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; DROP TABLE test_joinGet; -DROP TABLE test_join_joinGet; diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference new file mode 100644 index 00000000000..49d59571fbf --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference @@ -0,0 +1 @@ +0.1 diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql new file mode 100644 index 00000000000..73068270762 --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test_joinGet; + +CREATE TABLE test_joinGet(a String, b String, c Float64) ENGINE = Join(any, left, a, b); + +INSERT INTO test_joinGet VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', 0.3); + +SELECT joinGet(test_joinGet, 'c', 'ab', '1'); + +DROP TABLE test_joinGet; From 763c337be99e7bcd482e442c54e383f47c2f1a32 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 Jul 2020 19:45:18 +0000 Subject: [PATCH 019/535] Remove redundant, move subscription --- .../ReadBufferFromRabbitMQConsumer.cpp | 172 +++++------------- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 11 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 12 +- 3 files changed, 54 insertions(+), 141 deletions(-) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 27bb7c12d3d..9f036a8a9b6 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -21,15 +21,12 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - const AMQP::ExchangeType & exchange_type_, - const Names & routing_keys_, size_t channel_id_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, size_t num_queues_, - const String & local_exchange_, const String & deadletter_exchange_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) @@ -37,8 +34,6 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , setup_channel(setup_channel_) , event_handler(event_handler_) , exchange_name(exchange_name_) - , exchange_type(exchange_type_) - , routing_keys(routing_keys_) , channel_id(channel_id_) , queue_base(queue_base_) , hash_exchange(hash_exchange_) @@ -46,24 +41,24 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) - , local_exchange(local_exchange_) , deadletter_exchange(deadletter_exchange_) , received(QUEUE_SIZE * num_queues) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) - initQueueBindings(queue_id); + bindQueue(queue_id); + + consumer_channel->onReady([&]() { subscribe(); }); } ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { consumer_channel->close(); - received.clear(); BufferBase::set(nullptr, 0, 0); } -void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) +void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) { std::atomic bindings_created = false, bindings_error = false; @@ -75,87 +70,17 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) if (msgcount) LOG_TRACE(log, "Queue " + queue_name_ + " is non-empty. Non-consumed messaged will also be delivered."); - subscribed_queue[queue_name_] = false; - subscribe(queues.back()); - - if (hash_exchange) + /// Binding key must be a string integer in case of hash exchange (here it is either hash or fanout). + setup_channel->bindQueue(exchange_name, queue_name_, std::to_string(channel_id)) + .onSuccess([&] { - String binding_key; - if (queues.size() == 1) - binding_key = std::to_string(channel_id); - else - binding_key = std::to_string(channel_id + queue_id); - - /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor - * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. - */ - String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_exchange; - - setup_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); - }); - } - else if (exchange_type == AMQP::ExchangeType::fanout) + bindings_created = true; + }) + .onError([&](const char * message) { - setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind to key. Reason: {}", message); - }); - } - else if (exchange_type == AMQP::ExchangeType::headers) - { - AMQP::Table binding_arguments; - std::vector matching; - - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - binding_arguments[matching[0]] = matching[1]; - matching.clear(); - } - - setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); - }); - } - else - { - /// Means there is only one queue with one consumer - no even distribution needed - no hash-exchange. - for (const auto & routing_key : routing_keys) - { - /// Binding directly to exchange, specified by the client. - setup_channel->bindQueue(exchange_name, queue_name_, routing_key) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); - }); - } - } + bindings_error = true; + LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); + }); }; auto error_callback([&](const char * message) @@ -187,43 +112,42 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) } -void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) +void ReadBufferFromRabbitMQConsumer::subscribe() { - if (subscribed_queue[queue_name]) - return; - - consumer_channel->consume(queue_name) - .onSuccess([&](const std::string & consumer) + count_subscribed = 0; + for (const auto & queue_name : queues) { - subscribed_queue[queue_name] = true; - ++count_subscribed; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); - - consumer_error = false; - consumer_tag = consumer; - - consumer_channel->onError([&](const char * message) + consumer_channel->consume(queue_name) + .onSuccess([&](const std::string & consumer) { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + ++count_subscribed; + LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + + consumer_error = false; + consumer_tag = consumer; + + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + }); + }) + .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) + { + if (message.bodySize()) + { + String message_received = std::string(message.body(), message.body() + message.bodySize()); + if (row_delimiter != '\0') + message_received += row_delimiter; + + received.push({delivery_tag, message_received, redelivered}); + } + }) + .onError([&](const char * message) + { + consumer_error = true; + LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); }); - }) - .onReceived([&](const AMQP::Message & message, uint64_t deliveryTag, bool redelivered) - { - size_t message_size = message.bodySize(); - if (message_size && message.body() != nullptr) - { - String message_received = std::string(message.body(), message.body() + message_size); - if (row_delimiter != '\0') - message_received += row_delimiter; - - received.push({deliveryTag, message_received, redelivered}); - } - }) - .onError([&](const char * message) - { - consumer_error = true; - LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); - }); + } } @@ -246,11 +170,7 @@ void ReadBufferFromRabbitMQConsumer::checkSubscription() if (count_subscribed == num_queues) return; - /// A case that should never normally happen. - for (auto & queue : queues) - { - subscribe(queue); - } + subscribe(); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 4854858c9b9..6448389aea5 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -27,15 +27,12 @@ public: ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - const AMQP::ExchangeType & exchange_type_, - const Names & routing_keys_, size_t channel_id_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, size_t num_queues_, - const String & local_exchange_, const String & deadletter_exchange_, const std::atomic & stopped_); @@ -63,8 +60,6 @@ private: HandlerPtr event_handler; const String exchange_name; - const AMQP::ExchangeType exchange_type; - const Names routing_keys; const size_t channel_id; const String queue_base; const bool hash_exchange; @@ -75,7 +70,7 @@ private: bool allowed = true; const std::atomic & stopped; - const String local_exchange, deadletter_exchange; + const String deadletter_exchange; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; @@ -88,8 +83,8 @@ private: bool nextImpl() override; - void initQueueBindings(const size_t queue_id); - void subscribe(const String & queue_name); + void bindQueue(size_t queue_id); + void subscribe(); void iterateEventLoop(); }; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 6a842a69550..f31cf3f4f72 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -126,7 +126,7 @@ StorageRabbitMQ::StorageRabbitMQ( if (exchange_type_ != ExchangeType::DEFAULT) { - if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; + if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct; else if (exchange_type_ == ExchangeType::TOPIC) exchange_type = AMQP::ExchangeType::topic; else if (exchange_type_ == ExchangeType::HASH) exchange_type = AMQP::ExchangeType::consistent_hash; @@ -140,12 +140,11 @@ StorageRabbitMQ::StorageRabbitMQ( if (exchange_type == AMQP::ExchangeType::headers) { - std::vector matching; for (const auto & header : routing_keys) { + std::vector matching; boost::split(matching, header, [](char c){ return c == '='; }); bind_headers[matching[0]] = matching[1]; - matching.clear(); } } @@ -192,7 +191,7 @@ void StorageRabbitMQ::initExchange() + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - /// Bridge exchange is needed to easily disconnect consumer queues. + /// Bridge exchange is needed to easily disconnect consumer queues. Also simplifies queue bindings a lot. setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) .onError([&](const char * message) { @@ -230,7 +229,6 @@ void StorageRabbitMQ::bindExchange() std::atomic binding_created = false; size_t bound_keys = 0; - /// Bridge exchange connects client's exchange with consumers' queues. if (exchange_type == AMQP::ExchangeType::headers) { setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) @@ -434,9 +432,9 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, + consumer_channel, setup_channel, event_handler, consumer_exchange, next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, - local_exchange, deadletter_exchange, stream_cancelled); + deadletter_exchange, stream_cancelled); } From 5a934c079e691d4231b08a1a96204a6ebd5d85d2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 31 Jul 2020 04:59:56 +0000 Subject: [PATCH 020/535] Add connection restore in insert, better confirms --- .../WriteBufferToRabbitMQProducer.cpp | 127 +++++++++++++----- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 14 +- 2 files changed, 104 insertions(+), 37 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 82cb3f2311d..d74e94d74d2 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -25,7 +25,7 @@ static const auto LOOP_WAIT = 10; static const auto BATCH = 10000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( - std::pair & parsed_address, + std::pair & parsed_address_, Context & global_context, const std::pair & login_password_, const Names & routing_keys_, @@ -39,6 +39,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( size_t rows_per_message, size_t chunk_size_) : WriteBuffer(nullptr, 0) + , parsed_address(parsed_address_) , login_password(login_password_) , routing_keys(routing_keys_) , exchange_name(exchange_name_) @@ -55,11 +56,45 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( loop = std::make_unique(); uv_loop_init(loop.get()); - event_handler = std::make_unique(loop.get(), log); + + /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + setupConnection(0); + setupChannel(0); + + writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); + writing_task->deactivate(); + + if (exchange_type == AMQP::ExchangeType::headers) + { + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + key_arguments[matching[0]] = matching[1]; + } + } +} + + +WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() +{ + writing_task->deactivate(); + connection->close(); + assert(rows == 0 && chunks.empty()); +} + + +void WriteBufferToRabbitMQProducer::setupConnection(bool remove_prev_connection) +{ + if (remove_prev_connection && connection) + { + connection->close(); + connection.release(); + } + connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - /// New coonection for each publisher because cannot publish from different threads.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) size_t cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { @@ -71,8 +106,18 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( { throw Exception("Cannot set up connection for producer", ErrorCodes::CANNOT_CONNECT_RABBITMQ); } +} - producer_channel = std::make_shared(connection.get()); + +void WriteBufferToRabbitMQProducer::setupChannel(bool remove_prev_channel) +{ + if (remove_prev_channel && producer_channel) + { + producer_channel->close(); + producer_channel.release(); + } + + producer_channel = std::make_unique(connection.get()); producer_channel->onError([&](const char * message) { LOG_ERROR(log, "Prodcuer error: {}", message); @@ -84,38 +129,38 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( } else { + /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ + remove_confirmed_tag = [&](uint64_t received_delivery_tag, bool multiple) + { + std::lock_guard lock(mutex); + auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); + if (found_tag_pos != delivery_tags_record.end()) + { + if (multiple) + { + ++found_tag_pos; + delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); + } + else + delivery_tags_record.erase(found_tag_pos); + } + }; + + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it + * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is + * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. + */ producer_channel->confirmSelect() - .onAck([&](uint64_t deliveryTag, bool /* multiple */) + .onAck([&](uint64_t acked_delivery_tag, bool multiple) { - if (deliveryTag > last_processed) - last_processed = deliveryTag; + remove_confirmed_tag(acked_delivery_tag, multiple); }) - .onNack([&](uint64_t /* deliveryTag */, bool /* multiple */, bool /* requeue */) + .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) { + if (!persistent) + remove_confirmed_tag(nacked_delivery_tag, multiple); }); } - - writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); - writing_task->deactivate(); - - if (exchange_type == AMQP::ExchangeType::headers) - { - std::vector matching; - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - key_arguments[matching[0]] = matching[1]; - matching.clear(); - } - } -} - - -WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() -{ - writing_task->deactivate(); - connection->close(); - assert(rows == 0 && chunks.empty()); } @@ -143,6 +188,9 @@ void WriteBufferToRabbitMQProducer::countRow() ++delivery_tag; payloads.push(payload); + + std::lock_guard lock(mutex); + delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); } } @@ -180,7 +228,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() else if (exchange_type == AMQP::ExchangeType::headers) { envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope, key_arguments).onReturned(returned_callback); + producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); } else { @@ -191,7 +239,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() iterateEventLoop(); } - if (wait_num.load() && last_processed.load() >= wait_num.load()) + if (wait_num.load() && delivery_tags_record.empty()) { wait_all.store(false); LOG_DEBUG(log, "All messages are successfully published"); @@ -200,7 +248,22 @@ void WriteBufferToRabbitMQProducer::writingFunc() { iterateEventLoop(); } + + /// Most channel based errors result in channel closure, which is very likely to trigger connection closure. + if (connection->usable() && connection->ready() && !producer_channel->usable()) + { + LOG_DEBUG(log, "Channel is not usable. Creating a new one"); + setupChannel(1); + } + else if (!connection->usable() || !connection->ready()) + { + LOG_DEBUG(log, "Connection is not usable. Creating a new one"); + setupConnection(1); + setupChannel(1); + } } + + LOG_DEBUG(log, "Delivered messages"); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 30e647af471..188bd5676f4 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -14,13 +14,11 @@ namespace DB { -using ChannelPtr = std::shared_ptr; - class WriteBufferToRabbitMQProducer : public WriteBuffer { public: WriteBufferToRabbitMQProducer( - std::pair & parsed_address, + std::pair & parsed_address_, Context & global_context, const std::pair & login_password_, const Names & routing_keys_, @@ -46,7 +44,10 @@ private: void nextImpl() override; void iterateEventLoop(); void writingFunc(); + void setupConnection(bool remove_prev_connection); + void setupChannel(bool remove_prev_channel); + std::pair parsed_address; const std::pair login_password; const Names routing_keys; const String exchange_name; @@ -61,12 +62,15 @@ private: std::unique_ptr loop; std::unique_ptr event_handler; std::unique_ptr connection; - ChannelPtr producer_channel; + std::unique_ptr producer_channel; ConcurrentBoundedQueue payloads; UInt64 delivery_tag = 0; std::atomic wait_all = true; - std::atomic wait_num = 0, last_processed = 0; + std::atomic wait_num = 0; + std::set delivery_tags_record; + std::mutex mutex; + std::function remove_confirmed_tag; Poco::Logger * log; const std::optional delim; From 40504f6a6e9b54bdcdb0c63a5724648bf5bc04f5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 31 Jul 2020 17:57:00 +0300 Subject: [PATCH 021/535] Simpler version of #12999 w/o `pos` changes --- programs/client/Client.cpp | 126 +++++++++++++----- ...06_insert_values_and_expressions.reference | 2 + .../00306_insert_values_and_expressions.sql | 9 ++ 3 files changed, 103 insertions(+), 34 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 797342a1b44..78a6d7fe2d9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -908,74 +908,127 @@ private: return processMultiQuery(text); } - bool processMultiQuery(const String & text) + bool processMultiQuery(const String & all_queries_text) { const bool test_mode = config().has("testmode"); { /// disable logs if expects errors - TestHint test_hint(test_mode, text); + TestHint test_hint(test_mode, all_queries_text); if (test_hint.clientError() || test_hint.serverError()) processTextAsSingleQuery("SET send_logs_level = 'none'"); } /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. + /// An exception is VALUES format where we also support semicolon in + /// addition to end of line. - const char * begin = text.data(); - const char * end = begin + text.size(); + const char * this_query_begin = all_queries_text.data(); + const char * all_queries_end = all_queries_text.data() + all_queries_text.size(); - while (begin < end) + while (this_query_begin < all_queries_end) { - const char * pos = begin; - ASTPtr orig_ast = parseQuery(pos, end, true); + // Use the token iterator to skip any whitespace, semicolons and + // comments at the beginning of the query. An example from regression + // tests: + // insert into table t values ('invalid'); -- { serverError 469 } + // select 1 + // Here the test hint comment gets parsed as a part of second query. + // We parse the `INSERT VALUES` up to the semicolon, and the rest + // looks like a two-line query: + // -- { serverError 469 } + // select 1 + // and we expect it to fail with error 469, but this hint is actually + // for the previous query. Test hints should go after the query, so + // we can fix this by skipping leading comments. Token iterator skips + // comments and whitespace by itself, so we only have to check for + // semicolons. + // The code block is to limit visibility of `tokens` because we have + // another such variable further down the code, and get warnings for + // that. + { + Tokens tokens(this_query_begin, all_queries_end); + IParser::Pos token_iterator(tokens, + context.getSettingsRef().max_parser_depth); + while (token_iterator->type == TokenType::Semicolon + && token_iterator.isValid()) + { + ++token_iterator; + } + this_query_begin = token_iterator->begin; + if (this_query_begin >= all_queries_end) + { + break; + } + } - if (!orig_ast) + // Try to parse the query. + const char * this_query_end = this_query_begin; + parsed_query = parseQuery(this_query_end, all_queries_end, true); + + if (!parsed_query) { if (ignore_error) { - Tokens tokens(begin, end); + Tokens tokens(this_query_begin, all_queries_end); IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth); while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) ++token_iterator; - begin = token_iterator->end; + this_query_begin = token_iterator->end; continue; } return true; } - auto * insert = orig_ast->as(); - - if (insert && insert->data) + // INSERT queries may have the inserted data in the query text + // that follow the query itself, e.g. "insert into t format CSV 1;2". + // They need special handling. First of all, here we find where the + // inserted data ends. In multy-query mode, it is delimited by a + // newline. + // The VALUES format needs even more handling -- we also allow the + // data to be delimited by semicolon. This case is handled later by + // the format parser itself. + auto * insert_ast = parsed_query->as(); + if (insert_ast && insert_ast->data) { - pos = find_first_symbols<'\n'>(insert->data, end); - insert->end = pos; + this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end); + insert_ast->end = this_query_end; + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + insert_ast->data - this_query_begin); + } + else + { + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); } - String str = text.substr(begin - text.data(), pos - begin); + // full_query is the query + inline INSERT data. + full_query = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); - begin = pos; - while (isWhitespaceASCII(*begin) || *begin == ';') - ++begin; - - TestHint test_hint(test_mode, str); + // Look for the hint in the text of query + insert data, if any. + // e.g. insert into t format CSV 'a' -- { serverError 123 }. + TestHint test_hint(test_mode, full_query); expected_client_error = test_hint.clientError(); expected_server_error = test_hint.serverError(); try { - auto ast_to_process = orig_ast; - if (insert && insert->data) + processParsedSingleQuery(); + + if (insert_ast && insert_ast->data) { - ast_to_process = nullptr; - processTextAsSingleQuery(str); - } - else - { - parsed_query = ast_to_process; - full_query = str; - query_to_send = str; - processParsedSingleQuery(); + // For VALUES format: use the end of inline data as reported + // by the format parser (it is saved in sendData()). This + // allows us to handle queries like: + // insert into t values (1); select 1 + //, where the inline data is delimited by semicolon and not + // by a newline. + this_query_end = parsed_query->as()->end; } } catch (...) @@ -983,7 +1036,7 @@ private: last_exception_received_from_server = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); actual_client_error = last_exception_received_from_server->code(); if (!ignore_error && (!actual_client_error || actual_client_error != expected_client_error)) - std::cerr << "Error on processing query: " << str << std::endl << last_exception_received_from_server->message(); + std::cerr << "Error on processing query: " << full_query << std::endl << last_exception_received_from_server->message(); received_exception_from_server = true; } @@ -997,6 +1050,8 @@ private: else return false; } + + this_query_begin = this_query_end; } return true; @@ -1407,7 +1462,7 @@ private: void sendData(Block & sample, const ColumnsDescription & columns_description) { /// If INSERT data must be sent. - const auto * parsed_insert_query = parsed_query->as(); + auto * parsed_insert_query = parsed_query->as(); if (!parsed_insert_query) return; @@ -1416,6 +1471,9 @@ private: /// Send data contained in the query. ReadBufferFromMemory data_in(parsed_insert_query->data, parsed_insert_query->end - parsed_insert_query->data); sendDataFrom(data_in, sample, columns_description); + // Remember where the data ended. We use this info later to determine + // where the next query begins. + parsed_insert_query->end = data_in.buffer().begin() + data_in.count(); } else if (!is_interactive) { diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference index 960773dc489..e80a28accf4 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference @@ -2,3 +2,5 @@ 2 Hello, world 00000000-0000-0000-0000-000000000000 2016-01-02 2016-01-02 03:04:00 [0,1] 3 hello, world! ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-03 2016-01-02 03:00:00 [] 4 World ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-04 2016-12-11 10:09:08 [3,2,1] +11111 +1 diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql index a57e9e69fe6..10a1415f287 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql @@ -5,3 +5,12 @@ INSERT INTO insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', ' SELECT * FROM insert ORDER BY i; DROP TABLE insert; + +-- Test the case where the VALUES are delimited by semicolon and a query follows +-- w/o newline. With most formats the query in the same line would be ignored or +-- lead to an error, but VALUES are an exception and support semicolon delimiter, +-- in addition to the newline. +create table if not exists t_306 (a int) engine Memory; +insert into t_306 values (1); select 11111; +select * from t_306; +drop table if exists t_306; From c2bed351ae57c6eb69fb04154d7617e4f13a4c8b Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 Aug 2020 12:52:00 +0000 Subject: [PATCH 022/535] Add consumer connection track and restore --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 3 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 18 ++-- src/Storages/RabbitMQ/RabbitMQHandler.h | 4 +- .../ReadBufferFromRabbitMQConsumer.cpp | 76 ++++++++-------- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 7 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 86 +++++++++++++++---- src/Storages/RabbitMQ/StorageRabbitMQ.h | 6 +- 7 files changed, 131 insertions(+), 69 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 1a20699d23a..589f5b39d2e 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -52,7 +52,8 @@ void RabbitMQBlockInputStream::readPrefixImpl() if (!buffer || finished) return; - buffer->checkSubscription(); + if (!buffer->channelUsable() && (storage.connectionRunning() || storage.restoreConnection())) + buffer->restoreChannel(storage.getChannel()); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index 5d17ff23b64..ecaa109c184 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_CONNECT_RABBITMQ; -} - /* The object of this class is shared between concurrent consumers (who share the same connection == share the same * event loop and handler). */ @@ -20,19 +15,26 @@ RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : { } +///Method that is called when the connection ends up in an error state. void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * message) { + connection_running.store(false); LOG_ERROR(log, "Library error report: {}", message); - if (!connection->usable() || !connection->ready()) - throw Exception("Connection error", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + if (connection) + connection->close(); +} + +void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) +{ + connection_running.store(true); } void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); /// stop_loop variable is updated in a separate thread - while (!stop_loop.load()) + while (!stop_loop.load() && connection_running.load()) uv_run(loop, UV_RUN_NOWAIT); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 5893ace1d2f..2a992f68d27 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -17,16 +17,18 @@ class RabbitMQHandler : public AMQP::LibUvHandler public: RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_); void onError(AMQP::TcpConnection * connection, const char * message) override; + void onReady(AMQP::TcpConnection * connection) override; void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); + bool connectionRunning() { return connection_running.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false; + std::atomic stop_loop = false, connection_running = false; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 9f036a8a9b6..2c9834ae077 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -47,7 +47,16 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) bindQueue(queue_id); - consumer_channel->onReady([&]() { subscribe(); }); + consumer_channel->onReady([&]() + { + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + channel_error.store(true); + }); + + subscribe(); + }); } @@ -62,16 +71,16 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) { std::atomic bindings_created = false, bindings_error = false; - auto success_callback = [&](const std::string & queue_name_, int msgcount, int /* consumercount */) + auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) { - queues.emplace_back(queue_name_); - LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); + queues.emplace_back(queue_name); + LOG_DEBUG(log, "Queue {} is declared", queue_name); if (msgcount) - LOG_TRACE(log, "Queue " + queue_name_ + " is non-empty. Non-consumed messaged will also be delivered."); + LOG_TRACE(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); /// Binding key must be a string integer in case of hash exchange (here it is either hash or fanout). - setup_channel->bindQueue(exchange_name, queue_name_, std::to_string(channel_id)) + setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id)) .onSuccess([&] { bindings_created = true; @@ -114,22 +123,13 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) void ReadBufferFromRabbitMQConsumer::subscribe() { - count_subscribed = 0; for (const auto & queue_name : queues) { consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { - ++count_subscribed; LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); - - consumer_error = false; consumer_tag = consumer; - - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); - }); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { @@ -144,36 +144,12 @@ void ReadBufferFromRabbitMQConsumer::subscribe() }) .onError([&](const char * message) { - consumer_error = true; LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); }); } } -void ReadBufferFromRabbitMQConsumer::checkSubscription() -{ - if (count_subscribed == num_queues || !consumer_channel->usable()) - return; - - wait_subscribed = num_queues; - - /// These variables are updated in a separate thread. - while (count_subscribed != wait_subscribed && !consumer_error) - { - iterateEventLoop(); - } - - LOG_TRACE(log, "Consumer {} is subscribed to {} queues", channel_id, count_subscribed); - - /// Updated in callbacks which are run by the loop. - if (count_subscribed == num_queues) - return; - - subscribe(); -} - - void ReadBufferFromRabbitMQConsumer::ackMessages() { UInt64 delivery_tag = last_inserted_delivery_tag; @@ -209,4 +185,26 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() return false; } + +void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) +{ + if (consumer_channel->usable()) + return; + + consumer_channel = std::move(new_channel); + consumer_channel->onReady([&]() + { + LOG_TRACE(log, "Channel {} is restored", channel_id); + channel_error.store(false); + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + channel_error.store(true); + }); + + subscribe(); + }); +} + + } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 6448389aea5..d3f560fad3b 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -46,7 +46,8 @@ public: }; void allowNext() { allowed = true; } // Allow to read next message. - void checkSubscription(); + bool channelUsable() { return !channel_error.load(); } + void restoreChannel(ChannelPtr new_channel); void updateNextDeliveryTag(UInt64 delivery_tag) { last_inserted_delivery_tag = delivery_tag; } void ackMessages(); @@ -71,15 +72,13 @@ private: const std::atomic & stopped; const String deadletter_exchange; - std::atomic consumer_error = false; - std::atomic count_subscribed = 0, wait_subscribed; + std::atomic channel_error = false; String consumer_tag; ConcurrentBoundedQueue received; UInt64 last_inserted_delivery_tag = 0, prev_tag = 0; MessageData current; std::vector queues; - std::unordered_map subscribed_queue; bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f31cf3f4f72..67f3daa81ec 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -39,7 +39,7 @@ namespace DB { static const auto CONNECT_SLEEP = 200; -static const auto RETRIES_MAX = 1000; +static const auto RETRIES_MAX = 20; static const auto HEARTBEAT_RESCHEDULE_MS = 3000; namespace ErrorCodes @@ -98,7 +98,6 @@ StorageRabbitMQ::StorageRabbitMQ( { loop = std::make_unique(); uv_loop_init(loop.get()); - event_handler = std::make_shared(loop.get(), log); connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); @@ -138,16 +137,6 @@ StorageRabbitMQ::StorageRabbitMQ( exchange_type = AMQP::ExchangeType::fanout; } - if (exchange_type == AMQP::ExchangeType::headers) - { - for (const auto & header : routing_keys) - { - std::vector matching; - boost::split(matching, header, [](char c){ return c == '='; }); - bind_headers[matching[0]] = matching[1]; - } - } - auto table_id = getStorageID(); String table_name = table_id.table_name; @@ -163,7 +152,7 @@ StorageRabbitMQ::StorageRabbitMQ( void StorageRabbitMQ::heartbeatFunc() { - if (!stream_cancelled) + if (!stream_cancelled && event_handler->connectionRunning()) { LOG_TRACE(log, "Sending RabbitMQ heartbeat"); connection->heartbeat(); @@ -174,8 +163,11 @@ void StorageRabbitMQ::heartbeatFunc() void StorageRabbitMQ::loopingFunc() { - LOG_DEBUG(log, "Starting event looping iterations"); - event_handler->startLoop(); + if (event_handler->connectionRunning()) + { + LOG_DEBUG(log, "Starting event looping iterations"); + event_handler->startLoop(); + } } @@ -231,6 +223,14 @@ void StorageRabbitMQ::bindExchange() if (exchange_type == AMQP::ExchangeType::headers) { + AMQP::Table bind_headers; + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + bind_headers[matching[0]] = matching[1]; + } + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) .onSuccess([&]() { @@ -299,10 +299,66 @@ void StorageRabbitMQ::unbindExchange() event_handler->stop(); looping_task->deactivate(); + heartbeat_task->deactivate(); }); } +bool StorageRabbitMQ::restoreConnection() +{ + if (restore_connection.try_lock()) + { + /// This lock is to synchronize with getChannel(). + std::lock_guard lk(connection_mutex); + + if (!connection->usable() || !connection->ready()) + { + LOG_TRACE(log, "Trying to restore consumer connection"); + + if (!connection->closed()) + connection->close(); + + connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + + size_t cnt_retries = 0; + while (!connection->ready() && ++cnt_retries != RETRIES_MAX) + { + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } + } + + if (event_handler->connectionRunning()) + { + LOG_TRACE(log, "Connection restored"); + + heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); + looping_task->activateAndSchedule(); + } + else + { + LOG_TRACE(log, "Connection refused"); + } + + restore_connection.unlock(); + } + else + { + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } + + return event_handler->connectionRunning(); +} + + +ChannelPtr StorageRabbitMQ::getChannel() +{ + std::lock_guard lk(connection_mutex); + ChannelPtr new_channel = std::make_shared(connection.get()); + return new_channel; +} + + Pipes StorageRabbitMQ::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 9c7df1b1421..31e045ddb87 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -58,6 +58,10 @@ public: bool checkBridge() const { return !exchange_removed.load(); } void unbindExchange(); + bool connectionRunning() { return event_handler->connectionRunning(); } + bool restoreConnection(); + ChannelPtr getChannel(); + protected: StorageRabbitMQ( const StorageID & table_id_, @@ -109,11 +113,11 @@ private: String local_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; - AMQP::Table bind_headers; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; std::atomic loop_started = false, exchange_removed = false; ChannelPtr setup_channel; + std::mutex connection_mutex, restore_connection; BackgroundSchedulePool::TaskHolder streaming_task; BackgroundSchedulePool::TaskHolder heartbeat_task; From 62293f80c0a20144f260b5796ecf687fd8c3642d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 Aug 2020 05:46:57 +0000 Subject: [PATCH 023/535] Small fixes --- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 5 +- .../WriteBufferToRabbitMQProducer.cpp | 253 +++++++++--------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 10 +- 6 files changed, 139 insertions(+), 134 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 37b39bbaeae..517b6bfaf68 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -63,7 +63,7 @@ void RabbitMQBlockOutputStream::writeSuffix() if (buffer) { buffer->updateMaxWait(); - buffer->finilizeProducer(); + buffer->commit(); } } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index ecaa109c184..c7186e3d3ff 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -27,6 +27,7 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) { + LOG_TRACE(log, "Connection is ready"); connection_running.store(true); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 2c9834ae077..cb3ef43d4d3 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -128,8 +128,8 @@ void ReadBufferFromRabbitMQConsumer::subscribe() consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); consumer_tag = consumer; + LOG_TRACE(log, "Consumer {} (consumer tag: {}) is subscribed to queue {}", channel_id, consumer, queue_name); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 67f3daa81ec..1e6e22c7c6e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -109,7 +109,10 @@ StorageRabbitMQ::StorageRabbitMQ( } if (!connection->ready()) + { + uv_loop_close(loop.get()); throw Exception("Cannot set up connection for consumers", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + } rabbitmq_context.makeQueryContext(); StorageInMemoryMetadata storage_metadata; @@ -498,7 +501,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - log, num_consumers * num_queues, use_transactional_channel, persistent, + log, use_transactional_channel, persistent, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index d74e94d74d2..ee8d8cf88da 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -20,8 +20,7 @@ namespace ErrorCodes static const auto QUEUE_SIZE = 50000; static const auto CONNECT_SLEEP = 200; -static const auto RETRIES_MAX = 1000; -static const auto LOOP_WAIT = 10; +static const auto RETRIES_MAX = 20; static const auto BATCH = 10000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( @@ -32,7 +31,6 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const String & exchange_name_, const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, - size_t num_queues_, const bool use_transactional_channel_, const bool persistent_, std::optional delimiter, @@ -44,10 +42,9 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , routing_keys(routing_keys_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) - , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) , persistent(persistent_) - , payloads(QUEUE_SIZE * num_queues) + , payloads(QUEUE_SIZE) , log(log_) , delim(delimiter) , max_rows(rows_per_message) @@ -59,8 +56,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( event_handler = std::make_unique(loop.get(), log); /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) - setupConnection(0); - setupChannel(0); + setupConnection(); + setupChannel(); writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); @@ -85,85 +82,6 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() } -void WriteBufferToRabbitMQProducer::setupConnection(bool remove_prev_connection) -{ - if (remove_prev_connection && connection) - { - connection->close(); - connection.release(); - } - - connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - - size_t cnt_retries = 0; - while (!connection->ready() && ++cnt_retries != RETRIES_MAX) - { - event_handler->iterateLoop(); - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); - } - - if (!connection->ready()) - { - throw Exception("Cannot set up connection for producer", ErrorCodes::CANNOT_CONNECT_RABBITMQ); - } -} - - -void WriteBufferToRabbitMQProducer::setupChannel(bool remove_prev_channel) -{ - if (remove_prev_channel && producer_channel) - { - producer_channel->close(); - producer_channel.release(); - } - - producer_channel = std::make_unique(connection.get()); - producer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Prodcuer error: {}", message); - }); - - if (use_transactional_channel) - { - producer_channel->startTransaction(); - } - else - { - /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ - remove_confirmed_tag = [&](uint64_t received_delivery_tag, bool multiple) - { - std::lock_guard lock(mutex); - auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); - if (found_tag_pos != delivery_tags_record.end()) - { - if (multiple) - { - ++found_tag_pos; - delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); - } - else - delivery_tags_record.erase(found_tag_pos); - } - }; - - /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it - * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is - * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. - */ - producer_channel->confirmSelect() - .onAck([&](uint64_t acked_delivery_tag, bool multiple) - { - remove_confirmed_tag(acked_delivery_tag, multiple); - }) - .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) - { - if (!persistent) - remove_confirmed_tag(nacked_delivery_tag, multiple); - }); - } -} - - void WriteBufferToRabbitMQProducer::countRow() { if (++rows % max_rows == 0) @@ -195,18 +113,100 @@ void WriteBufferToRabbitMQProducer::countRow() } +bool WriteBufferToRabbitMQProducer::setupConnection() +{ + connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + + size_t cnt_retries = 0; + while (!connection->ready() && ++cnt_retries != RETRIES_MAX) + { + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } + + if (!connection->ready()) + return false; + + return true; +} + + +void WriteBufferToRabbitMQProducer::setupChannel() +{ + producer_channel = std::make_unique(connection.get()); + producer_channel->onError([&](const char * message) + { + /// Means channel ends up in an error state and is not usable anymore. + LOG_ERROR(log, "Producer error: {}", message); + producer_channel->close(); + }); + + producer_channel->onReady([&]() + { + LOG_TRACE(log, "Producer channel is ready"); + + if (use_transactional_channel) + { + producer_channel->startTransaction(); + } + else + { + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it + * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is + * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. + * Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given routing key - + * this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but it is pointless. Probably + */ + producer_channel->confirmSelect() + .onAck([&](uint64_t acked_delivery_tag, bool multiple) + { + removeConfirmed(acked_delivery_tag, multiple); + }) + .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) + { + if (!persistent) + removeConfirmed(nacked_delivery_tag, multiple); + }); + } + }); +} + + +void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag, bool multiple) +{ + /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ + std::lock_guard lock(mutex); + auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); + if (found_tag_pos != delivery_tags_record.end()) + { + /// If multiple is true, then all delivery tags up to and including current are confirmed. + if (multiple) + { + ++found_tag_pos; + delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); + LOG_TRACE(log, "Confirmed all delivery tags up to {}", received_delivery_tag); + } + else + { + delivery_tags_record.erase(found_tag_pos); + LOG_TRACE(log, "Confirmed delivery tag {}", received_delivery_tag); + } + } +} + + void WriteBufferToRabbitMQProducer::writingFunc() { String payload; UInt64 message_id = 0; - auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) + auto returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) { payloads.push(std::string(message.body(), message.size())); - //LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); + LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); }; - while ((!payloads.empty() || wait_all) && connection->usable()) + while (!payloads.empty() || wait_all) { while (!payloads.empty() && producer_channel->usable()) { @@ -242,7 +242,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() if (wait_num.load() && delivery_tags_record.empty()) { wait_all.store(false); - LOG_DEBUG(log, "All messages are successfully published"); + LOG_TRACE(log, "All messages are successfully published"); } else { @@ -252,57 +252,60 @@ void WriteBufferToRabbitMQProducer::writingFunc() /// Most channel based errors result in channel closure, which is very likely to trigger connection closure. if (connection->usable() && connection->ready() && !producer_channel->usable()) { - LOG_DEBUG(log, "Channel is not usable. Creating a new one"); - setupChannel(1); + LOG_TRACE(log, "Channel is not usable. Creating a new one"); + setupChannel(); } else if (!connection->usable() || !connection->ready()) { - LOG_DEBUG(log, "Connection is not usable. Creating a new one"); - setupConnection(1); - setupChannel(1); + LOG_TRACE(log, "Trying to restore connection"); + + if (setupConnection()) + { + LOG_TRACE(log, "Connection restored. Creating a channel"); + setupChannel(); + } + + LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} messages are waiting to be published", delivery_tags_record.size(), payloads.size()); } } - - LOG_DEBUG(log, "Delivered messages"); } -void WriteBufferToRabbitMQProducer::finilizeProducer() +void WriteBufferToRabbitMQProducer::commit() { - if (use_transactional_channel) + if (!use_transactional_channel) + return; + + std::atomic answer_received = false, wait_rollback = false; + producer_channel->commitTransaction() + .onSuccess([&]() { - std::atomic answer_received = false, wait_rollback = false; - producer_channel->commitTransaction() + answer_received = true; + wait_all.store(false); + LOG_TRACE(log, "All messages were successfully published"); + }) + .onError([&](const char * message1) + { + answer_received = true; + wait_all.store(false); + LOG_TRACE(log, "Publishing not successful: {}", message1); + + wait_rollback = true; + producer_channel->rollbackTransaction() .onSuccess([&]() { - answer_received = true; - wait_all.store(false); - LOG_TRACE(log, "All messages were successfully published"); + wait_rollback = false; }) - .onError([&](const char * message1) + .onError([&](const char * message2) { - answer_received = true; - wait_all.store(false); - wait_rollback = true; - LOG_TRACE(log, "Publishing not successful: {}", message1); - producer_channel->rollbackTransaction() - .onSuccess([&]() - { - wait_rollback = false; - }) - .onError([&](const char * message2) - { - LOG_ERROR(log, "Failed to rollback transaction: {}", message2); - wait_rollback = false; - }); + LOG_ERROR(log, "Failed to rollback transaction: {}", message2); + wait_rollback = false; }); + }); - size_t count_retries = 0; - while ((!answer_received || wait_rollback) && ++count_retries != RETRIES_MAX) - { - iterateEventLoop(); - std::this_thread::sleep_for(std::chrono::milliseconds(LOOP_WAIT)); - } + while (!answer_received || wait_rollback) + { + iterateEventLoop(); } } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 188bd5676f4..9b809c1af81 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -25,7 +25,6 @@ public: const String & exchange_name_, const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, - size_t num_queues_, const bool use_transactional_channel_, const bool persistent_, std::optional delimiter, @@ -37,22 +36,22 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } - void finilizeProducer(); + void commit(); void updateMaxWait() { wait_num.store(delivery_tag); } private: void nextImpl() override; void iterateEventLoop(); void writingFunc(); - void setupConnection(bool remove_prev_connection); - void setupChannel(bool remove_prev_channel); + bool setupConnection(); + void setupChannel(); + void removeConfirmed(UInt64 received_delivery_tag, bool multiple); std::pair parsed_address; const std::pair login_password; const Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - const size_t num_queues; const bool use_transactional_channel; const bool persistent; @@ -70,7 +69,6 @@ private: std::atomic wait_num = 0; std::set delivery_tags_record; std::mutex mutex; - std::function remove_confirmed_tag; Poco::Logger * log; const std::optional delim; From d5b1332b6717485f7b5c6ca08f454aa8dc775d86 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 2 Aug 2020 19:30:55 +0000 Subject: [PATCH 024/535] Stop publish untill batch is confirmed --- .../WriteBufferToRabbitMQProducer.cpp | 181 ++++++++++-------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 7 +- .../integration/test_storage_rabbitmq/test.py | 4 +- 3 files changed, 110 insertions(+), 82 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index ee8d8cf88da..883ee70f5d5 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -18,10 +18,9 @@ namespace ErrorCodes extern const int CANNOT_CONNECT_RABBITMQ; } -static const auto QUEUE_SIZE = 50000; static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -static const auto BATCH = 10000; +static const auto BATCH = 512; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address_, @@ -44,7 +43,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , exchange_type(exchange_type_) , use_transactional_channel(use_transactional_channel_) , persistent(persistent_) - , payloads(QUEUE_SIZE) + , payloads(BATCH) + , returned(BATCH << 6) , log(log_) , delim(delimiter) , max_rows(rows_per_message) @@ -56,8 +56,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( event_handler = std::make_unique(loop.get(), log); /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) - setupConnection(); - setupChannel(); + if (setupConnection()) + setupChannel(); writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); @@ -104,11 +104,8 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - ++delivery_tag; payloads.push(payload); - - std::lock_guard lock(mutex); - delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); + ++payload_counter; } } @@ -117,7 +114,9 @@ bool WriteBufferToRabbitMQProducer::setupConnection() { connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + LOG_TRACE(log, "Trying to set up connection"); size_t cnt_retries = 0; + while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { event_handler->iterateLoop(); @@ -136,14 +135,20 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel = std::make_unique(connection.get()); producer_channel->onError([&](const char * message) { + LOG_DEBUG(log, "Producer error: {}. Currently {} messages have not been confirmed yet, {} messages are waiting to be published", + message, delivery_tags_record.size(), payloads.size()); + /// Means channel ends up in an error state and is not usable anymore. - LOG_ERROR(log, "Producer error: {}", message); producer_channel->close(); }); producer_channel->onReady([&]() { - LOG_TRACE(log, "Producer channel is ready"); + LOG_DEBUG(log, "Producer channel is ready"); + + /// Delivery tags are scoped per channel. + delivery_tags_record.clear(); + delivery_tag = 0; if (use_transactional_channel) { @@ -151,11 +156,11 @@ void WriteBufferToRabbitMQProducer::setupChannel() } else { - /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it - * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is - * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. - * Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given routing key - - * this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but it is pointless. Probably + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, + * it will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it + * is not requeued. First option is two times slower than the second, so default is second and the first is turned on in table + * setting. Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given + * routing key - this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but pointless. */ producer_channel->confirmSelect() .onAck([&](uint64_t acked_delivery_tag, bool multiple) @@ -184,90 +189,110 @@ void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag { ++found_tag_pos; delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); - LOG_TRACE(log, "Confirmed all delivery tags up to {}", received_delivery_tag); + //LOG_DEBUG(log, "Confirmed all delivery tags up to {}", received_delivery_tag); } else { delivery_tags_record.erase(found_tag_pos); - LOG_TRACE(log, "Confirmed delivery tag {}", received_delivery_tag); + //LOG_DEBUG(log, "Confirmed delivery tag {}", received_delivery_tag); } } } -void WriteBufferToRabbitMQProducer::writingFunc() +void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages) { String payload; - UInt64 message_id = 0; - - auto returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) + while (!messages.empty()) { - payloads.push(std::string(message.body(), message.size())); + messages.pop(payload); + AMQP::Envelope envelope(payload.data(), payload.size()); + + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. + if (persistent) + envelope.setDeliveryMode(2); + + if (exchange_type == AMQP::ExchangeType::consistent_hash) + { + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope).onReturned(returned_callback); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + envelope.setHeaders(key_arguments); + producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); + } + else + { + producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); + } + + if (producer_channel->usable()) + { + ++delivery_tag; + delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); + + if (delivery_tag % BATCH == 0) + break; + } + else + { + break; + } + } + + iterateEventLoop(); +} + +/* Currently implemented “asynchronous publisher confirms” - does not stop after each publish to wait for each individual confirm. An + * asynchronous publisher may have any number of messages in-flight (unconfirmed) at a time. + * Synchronous publishing is where after each publish need to wait for the acknowledgement (ack/nack - see confirmSelect() in channel + * declaration), which is very slow because takes starting event loop and waiting for corresponding callback - can really take a while. + * + * Async publishing works well in all failure cases except for connection failure, because if connection fails - not all Ack/Nack might be + * receieved from the server (and even if all messages were successfully delivered, publisher will not be able to know it). Also in this + * case onReturned callback will not be received, so loss is possible for messages that were published but have not received confirm from + * server before connection loss, because then publisher won't know if message was delivered or not. + * + * To make it a delivery with no loss and minimal possible amount of duplicates - need to use synchronous publishing (which is too slow). + * With async publishing at-least-once delivery is achieved with (batch) publishing and manual republishing in case when not all delivery + * tags were confirmed (ack/nack) before connection loss. Here the maximum number of possible duplicates is no more than batch size. + * (Manual last batch republishing is only for case of connection loss, in all other failure cases - onReturned callback will be received.) + * + * So currently implemented async batch publishing, but for now without manual republishing (because still in doubt how to do it nicely, + * but current idea is to store in delivery_tags_record not just delivery tags, but pair: (delivery_tag, message). As currently once the + * publisher receives acknowledgement from the server that the message was sucessfully delivered - a "confirmListener" will delete its + * delivery tag from the set of pending acknowledgemens, then we can as well delete the payload. If connection fails, undeleted delivery + * tags indicate messages, whose fate is unknown, so corresponding payloads should be republished.) +*/ +void WriteBufferToRabbitMQProducer::writingFunc() +{ + returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) + { + returned.tryPush(std::string(message.body(), message.size())); LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); + + /* Here can be added a value to AMQP::Table field of AMQP::Envelope (and then it should be queue instead of + * queue) - to indicate that message was republished. Later a consumer will be able to extract this field and understand + * that this message was republished and can probably be a duplicate (as RabbitMQ does not guarantee exactly-once delivery). + */ }; while (!payloads.empty() || wait_all) { - while (!payloads.empty() && producer_channel->usable()) - { - payloads.pop(payload); - AMQP::Envelope envelope(payload.data(), payload.size()); + if (!returned.empty() && producer_channel->usable()) + publish(returned); + else if (!payloads.empty() && delivery_tags_record.empty() && producer_channel->usable()) + publish(payloads); - ++message_id; - if (wait_num) - message_id %= wait_num; + iterateEventLoop(); - /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. - if (persistent) - envelope.setDeliveryMode(2); - - if (exchange_type == AMQP::ExchangeType::consistent_hash) - { - producer_channel->publish(exchange_name, std::to_string(message_id), envelope).onReturned(returned_callback); - } - else if (exchange_type == AMQP::ExchangeType::headers) - { - envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); - } - else - { - producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); - } - - if (message_id % BATCH == 0) - iterateEventLoop(); - } - - if (wait_num.load() && delivery_tags_record.empty()) - { + if (wait_num.load() && delivery_tags_record.empty() && payloads.empty()) wait_all.store(false); - LOG_TRACE(log, "All messages are successfully published"); - } - else - { - iterateEventLoop(); - } - - /// Most channel based errors result in channel closure, which is very likely to trigger connection closure. - if (connection->usable() && connection->ready() && !producer_channel->usable()) - { - LOG_TRACE(log, "Channel is not usable. Creating a new one"); + else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection())) setupChannel(); - } - else if (!connection->usable() || !connection->ready()) - { - LOG_TRACE(log, "Trying to restore connection"); - - if (setupConnection()) - { - LOG_TRACE(log, "Connection restored. Creating a channel"); - setupChannel(); - } - - LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} messages are waiting to be published", delivery_tags_record.size(), payloads.size()); - } } + + LOG_DEBUG(log, "Processing ended"); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 9b809c1af81..d8e3db37043 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -37,7 +37,7 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } void commit(); - void updateMaxWait() { wait_num.store(delivery_tag); } + void updateMaxWait() { wait_num.store(payload_counter); } private: void nextImpl() override; @@ -46,6 +46,7 @@ private: bool setupConnection(); void setupChannel(); void removeConfirmed(UInt64 received_delivery_tag, bool multiple); + void publish(ConcurrentBoundedQueue & message); std::pair parsed_address; const std::pair login_password; @@ -63,12 +64,14 @@ private: std::unique_ptr connection; std::unique_ptr producer_channel; - ConcurrentBoundedQueue payloads; + ConcurrentBoundedQueue payloads, returned; UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; std::set delivery_tags_record; std::mutex mutex; + UInt64 payload_counter = 0; + std::function returned_callback; Poco::Logger * log; const std::optional delim; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index bc4585fb6f2..e45afa47425 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1382,8 +1382,8 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): for consumer_id in range(num_tables_to_receive + num_tables_to_ignore): instance.query(''' - DROP TABLE IF EXISTS test.direct_exchange_{0}; - DROP TABLE IF EXISTS test.direct_exchange_{0}_mv; + DROP TABLE IF EXISTS test.headers_exchange_{0}_mv; + DROP TABLE IF EXISTS test.headers_exchange_{0}; '''.format(consumer_id)) instance.query(''' From 053f31cb77235e4da3d3401f64b24cb3b4cfc413 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 4 Aug 2020 15:13:09 +0000 Subject: [PATCH 025/535] Better confirmListener --- .../WriteBufferToRabbitMQProducer.cpp | 145 ++++++++---------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 8 +- 2 files changed, 69 insertions(+), 84 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 883ee70f5d5..c2ab8e3e843 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -13,11 +13,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_CONNECT_RABBITMQ; -} - static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; static const auto BATCH = 512; @@ -133,23 +128,29 @@ bool WriteBufferToRabbitMQProducer::setupConnection() void WriteBufferToRabbitMQProducer::setupChannel() { producer_channel = std::make_unique(connection.get()); + producer_channel->onError([&](const char * message) { - LOG_DEBUG(log, "Producer error: {}. Currently {} messages have not been confirmed yet, {} messages are waiting to be published", - message, delivery_tags_record.size(), payloads.size()); + LOG_ERROR(log, "Producer error: {}", message); /// Means channel ends up in an error state and is not usable anymore. producer_channel->close(); + + for (auto record = delivery_record.begin(); record != delivery_record.end(); record++) + returned.tryPush(record->second); + + LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} waiting to be published, {} will be republished", + delivery_record.size(), payloads.size(), returned.size()); + + /// Delivery tags are scoped per channel. + delivery_record.clear(); + delivery_tag = 0; }); producer_channel->onReady([&]() { LOG_DEBUG(log, "Producer channel is ready"); - /// Delivery tags are scoped per channel. - delivery_tags_record.clear(); - delivery_tag = 0; - if (use_transactional_channel) { producer_channel->startTransaction(); @@ -157,56 +158,76 @@ void WriteBufferToRabbitMQProducer::setupChannel() else { /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, - * it will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it - * is not requeued. First option is two times slower than the second, so default is second and the first is turned on in table - * setting. Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given - * routing key - this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but pointless. + * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times + * slower than the second, so default is second and the first is turned on in table setting. + * + * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html */ producer_channel->confirmSelect() .onAck([&](uint64_t acked_delivery_tag, bool multiple) { - removeConfirmed(acked_delivery_tag, multiple); + removeConfirmed(acked_delivery_tag, multiple, false); }) .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) { - if (!persistent) - removeConfirmed(nacked_delivery_tag, multiple); + removeConfirmed(nacked_delivery_tag, multiple, true); }); } }); } -void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag, bool multiple) +void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish) { - /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ - std::lock_guard lock(mutex); - auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); - if (found_tag_pos != delivery_tags_record.end()) + auto record_iter = delivery_record.find(received_delivery_tag); + + if (record_iter != delivery_record.end()) { - /// If multiple is true, then all delivery tags up to and including current are confirmed. if (multiple) { - ++found_tag_pos; - delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); + /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). + ++record_iter; + + if (republish) + for (auto record = delivery_record.begin(); record != record_iter; ++record) + returned.tryPush(record->second); + + /// Delete the records even in case when republished because new delivery tags will be assigned by the server. + delivery_record.erase(delivery_record.begin(), record_iter); + //LOG_DEBUG(log, "Confirmed all delivery tags up to {}", received_delivery_tag); } else { - delivery_tags_record.erase(found_tag_pos); + if (republish) + returned.tryPush(record_iter->second); + + delivery_record.erase(record_iter); + //LOG_DEBUG(log, "Confirmed delivery tag {}", received_delivery_tag); } } + /// else is theoretically not possible } -void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages) +void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages, bool republishing) { String payload; - while (!messages.empty()) + while (!messages.empty() && producer_channel->usable()) { messages.pop(payload); AMQP::Envelope envelope(payload.data(), payload.size()); + AMQP::Table message_settings = key_arguments; + + /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the + * server, then it means that publisher will never now whether those messages were delivered or not, and therefore those records + * that received no ack/nack before connection loss will be republished, so there might be duplicates. To let consumer know that + * received message might be a possible duplicate - a "republished" field is added to message metadata. + */ + message_settings["republished"] = std::to_string(republishing); + + envelope.setHeaders(message_settings); /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) @@ -214,79 +235,45 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & mes if (exchange_type == AMQP::ExchangeType::consistent_hash) { - producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); } else if (exchange_type == AMQP::ExchangeType::headers) { - envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, "", envelope); } else { - producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, routing_keys[0], envelope); } - if (producer_channel->usable()) - { - ++delivery_tag; - delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); + ++delivery_tag; + delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - if (delivery_tag % BATCH == 0) - break; - } - else - { + /// Need to break to let event loop run, because no publishing actually happend before looping. + if (delivery_tag % BATCH == 0) break; - } } iterateEventLoop(); } -/* Currently implemented “asynchronous publisher confirms” - does not stop after each publish to wait for each individual confirm. An - * asynchronous publisher may have any number of messages in-flight (unconfirmed) at a time. - * Synchronous publishing is where after each publish need to wait for the acknowledgement (ack/nack - see confirmSelect() in channel - * declaration), which is very slow because takes starting event loop and waiting for corresponding callback - can really take a while. - * - * Async publishing works well in all failure cases except for connection failure, because if connection fails - not all Ack/Nack might be - * receieved from the server (and even if all messages were successfully delivered, publisher will not be able to know it). Also in this - * case onReturned callback will not be received, so loss is possible for messages that were published but have not received confirm from - * server before connection loss, because then publisher won't know if message was delivered or not. - * - * To make it a delivery with no loss and minimal possible amount of duplicates - need to use synchronous publishing (which is too slow). - * With async publishing at-least-once delivery is achieved with (batch) publishing and manual republishing in case when not all delivery - * tags were confirmed (ack/nack) before connection loss. Here the maximum number of possible duplicates is no more than batch size. - * (Manual last batch republishing is only for case of connection loss, in all other failure cases - onReturned callback will be received.) - * - * So currently implemented async batch publishing, but for now without manual republishing (because still in doubt how to do it nicely, - * but current idea is to store in delivery_tags_record not just delivery tags, but pair: (delivery_tag, message). As currently once the - * publisher receives acknowledgement from the server that the message was sucessfully delivered - a "confirmListener" will delete its - * delivery tag from the set of pending acknowledgemens, then we can as well delete the payload. If connection fails, undeleted delivery - * tags indicate messages, whose fate is unknown, so corresponding payloads should be republished.) -*/ + void WriteBufferToRabbitMQProducer::writingFunc() { - returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) - { - returned.tryPush(std::string(message.body(), message.size())); - LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); - - /* Here can be added a value to AMQP::Table field of AMQP::Envelope (and then it should be queue instead of - * queue) - to indicate that message was republished. Later a consumer will be able to extract this field and understand - * that this message was republished and can probably be a duplicate (as RabbitMQ does not guarantee exactly-once delivery). - */ - }; - while (!payloads.empty() || wait_all) { + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned.queue never grows too big + * and returned messages are republished as fast as possible. Also payloads.queue is fixed size and push attemt would block thread + * in countRow() once there is no space - that is intended. + */ if (!returned.empty() && producer_channel->usable()) - publish(returned); - else if (!payloads.empty() && delivery_tags_record.empty() && producer_channel->usable()) - publish(payloads); + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); iterateEventLoop(); - if (wait_num.load() && delivery_tags_record.empty() && payloads.empty()) + if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) wait_all.store(false); else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection())) setupChannel(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index d8e3db37043..b9378695d8d 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -45,8 +45,8 @@ private: void writingFunc(); bool setupConnection(); void setupChannel(); - void removeConfirmed(UInt64 received_delivery_tag, bool multiple); - void publish(ConcurrentBoundedQueue & message); + void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); + void publish(ConcurrentBoundedQueue & message, bool republishing); std::pair parsed_address; const std::pair login_password; @@ -68,10 +68,8 @@ private: UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; - std::set delivery_tags_record; - std::mutex mutex; UInt64 payload_counter = 0; - std::function returned_callback; + std::map delivery_record; Poco::Logger * log; const std::optional delim; From 24b032b3786f350a77f32871e6f36c6a81ca13ce Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Aug 2020 13:33:46 +0000 Subject: [PATCH 026/535] Allow multiple consumers for same queues --- .../ReadBufferFromRabbitMQConsumer.cpp | 24 ++--- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 31 ++++-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 +- .../integration/test_storage_rabbitmq/test.py | 96 +++++++++++++++++-- 4 files changed, 126 insertions(+), 29 deletions(-) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index cb3ef43d4d3..47c15df3bd3 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -79,7 +79,10 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) if (msgcount) LOG_TRACE(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); - /// Binding key must be a string integer in case of hash exchange (here it is either hash or fanout). + /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are + * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for + * fanout exchange it can be arbitrary. + */ setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id)) .onSuccess([&] { @@ -104,15 +107,11 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) queue_settings["x-dead-letter-exchange"] = deadletter_exchange; } - if (!queue_base.empty()) - { - const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); - setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - } - else - { - setup_channel->declareQueue(AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - } + /* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one + * specific queue when its name is specified in queue_base setting. + */ + const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); + setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); while (!bindings_created && !bindings_error) { @@ -128,8 +127,9 @@ void ReadBufferFromRabbitMQConsumer::subscribe() consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { - consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} (consumer tag: {}) is subscribed to queue {}", channel_id, consumer, queue_name); + if (consumer_tag.empty()) + consumer_tag = consumer; + LOG_TRACE(log, "Consumer {} is subscribed to queue {}, consumer tag {}", channel_id, queue_name, consumer); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 1e6e22c7c6e..80f66c6be0d 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -143,9 +143,28 @@ StorageRabbitMQ::StorageRabbitMQ( auto table_id = getStorageID(); String table_name = table_id.table_name; - /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name - local_exchange = exchange_name + "_" + table_name; - bridge_exchange = local_exchange + "_bridge"; + if (queue_base.empty()) + { + /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name + sharding_exchange = exchange_name + "_" + table_name; + + /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better + * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every + * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base + * for the names of later declared queue (as everything is based on names). + */ + queue_base = "queue_" + table_name; + } + else + { + /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and + * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need + * to share sharding exchange. + */ + sharding_exchange = exchange_name + queue_base; + } + + bridge_exchange = sharding_exchange + "_bridge"; /// One looping task for all consumers as they share the same connection == the same handler == the same event loop looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); @@ -203,19 +222,19 @@ void StorageRabbitMQ::initExchange() AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; - setup_channel->declareExchange(local_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) + setup_channel->declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) .onError([&](const char * message) { throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - setup_channel->bindExchange(bridge_exchange, local_exchange, routing_keys[0]) + setup_channel->bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) .onError([&](const char * message) { throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - consumer_exchange = local_exchange; + consumer_exchange = sharding_exchange; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 31e045ddb87..5aa030d821c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -95,7 +95,7 @@ private: bool hash_exchange; size_t num_queues; const bool use_transactional_channel; - const String queue_base; + String queue_base; const String deadletter_exchange; const bool persistent; @@ -111,7 +111,7 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers - String local_exchange, bridge_exchange, consumer_exchange; + String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index e45afa47425..be45298b52f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -512,8 +512,6 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq; ''') - time.sleep(1) - i = [0] messages_num = 10000 @@ -1546,7 +1544,7 @@ def test_rabbitmq_queue_resume_1(rabbitmq_cluster): ''') i = [0] - messages_num = 5000 + messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) @@ -1635,7 +1633,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): ''') i = [0] - messages_num = 5000 + messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) @@ -1689,8 +1687,6 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): if int(result1) > collected: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") - instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_queue_resume; DROP TABLE IF EXISTS test.consumer; @@ -1698,7 +1694,6 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): ''') assert int(result1) > collected, 'ClickHouse lost some messages: {}'.format(result) - assert int(result2) == 2 @pytest.mark.timeout(420) @@ -1778,8 +1773,6 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): if int(result1) >= messages_num * threads_num: break - #result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") - instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; DROP TABLE IF EXISTS test.consumer; @@ -1790,6 +1783,91 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) +@pytest.mark.timeout(420) +def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.destination; + CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) + ENGINE = MergeTree() + ORDER BY key; + ''') + + num_tables = 4 + for table_id in range(num_tables): + print("Setting up table {}".format(table_id)) + instance.query(''' + DROP TABLE IF EXISTS test.many_consumers_{0}; + DROP TABLE IF EXISTS test.many_consumers_{0}_mv; + CREATE TABLE test.many_consumers_{0} (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'many_consumers', + rabbitmq_num_queues = 2, + rabbitmq_num_consumers = 2, + rabbitmq_queue_base = 'many_consumers', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS + SELECT key, value, _consumer_tag as consumer_tag FROM test.many_consumers_{0}; + '''.format(table_id)) + + i = [0] + messages_num = 1000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + current = 0 + for message in messages: + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='many_consumers', routing_key='', + properties=pika.BasicProperties(message_id=mes_id), body=message) + connection.close() + + threads = [] + threads_num = 20 + + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + result1 = '' + while True: + result1 = instance.query('SELECT count() FROM test.destination') + time.sleep(1) + if int(result1) == messages_num * threads_num: + break + + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") + + for thread in threads: + thread.join() + + for consumer_id in range(num_tables): + instance.query(''' + DROP TABLE IF EXISTS test.many_consumers_{0}; + DROP TABLE IF EXISTS test.many_consumers_{0}_mv; + '''.format(consumer_id)) + + instance.query(''' + DROP TABLE IF EXISTS test.destination; + ''') + + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + # 4 tables, 2 consumers for each table => 8 consumer tags + assert int(result2) == 8 + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 1213161cf4201c201112cba5ac8bece9c0e6fd5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Aug 2020 20:34:13 +0000 Subject: [PATCH 027/535] Add some message properties --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 8 +-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 +- .../WriteBufferToRabbitMQProducer.cpp | 66 ++++++++++++------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 14 ++-- 4 files changed, 54 insertions(+), 37 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 80f66c6be0d..29a56934441 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -503,15 +503,11 @@ ConsumerBufferPtr StorageRabbitMQ::popReadBuffer(std::chrono::milliseconds timeo ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() { - if (update_channel_id) - next_channel_id += num_queues; - update_channel_id = true; - ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( consumer_channel, setup_channel, event_handler, consumer_exchange, - next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, + ++consumer_id, queue_base, log, row_delimiter, hash_exchange, num_queues, deadletter_exchange, stream_cancelled); } @@ -520,7 +516,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - log, use_transactional_channel, persistent, + ++producer_id, use_transactional_channel, persistent, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 5aa030d821c..8e62305fd03 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -113,8 +113,7 @@ private: String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; - size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 - bool update_channel_id = false; + size_t producer_id = 0, consumer_id = 0; std::atomic loop_started = false, exchange_removed = false; ChannelPtr setup_channel; std::mutex connection_mutex, restore_connection; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index c2ab8e3e843..429ca960378 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -24,9 +24,10 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const Names & routing_keys_, const String & exchange_name_, const AMQP::ExchangeType exchange_type_, - Poco::Logger * log_, - const bool use_transactional_channel_, + const size_t channel_id_, + const bool use_tx_, const bool persistent_, + Poco::Logger * log_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_) @@ -36,7 +37,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , routing_keys(routing_keys_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) - , use_transactional_channel(use_transactional_channel_) + , channel_id(std::to_string(channel_id_)) + , use_tx(use_tx_) , persistent(persistent_) , payloads(BATCH) , returned(BATCH << 6) @@ -50,7 +52,9 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( uv_loop_init(loop.get()); event_handler = std::make_unique(loop.get(), log); - /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + /* New coonection for each publisher because cannot publish from different threads with the same connection. + * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + */ if (setupConnection()) setupChannel(); @@ -99,15 +103,17 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - payloads.push(payload); ++payload_counter; + payloads.push(std::make_pair(payload_counter, payload)); } } bool WriteBufferToRabbitMQProducer::setupConnection() { - connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + /// Need to manually restore connection if it is lost. + connection = std::make_unique(event_handler.get(), + AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); LOG_TRACE(log, "Trying to set up connection"); size_t cnt_retries = 0; @@ -118,10 +124,7 @@ bool WriteBufferToRabbitMQProducer::setupConnection() std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); } - if (!connection->ready()) - return false; - - return true; + return connection->ready(); } @@ -133,9 +136,14 @@ void WriteBufferToRabbitMQProducer::setupChannel() { LOG_ERROR(log, "Producer error: {}", message); - /// Means channel ends up in an error state and is not usable anymore. + /* Means channel ends up in an error state and is not usable anymore. + * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) + */ producer_channel->close(); + if (use_tx) + return; + for (auto record = delivery_record.begin(); record != delivery_record.end(); record++) returned.tryPush(record->second); @@ -151,7 +159,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() { LOG_DEBUG(log, "Producer channel is ready"); - if (use_transactional_channel) + if (use_tx) { producer_channel->startTransaction(); } @@ -211,24 +219,31 @@ void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag } -void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages, bool republishing) +void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) { - String payload; + std::pair payload; while (!messages.empty() && producer_channel->usable()) { messages.pop(payload); - AMQP::Envelope envelope(payload.data(), payload.size()); + AMQP::Envelope envelope(payload.second.data(), payload.second.size()); + + /// if headers exchange - routing keys are added here via headers, else - it is just empty. AMQP::Table message_settings = key_arguments; /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the * server, then it means that publisher will never now whether those messages were delivered or not, and therefore those records - * that received no ack/nack before connection loss will be republished, so there might be duplicates. To let consumer know that - * received message might be a possible duplicate - a "republished" field is added to message metadata. + * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To + * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata. */ message_settings["republished"] = std::to_string(republishing); envelope.setHeaders(message_settings); + /* Adding here a message_id property to message metadata. + * (See https://stackoverflow.com/questions/59384305/rabbitmq-how-to-handle-unwanted-duplicate-un-ack-message-after-connection-lost) + */ + envelope.setMessageID(channel_id + "-" + std::to_string(payload.first)); + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) envelope.setDeliveryMode(2); @@ -249,7 +264,7 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & mes ++delivery_tag; delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - /// Need to break to let event loop run, because no publishing actually happend before looping. + /// Need to break at some point to let event loop run, because no publishing actually happend before looping. if (delivery_tag % BATCH == 0) break; } @@ -270,12 +285,14 @@ void WriteBufferToRabbitMQProducer::writingFunc() publish(returned, true); else if (!payloads.empty() && producer_channel->usable()) publish(payloads, false); + else if (use_tx) + break; iterateEventLoop(); if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) - wait_all.store(false); - else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection())) + wait_all = false; + else if ((!producer_channel->usable() && connection->usable()) || (!use_tx && !connection->usable() && setupConnection())) setupChannel(); } @@ -285,7 +302,12 @@ void WriteBufferToRabbitMQProducer::writingFunc() void WriteBufferToRabbitMQProducer::commit() { - if (!use_transactional_channel) + /* Actually have not yet found any information about how is it supposed work once any error occurs with a channel, because any channel + * error closes this channel and any operation on a closed channel will fail (but transaction is unique to channel). + * RabbitMQ transactions seem not trust-worthy at all - see https://www.rabbitmq.com/semantics.html. Seems like its best to always + * use "publisher confirms" rather than transactions (and by default it is so). Probably even need to delete this option. + */ + if (!use_tx || !producer_channel->usable()) return; std::atomic answer_received = false, wait_rollback = false; @@ -293,13 +315,11 @@ void WriteBufferToRabbitMQProducer::commit() .onSuccess([&]() { answer_received = true; - wait_all.store(false); LOG_TRACE(log, "All messages were successfully published"); }) .onError([&](const char * message1) { answer_received = true; - wait_all.store(false); LOG_TRACE(log, "Publishing not successful: {}", message1); wait_rollback = true; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index b9378695d8d..0773863c31a 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -24,9 +24,10 @@ public: const Names & routing_keys_, const String & exchange_name_, const AMQP::ExchangeType exchange_type_, - Poco::Logger * log_, - const bool use_transactional_channel_, + const size_t channel_id_, + const bool use_tx_, const bool persistent_, + Poco::Logger * log_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_ @@ -46,14 +47,15 @@ private: bool setupConnection(); void setupChannel(); void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); - void publish(ConcurrentBoundedQueue & message, bool republishing); + void publish(ConcurrentBoundedQueue> & message, bool republishing); std::pair parsed_address; const std::pair login_password; const Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - const bool use_transactional_channel; + const String channel_id; + const bool use_tx; const bool persistent; AMQP::Table key_arguments; @@ -64,12 +66,12 @@ private: std::unique_ptr connection; std::unique_ptr producer_channel; - ConcurrentBoundedQueue payloads, returned; + ConcurrentBoundedQueue> payloads, returned; UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; UInt64 payload_counter = 0; - std::map delivery_record; + std::map> delivery_record; Poco::Logger * log; const std::optional delim; From 70fca95a5a63e1a9ddcdab60108ff330bbbf9f16 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 7 Aug 2020 11:42:04 +0300 Subject: [PATCH 028/535] mysql/postgresql: move Dockerfiles and docker_compose to docker/test --- .../integration/mysql_golang_client}/0.reference | 0 .../integration/mysql_golang_client}/Dockerfile | 3 +++ .../test/integration/mysql_golang_client}/main.go | 0 .../test/integration/mysql_java_client}/0.reference | 0 .../test/integration/mysql_java_client}/Dockerfile | 3 +++ .../test/integration/mysql_java_client}/Test.java | 0 docker/test/integration/mysql_js_client/Dockerfile | 8 ++++++++ .../test/integration/mysql_js_client}/test.js | 0 .../test/integration/mysql_php_client}/Dockerfile | 3 +++ .../test/integration/mysql_php_client}/client.crt | 0 .../test/integration/mysql_php_client}/client.key | 0 .../test/integration/mysql_php_client}/test.php | 0 .../test/integration/mysql_php_client}/test_ssl.php | 0 .../integration/postgresql_java_client}/0.reference | 0 .../integration/postgresql_java_client}/Dockerfile | 3 +++ .../integration/postgresql_java_client}/Test.java | 0 .../runner/compose/docker_compose_mysql_client.yml | 0 .../compose/docker_compose_mysql_golang_client.yml | 4 +--- .../compose/docker_compose_mysql_java_client.yml | 4 +--- .../compose/docker_compose_mysql_js_client.yml | 4 +--- .../compose/docker_compose_mysql_php_client.yml | 3 +-- .../runner/compose/docker_compose_postgesql.yml | 0 .../docker_compose_postgesql_java_client.yml | 4 +--- .../test_mysql_protocol/clients/mysqljs/Dockerfile | 5 ----- tests/integration/test_mysql_protocol/test.py | 13 +++++++------ tests/integration/test_postgresql_protocol/test.py | 5 +++-- 26 files changed, 35 insertions(+), 27 deletions(-) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/0.reference (100%) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/Dockerfile (52%) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/main.go (100%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/0.reference (100%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/Dockerfile (84%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/Test.java (100%) create mode 100644 docker/test/integration/mysql_js_client/Dockerfile rename {tests/integration/test_mysql_protocol/clients/mysqljs => docker/test/integration/mysql_js_client}/test.js (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/Dockerfile (65%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/client.crt (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/client.key (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/test.php (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/test_ssl.php (100%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/0.reference (100%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/Dockerfile (82%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/Test.java (100%) rename tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_client.yml (100%) rename tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml (66%) rename tests/integration/test_mysql_protocol/clients/java/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml (65%) rename tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml (66%) rename tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml (66%) rename tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_postgesql.yml (100%) rename tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml (64%) delete mode 100644 tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile diff --git a/tests/integration/test_mysql_protocol/clients/golang/0.reference b/docker/test/integration/mysql_golang_client/0.reference similarity index 100% rename from tests/integration/test_mysql_protocol/clients/golang/0.reference rename to docker/test/integration/mysql_golang_client/0.reference diff --git a/tests/integration/test_mysql_protocol/clients/golang/Dockerfile b/docker/test/integration/mysql_golang_client/Dockerfile similarity index 52% rename from tests/integration/test_mysql_protocol/clients/golang/Dockerfile rename to docker/test/integration/mysql_golang_client/Dockerfile index d169c274a8b..4380383d1fb 100644 --- a/tests/integration/test_mysql_protocol/clients/golang/Dockerfile +++ b/docker/test/integration/mysql_golang_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-golang-client . +# MySQL golang client docker container + FROM golang:1.12.2 RUN go get "github.com/go-sql-driver/mysql" diff --git a/tests/integration/test_mysql_protocol/clients/golang/main.go b/docker/test/integration/mysql_golang_client/main.go similarity index 100% rename from tests/integration/test_mysql_protocol/clients/golang/main.go rename to docker/test/integration/mysql_golang_client/main.go diff --git a/tests/integration/test_mysql_protocol/clients/java/0.reference b/docker/test/integration/mysql_java_client/0.reference similarity index 100% rename from tests/integration/test_mysql_protocol/clients/java/0.reference rename to docker/test/integration/mysql_java_client/0.reference diff --git a/tests/integration/test_mysql_protocol/clients/java/Dockerfile b/docker/test/integration/mysql_java_client/Dockerfile similarity index 84% rename from tests/integration/test_mysql_protocol/clients/java/Dockerfile rename to docker/test/integration/mysql_java_client/Dockerfile index 96713a68e66..fcb6a39f33b 100644 --- a/tests/integration/test_mysql_protocol/clients/java/Dockerfile +++ b/docker/test/integration/mysql_java_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-java-client . +# MySQL Java client docker container + FROM ubuntu:18.04 RUN apt-get update && \ diff --git a/tests/integration/test_mysql_protocol/clients/java/Test.java b/docker/test/integration/mysql_java_client/Test.java similarity index 100% rename from tests/integration/test_mysql_protocol/clients/java/Test.java rename to docker/test/integration/mysql_java_client/Test.java diff --git a/docker/test/integration/mysql_js_client/Dockerfile b/docker/test/integration/mysql_js_client/Dockerfile new file mode 100644 index 00000000000..4f12de004ac --- /dev/null +++ b/docker/test/integration/mysql_js_client/Dockerfile @@ -0,0 +1,8 @@ +# docker build -t yandex/clickhouse-mysql-js-client . +# MySQL JavaScript client docker container + +FROM node:8 + +RUN npm install mysql + +COPY ./test.js test.js diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/test.js b/docker/test/integration/mysql_js_client/test.js similarity index 100% rename from tests/integration/test_mysql_protocol/clients/mysqljs/test.js rename to docker/test/integration/mysql_js_client/test.js diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile b/docker/test/integration/mysql_php_client/Dockerfile similarity index 65% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile rename to docker/test/integration/mysql_php_client/Dockerfile index 76125702076..e2ceb62f44f 100644 --- a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile +++ b/docker/test/integration/mysql_php_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-php-client . +# MySQL PHP client docker container + FROM php:7.3-cli COPY ./client.crt client.crt diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.crt b/docker/test/integration/mysql_php_client/client.crt similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.crt rename to docker/test/integration/mysql_php_client/client.crt diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.key b/docker/test/integration/mysql_php_client/client.key similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.key rename to docker/test/integration/mysql_php_client/client.key diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php b/docker/test/integration/mysql_php_client/test.php similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php rename to docker/test/integration/mysql_php_client/test.php diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php b/docker/test/integration/mysql_php_client/test_ssl.php similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php rename to docker/test/integration/mysql_php_client/test_ssl.php diff --git a/tests/integration/test_postgresql_protocol/clients/java/0.reference b/docker/test/integration/postgresql_java_client/0.reference similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/java/0.reference rename to docker/test/integration/postgresql_java_client/0.reference diff --git a/tests/integration/test_postgresql_protocol/clients/java/Dockerfile b/docker/test/integration/postgresql_java_client/Dockerfile similarity index 82% rename from tests/integration/test_postgresql_protocol/clients/java/Dockerfile rename to docker/test/integration/postgresql_java_client/Dockerfile index f08470ee805..eab236c9590 100644 --- a/tests/integration/test_postgresql_protocol/clients/java/Dockerfile +++ b/docker/test/integration/postgresql_java_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-postgresql-java-client . +# PostgreSQL Java client docker container + FROM ubuntu:18.04 RUN apt-get update && \ diff --git a/tests/integration/test_postgresql_protocol/clients/java/Test.java b/docker/test/integration/postgresql_java_client/Test.java similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/java/Test.java rename to docker/test/integration/postgresql_java_client/Test.java diff --git a/tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_client.yml similarity index 100% rename from tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_client.yml diff --git a/tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml similarity index 66% rename from tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml index 4fe6fdaeecd..34c39caa795 100644 --- a/tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: golang1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-golang-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/java/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml similarity index 65% rename from tests/integration/test_mysql_protocol/clients/java/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml index 522f404cde6..9a556ce5a8e 100644 --- a/tests/integration/test_mysql_protocol/clients/java/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: java1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-java-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml similarity index 66% rename from tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml index ebb73bc611a..11645097354 100644 --- a/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: mysqljs1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-js-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml similarity index 66% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml index c197944f375..4a0616ca2fd 100644 --- a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml @@ -1,7 +1,6 @@ version: '2.3' services: php1: - build: - context: ./ + image: yandex/clickhouse-mysql-php-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_postgesql.yml similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_postgesql.yml diff --git a/tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml similarity index 64% rename from tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml index 7094c8b2359..1b716dc514a 100644 --- a/tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml @@ -1,8 +1,6 @@ version: '2.2' services: java: - build: - context: ./ - network: host + image: yandex/clickhouse-postgresql-java-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile b/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile deleted file mode 100644 index 5381915efba..00000000000 --- a/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile +++ /dev/null @@ -1,5 +0,0 @@ -FROM node:8 - -RUN npm install mysql - -COPY ./test.js test.js diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 4640c7b6b90..342fd5b451a 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -11,10 +11,11 @@ import pymysql.connections from docker.models.containers import Container -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DOCKER_COMPOSE_PATH = get_docker_compose_path() config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) @@ -34,7 +35,7 @@ def server_address(): @pytest.fixture(scope='module') def mysql_client(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'mysql', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') @@ -60,28 +61,28 @@ def mysql_server(mysql_client): @pytest.fixture(scope='module') def golang_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'golang', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'php-mysqlnd', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'mysqljs', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'java', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java1_1') diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index d9e2dfe3228..9a7d91b8fa2 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -20,6 +20,7 @@ psycopg2.extras.register_uuid() SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) config_dir = os.path.join(SCRIPT_DIR, './configs') +DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) @@ -38,7 +39,7 @@ def server_address(): @pytest.fixture(scope='module') def psql_client(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'psql', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_psql_1') @@ -61,7 +62,7 @@ def psql_server(psql_client): @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'java', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java_1') From 5d8acc3b1e6e791d6e8ec35e789b7d0af5eb1cd8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 7 Aug 2020 11:47:56 +0300 Subject: [PATCH 029/535] Build client containers in CI --- docker/images.json | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/docker/images.json b/docker/images.json index 09114cc9710..0c4307f8e15 100644 --- a/docker/images.json +++ b/docker/images.json @@ -103,5 +103,25 @@ "docker/test/integration/helper_container": { "name": "yandex/clickhouse-integration-helper", "dependent": [] + }, + "docker/test/integration/mysql_golang_client": { + "name": "yandex/clickhouse-mysql-golang-client", + "dependent": [] + }, + "docker/test/integration/mysql_java_client": { + "name": "yandex/clickhouse-mysql-java-client", + "dependent": [] + }, + "docker/test/integration/mysql_js_client": { + "name": "yandex/clickhouse-mysql-js-client", + "dependent": [] + }, + "docker/test/integration/mysql_php_client": { + "name": "yandex/clickhouse-mysql-php-client", + "dependent": [] + }, + "docker/test/integration/postgresql_java_client": { + "name": "yandex/clickhouse-postgresql-java-client", + "dependent": [] } } From 222b06f4e9e6684db54e44abd3b7629f21a435d7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 7 Aug 2020 13:42:39 +0300 Subject: [PATCH 030/535] Add lost import --- tests/integration/test_postgresql_protocol/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 9a7d91b8fa2..47edafe757e 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -14,7 +14,7 @@ import subprocess import time import uuid -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path psycopg2.extras.register_uuid() From eff0233184491ae96fffe087b5b85afb3fe6be09 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Aug 2020 20:52:26 +0000 Subject: [PATCH 031/535] Update docs --- .../engines/table-engines/integrations/rabbitmq.md | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index e870471b4eb..41429016898 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -40,13 +40,13 @@ Required parameters: Optional parameters: -- `rabbitmq_exchange_type` – The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent-hash`. Default: `fanout`. +- `rabbitmq_exchange_type` – The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent_hash`. Default: `fanout`. - `rabbitmq_routing_key_list` – A comma-separated list of routing keys. - `rabbitmq_row_delimiter` – Delimiter character, which ends the message. - `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. - `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. - `rabbitmq_transactional_channel` – Wrap insert queries in transactions. Default: `0`. -- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. This settings should be used to be able to restore reading from declared durable queues in case of some failure when not all messages were successfully consumed. Note: it makes sence only if messages are sent with delivery mode 2 (marked 'persistent', durable). To be able to resume consumption from one specific queue in case of failure - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. +- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. - `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified. - `persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`. @@ -95,11 +95,18 @@ Exchange type options: - `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`. - `consistent-hash` - Data is evenly distributed between all bound tables (where exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. +Setting `rabbitmq_queue_base` may be used for the following cases: +- to be able to restore reading from certain durable queues when not all messages were successfully consumed. Note: it makes sence only if messages are sent with delivery mode 2 - marked 'persistent', durable. To be able to resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables. +- to reuse queues as they are declared durable and not auto-deleted. +- to let different tables share queues, so that multiple consumers could be registered for the same queues, which makes better performance. If using `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings, the exact match of queues is achieved in case these parameters are the same. + If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then: - `rabbitmq-consistent-hash-exchange` plugin must be enabled. - `message_id` property of the published messages must be specified (unique for each message/batch). +For insert query there is message metadata, which is added for each published message: messageID and republished flag - can be accessed via message headers. + Do not use the same table for inserts and materialized views. Example: @@ -116,7 +123,7 @@ Example: rabbitmq_num_consumers = 5; CREATE TABLE daily (key UInt64, value UInt64) - ENGINE = MergeTree(); + ENGINE = MergeTree() ORDER BY key; CREATE MATERIALIZED VIEW consumer TO daily AS SELECT key, value FROM queue; From 2ea32a710a0ba12ff533b3b4cf083890ccd7e136 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 Aug 2020 16:45:52 +0000 Subject: [PATCH 032/535] More tests, better reconnect --- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 11 +- src/Storages/RabbitMQ/RabbitMQHandler.h | 12 +- .../ReadBufferFromRabbitMQConsumer.cpp | 6 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 46 +++-- .../WriteBufferToRabbitMQProducer.cpp | 113 ++++++++---- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 3 +- .../integration/test_storage_rabbitmq/test.py | 168 +++++++++++++++++- 7 files changed, 295 insertions(+), 64 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index c7186e3d3ff..d6b6ab440b2 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -11,7 +11,9 @@ namespace DB RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : AMQP::LibUvHandler(loop_), loop(loop_), - log(log_) + log(log_), + connection_running(false), + loop_state(Loop::STOP) { } @@ -27,15 +29,16 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) { - LOG_TRACE(log, "Connection is ready"); connection_running.store(true); + LOG_TRACE(log, "Connection is ready"); + + loop_state.store(Loop::RUN); } void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); - /// stop_loop variable is updated in a separate thread - while (!stop_loop.load() && connection_running.load()) + while (loop_state.load() == Loop::RUN) uv_run(loop, UV_RUN_NOWAIT); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 2a992f68d27..3c0c5a2af37 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -11,6 +11,12 @@ namespace DB { +namespace Loop +{ + static const UInt8 RUN = 1; + static const UInt8 STOP = 2; +} + class RabbitMQHandler : public AMQP::LibUvHandler { @@ -19,16 +25,18 @@ public: void onError(AMQP::TcpConnection * connection, const char * message) override; void onReady(AMQP::TcpConnection * connection) override; - void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); bool connectionRunning() { return connection_running.load(); } + void updateLoopState(UInt8 state) { loop_state.store(state); } + UInt8 getLoopState() { return loop_state.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false, connection_running = false; + std::atomic connection_running; + std::atomic loop_state; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 47c15df3bd3..d12d08fad25 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -51,7 +51,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( { consumer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + LOG_ERROR(log, "Consumer {} error: {}", channel_id, message); channel_error.store(true); }); @@ -129,7 +129,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe() { if (consumer_tag.empty()) consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}, consumer tag {}", channel_id, queue_name, consumer); + LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { @@ -157,7 +157,7 @@ void ReadBufferFromRabbitMQConsumer::ackMessages() { prev_tag = delivery_tag; consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); + LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", channel_id, prev_tag); } } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 29a56934441..f0b58d3e722 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -111,7 +111,7 @@ StorageRabbitMQ::StorageRabbitMQ( if (!connection->ready()) { uv_loop_close(loop.get()); - throw Exception("Cannot set up connection for consumers", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Cannot connect to RabbitMQ", ErrorCodes::CANNOT_CONNECT_RABBITMQ); } rabbitmq_context.makeQueryContext(); @@ -161,7 +161,7 @@ StorageRabbitMQ::StorageRabbitMQ( * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need * to share sharding exchange. */ - sharding_exchange = exchange_name + queue_base; + sharding_exchange = exchange_name + "_" + queue_base; } bridge_exchange = sharding_exchange + "_bridge"; @@ -319,7 +319,7 @@ void StorageRabbitMQ::unbindExchange() event_handler->iterateLoop(); } - event_handler->stop(); + event_handler->updateLoopState(Loop::STOP); looping_task->deactivate(); heartbeat_task->deactivate(); }); @@ -335,31 +335,40 @@ bool StorageRabbitMQ::restoreConnection() if (!connection->usable() || !connection->ready()) { - LOG_TRACE(log, "Trying to restore consumer connection"); + if (event_handler->getLoopState() == Loop::RUN) + { + event_handler->updateLoopState(Loop::STOP); + looping_task->deactivate(); + heartbeat_task->deactivate(); + } + /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) + * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP + * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + */ + size_t cnt_retries = 0; + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + event_handler->iterateLoop(); + + /// This will force immediate closure if not yet closed. if (!connection->closed()) - connection->close(); + connection->close(true); + LOG_TRACE(log, "Trying to restore consumer connection"); connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - size_t cnt_retries = 0; + cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { event_handler->iterateLoop(); std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); } - } - if (event_handler->connectionRunning()) - { - LOG_TRACE(log, "Connection restored"); - - heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); - looping_task->activateAndSchedule(); - } - else - { - LOG_TRACE(log, "Connection refused"); + if (event_handler->connectionRunning()) + { + looping_task->activateAndSchedule(); + heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); + } } restore_connection.unlock(); @@ -451,8 +460,7 @@ void StorageRabbitMQ::startup() void StorageRabbitMQ::shutdown() { stream_cancelled = true; - - event_handler->stop(); + event_handler->updateLoopState(Loop::STOP); looping_task->deactivate(); streaming_task->deactivate(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 429ca960378..78920bc13c6 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -52,10 +52,10 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( uv_loop_init(loop.get()); event_handler = std::make_unique(loop.get(), log); - /* New coonection for each publisher because cannot publish from different threads with the same connection. - * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + /* New coonection for each producer buffer because cannot publish from different threads with the same connection. + * (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) */ - if (setupConnection()) + if (setupConnection(false)) setupChannel(); writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); @@ -103,21 +103,41 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - ++payload_counter; - payloads.push(std::make_pair(payload_counter, payload)); + if (!use_tx) + { + /// "publisher confirms" will be used, this is default. + ++payload_counter; + payloads.push(std::make_pair(payload_counter, payload)); + } + else + { + /// means channel->startTransaction() was called, not default, enabled only with table setting. + publish(payload); + } } } -bool WriteBufferToRabbitMQProducer::setupConnection() +bool WriteBufferToRabbitMQProducer::setupConnection(bool reconnecting) { - /// Need to manually restore connection if it is lost. + size_t cnt_retries = 0; + if (reconnecting) + { + /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) + * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP + * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + */ + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + event_handler->iterateLoop(); + if (!connection->closed()) + connection->close(true); + } + + LOG_TRACE(log, "Trying to set up connection"); connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - LOG_TRACE(log, "Trying to set up connection"); - size_t cnt_retries = 0; - + cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { event_handler->iterateLoop(); @@ -136,16 +156,12 @@ void WriteBufferToRabbitMQProducer::setupChannel() { LOG_ERROR(log, "Producer error: {}", message); - /* Means channel ends up in an error state and is not usable anymore. - * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) - */ + /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) producer_channel->close(); - if (use_tx) - return; - - for (auto record = delivery_record.begin(); record != delivery_record.end(); record++) - returned.tryPush(record->second); + /// Records that have not received ack/nack from server before channel closure. + for (const auto & record : delivery_record) + returned.tryPush(record.second); LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} waiting to be published, {} will be republished", delivery_record.size(), payloads.size(), returned.size()); @@ -240,7 +256,7 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable()) - publish(returned, true); - else if (!payloads.empty() && producer_channel->usable()) - publish(payloads, false); - else if (use_tx) - break; + /// This check is to make sure that delivery_record.size() is never bigger than returned.size() + if (delivery_record.size() < (BATCH << 6)) + { + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned.queue never grows too + * big and returned messages are republished as fast as possible. Also payloads.queue is fixed size and push attemt would + * block thread in countRow() once there is no space - that is intended. + */ + if (!returned.empty() && producer_channel->usable()) + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); + } iterateEventLoop(); if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) wait_all = false; - else if ((!producer_channel->usable() && connection->usable()) || (!use_tx && !connection->usable() && setupConnection())) + else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection(true))) setupChannel(); } @@ -300,9 +321,34 @@ void WriteBufferToRabbitMQProducer::writingFunc() } +/* This publish is for the case when transaction is delcared on the channel with channel->startTransaction(). Here only publish + * once payload is available and then commitTransaction() is called, where a needed event loop will run. + */ +void WriteBufferToRabbitMQProducer::publish(const String & payload) +{ + AMQP::Envelope envelope(payload.data(), payload.size()); + + if (persistent) + envelope.setDeliveryMode(2); + + if (exchange_type == AMQP::ExchangeType::consistent_hash) + { + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + producer_channel->publish(exchange_name, "", envelope); + } + else + { + producer_channel->publish(exchange_name, routing_keys[0], envelope); + } +} + + void WriteBufferToRabbitMQProducer::commit() { - /* Actually have not yet found any information about how is it supposed work once any error occurs with a channel, because any channel + /* Actually have not yet found any information about how is it supposed work once any error occurs with a channel, because any channel * error closes this channel and any operation on a closed channel will fail (but transaction is unique to channel). * RabbitMQ transactions seem not trust-worthy at all - see https://www.rabbitmq.com/semantics.html. Seems like its best to always * use "publisher confirms" rather than transactions (and by default it is so). Probably even need to delete this option. @@ -311,6 +357,7 @@ void WriteBufferToRabbitMQProducer::commit() return; std::atomic answer_received = false, wait_rollback = false; + producer_channel->commitTransaction() .onSuccess([&]() { @@ -320,9 +367,9 @@ void WriteBufferToRabbitMQProducer::commit() .onError([&](const char * message1) { answer_received = true; + wait_rollback = true; LOG_TRACE(log, "Publishing not successful: {}", message1); - wait_rollback = true; producer_channel->rollbackTransaction() .onSuccess([&]() { @@ -330,8 +377,8 @@ void WriteBufferToRabbitMQProducer::commit() }) .onError([&](const char * message2) { - LOG_ERROR(log, "Failed to rollback transaction: {}", message2); wait_rollback = false; + LOG_ERROR(log, "Failed to rollback transaction: {}", message2); }); }); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 0773863c31a..95d505bafd5 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -44,10 +44,11 @@ private: void nextImpl() override; void iterateEventLoop(); void writingFunc(); - bool setupConnection(); + bool setupConnection(bool reconnecting); void setupChannel(); void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); void publish(ConcurrentBoundedQueue> & message, bool republishing); + void publish(const String & payload); std::pair parsed_address; const std::pair login_password; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index be45298b52f..a670ea8ab54 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -86,6 +86,18 @@ def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.refe return TSV(result) == TSV(reference) +def kill_rabbitmq(): + p = subprocess.Popen(('docker', 'stop', rabbitmq_id), stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + +def revive_rabbitmq(): + p = subprocess.Popen(('docker', 'start', rabbitmq_id), stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + # Fixtures @pytest.fixture(scope="module") @@ -1684,7 +1696,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): while True: result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) - if int(result1) > collected: + if int(result1) == messages_num * threads_num: break instance.query(''' @@ -1693,7 +1705,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): DROP TABLE IF EXISTS test.view; ''') - assert int(result1) > collected, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) @@ -1868,6 +1880,158 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): assert int(result2) == 8 +@pytest.mark.timeout(420) +def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'consumer_reconnect', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 20 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + for thread in threads: + thread.join() + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.consumer_reconnect; + ''') + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + kill_rabbitmq(); + time.sleep(4); + revive_rabbitmq(); + + collected = int(instance.query('SELECT count() FROM test.view')) + + while True: + result = instance.query('SELECT count() FROM test.view') + time.sleep(1) + print("receiived", result, "collected", collected) + if int(result) >= messages_num * threads_num: + break + + instance.query(''' + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer_reconnect; + ''') + + # >= because at-least-once + assert int(result) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + + +@pytest.mark.timeout(420) +def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.destination; + CREATE TABLE test.destination(key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + ''') + + instance.query(''' + DROP TABLE IF EXISTS test.consume; + DROP TABLE IF EXISTS test.consume_mv; + CREATE TABLE test.consume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'producer_reconnect', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.consume_mv TO test.destination AS + SELECT key, value FROM test.consume; + ''') + + instance.query(''' + DROP TABLE IF EXISTS test.producer_reconnect; + CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'producer_reconnect', + rabbitmq_persistent_mode = '1', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + messages_num = 100000 + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ','.join(values) + + while True: + try: + instance.query("INSERT INTO test.producer_reconnect VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if 'Local: Timed out.' in str(e): + continue + else: + raise + + while int(instance.query('SELECT count() FROM test.destination')) == 0: + time.sleep(0.1) + + kill_rabbitmq(); + time.sleep(4); + revive_rabbitmq(); + + while True: + result = instance.query('SELECT count() FROM test.destination') + time.sleep(1) + print(result, messages_num) + if int(result) >= messages_num: + break + + instance.query(''' + DROP TABLE IF EXISTS test.consume_mv; + DROP TABLE IF EXISTS test.consume; + DROP TABLE IF EXISTS test.producer_reconnect; + DROP TABLE IF EXISTS test.destination; + ''') + + assert int(result) >= messages_num, 'ClickHouse lost some messages: {}'.format(result) + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 1d0e4ca6706a9002bbbe92499699f3628c101128 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 10 Aug 2020 16:35:08 +0300 Subject: [PATCH 033/535] pass tag to docker_compose --- .../docker_compose_mysql_golang_client.yml | 2 +- .../docker_compose_mysql_java_client.yml | 2 +- .../docker_compose_mysql_js_client.yml | 2 +- .../docker_compose_mysql_php_client.yml | 2 +- .../docker_compose_postgesql_java_client.yml | 2 +- .../integration/runner/dockerd-entrypoint.sh | 6 +++++ tests/integration/runner | 24 ++++++++++++++++++- 7 files changed, 34 insertions(+), 6 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml index 34c39caa795..b172cbcb2c6 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: golang1: - image: yandex/clickhouse-mysql-golang-client + image: yandex/clickhouse-mysql-golang-client:${DOCKER_MYSQL_GOLANG_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml index 9a556ce5a8e..be1b3ad3f72 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: java1: - image: yandex/clickhouse-mysql-java-client + image: yandex/clickhouse-mysql-java-client:${DOCKER_MYSQL_JAVA_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml index 11645097354..83954229111 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: mysqljs1: - image: yandex/clickhouse-mysql-js-client + image: yandex/clickhouse-mysql-js-client:${DOCKER_MYSQL_JS_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml index 4a0616ca2fd..e61cb193b0e 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: php1: - image: yandex/clickhouse-mysql-php-client + image: yandex/clickhouse-mysql-php-client:${DOCKER_MYSQL_PHP_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml index 1b716dc514a..ef18d1edd7b 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml @@ -1,6 +1,6 @@ version: '2.2' services: java: - image: yandex/clickhouse-postgresql-java-client + image: yandex/clickhouse-postgresql-java-client:${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 8b0682396f8..6c2eaba6225 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -22,5 +22,11 @@ export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge +export ${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} +export ${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} +export ${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} +export ${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} +export ${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} + cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/integration/runner b/tests/integration/runner index 6369ebeea3e..ddd4c79b127 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -124,6 +124,10 @@ if __name__ == "__main__": default="latest", help="Version of docker image which runner will use to run tests") + parser.add_argument( + "--docker-compose-images-tags", + action="append", + help="Set non-default tags for images used in docker compose recipes(yandex/my_container:my_tag)") parser.add_argument('pytest_args', nargs='*', help="args for pytest command") @@ -135,6 +139,23 @@ if __name__ == "__main__": if not args.disable_net_host: net = "--net=host" + env_tags = "" + + for img_tag in args.docker_compose_images_tags: + [image, tag] = img_tag.split(":") + if image == "yandex/clickhouse-mysql-golang-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-java-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-js-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-php-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-postgresql-java-client": + env_tags += "-e {}={}".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + else: + raise Exception("Unknown image {}".format(image)) + # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time # @@ -148,13 +169,14 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ - --volume={name}_volume:/var/lib/docker -e PYTEST_OPTS='{opts}' {img} {command}".format( + --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{opts}' {img} {command}".format( net=net, tty=tty, bin=args.binary, bridge_bin=args.bridge_binary, base_cfg=args.base_configs_dir, cases_dir=args.cases_dir, + env_tags=env_tags, opts=' '.join(args.pytest_args), img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, From dd2449354a567adca3bc36fb889e07d9260ee5fd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:28:16 +0300 Subject: [PATCH 034/535] fix --- tests/integration/runner | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index ddd4c79b127..66f6e2d797d 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -144,15 +144,15 @@ if __name__ == "__main__": for img_tag in args.docker_compose_images_tags: [image, tag] = img_tag.split(":") if image == "yandex/clickhouse-mysql-golang-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-java-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-js-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-php-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) elif image == "yandex/clickhouse-postgresql-java-client": - env_tags += "-e {}={}".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) else: raise Exception("Unknown image {}".format(image)) From e3f7fea33a6098a5f4dee4af729cbaa4c37213fc Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:31:26 +0300 Subject: [PATCH 035/535] fix --- docker/test/integration/runner/dockerd-entrypoint.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 6c2eaba6225..c38260279ed 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -22,11 +22,11 @@ export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge -export ${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} -export ${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} -export ${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} -export ${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} -export ${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} +export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} +export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} +export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} +export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} +export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" From 5d52c306c8c40974e9d1819202c1c4f891924ce2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:35:31 +0300 Subject: [PATCH 036/535] fix --- tests/integration/runner | 29 +++++++++++++++-------------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 66f6e2d797d..20737fa9a2f 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -141,20 +141,21 @@ if __name__ == "__main__": env_tags = "" - for img_tag in args.docker_compose_images_tags: - [image, tag] = img_tag.split(":") - if image == "yandex/clickhouse-mysql-golang-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-mysql-java-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-mysql-js-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-mysql-php-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-postgresql-java-client": - env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) - else: - raise Exception("Unknown image {}".format(image)) + if args.docker_compose_images_tags in not None: + for img_tag in args.docker_compose_images_tags: + [image, tag] = img_tag.split(":") + if image == "yandex/clickhouse-mysql-golang-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-java-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-js-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-php-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-postgresql-java-client": + env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + else: + raise Exception("Unknown image {}".format(image)) # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time From 75d0b8245fa207775236c443c725e6c949b30841 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:36:26 +0300 Subject: [PATCH 037/535] fix typo --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 20737fa9a2f..e5d6eabe794 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -141,7 +141,7 @@ if __name__ == "__main__": env_tags = "" - if args.docker_compose_images_tags in not None: + if args.docker_compose_images_tags is not None: for img_tag in args.docker_compose_images_tags: [image, tag] = img_tag.split(":") if image == "yandex/clickhouse-mysql-golang-client": From fdd18e540dd0a15df81c4b1f3d75ac1aa6ad1d4a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 07:06:23 +0300 Subject: [PATCH 038/535] move reference files --- .../integration/test_mysql_protocol/golang.reference | 0 .../integration/test_mysql_protocol/java.reference | 0 .../integration/test_postgresql_protocol/java.reference | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename docker/test/integration/mysql_golang_client/0.reference => tests/integration/test_mysql_protocol/golang.reference (100%) rename docker/test/integration/mysql_java_client/0.reference => tests/integration/test_mysql_protocol/java.reference (100%) rename docker/test/integration/postgresql_java_client/0.reference => tests/integration/test_postgresql_protocol/java.reference (100%) diff --git a/docker/test/integration/mysql_golang_client/0.reference b/tests/integration/test_mysql_protocol/golang.reference similarity index 100% rename from docker/test/integration/mysql_golang_client/0.reference rename to tests/integration/test_mysql_protocol/golang.reference diff --git a/docker/test/integration/mysql_java_client/0.reference b/tests/integration/test_mysql_protocol/java.reference similarity index 100% rename from docker/test/integration/mysql_java_client/0.reference rename to tests/integration/test_mysql_protocol/java.reference diff --git a/docker/test/integration/postgresql_java_client/0.reference b/tests/integration/test_postgresql_protocol/java.reference similarity index 100% rename from docker/test/integration/postgresql_java_client/0.reference rename to tests/integration/test_postgresql_protocol/java.reference From 9767d9627499faf4eb9dcb7a0a0d9575a12a1863 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 11:16:46 +0300 Subject: [PATCH 039/535] Fix names --- tests/integration/test_mysql_protocol/test.py | 4 ++-- tests/integration/test_postgresql_protocol/test.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 342fd5b451a..a31961dbd16 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -329,7 +329,7 @@ def test_python_client(server_address): def test_golang_client(server_address, golang_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR, 'clients', 'golang', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR,'golang.reference')) as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user default --password 123 --database ' @@ -386,7 +386,7 @@ def test_mysqljs_client(server_address, nodejs_container): def test_java_client(server_address, java_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR, 'clients', 'java', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() # database not exists exception. diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 47edafe757e..527c652229e 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -133,7 +133,7 @@ def test_python_client(server_address): def test_java_client(server_address, java_container): - with open(os.path.join(SCRIPT_DIR, 'clients', 'java', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() # database not exists exception. From 160776f183116bfacac1d61e51c2821cb06bbcb8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 11:55:04 +0300 Subject: [PATCH 040/535] Add explicit main_configs, user_configs and dictionaries in integration tests. --- .../integration/runner/dockerd-entrypoint.sh | 3 +- tests/integration/CMakeLists.txt | 2 +- .../helpers/0_common_enable_dictionaries.xml | 4 + tests/integration/helpers/client.py | 2 +- tests/integration/helpers/cluster.py | 165 +++++++++++------ tests/integration/helpers/dictonaries | 1 + tests/integration/helpers/test_tools.py | 3 + tests/integration/runner | 2 +- .../test_access_control_on_cluster/test.py | 6 +- .../test_adaptive_granularity/test.py | 24 +-- .../test_allowed_client_hosts/test.py | 10 +- .../test_allowed_url_from_config/test.py | 18 +- .../test_atomic_drop_table/test.py | 2 +- .../configs/conf.d/clusters.xml | 150 ++++++++-------- tests/integration/test_cluster_copier/test.py | 10 +- .../test_cluster_copier/trivial_test.py | 2 +- .../test_config_corresponding_root/test.py | 4 +- .../integration/test_custom_settings/test.py | 2 +- .../configs/disable_ssl_verification.xml | 12 ++ .../configs/enable_dictionaries.xml | 4 + .../test.py | 18 +- .../configs/enable_dictionaries.xml | 5 + .../test.py | 7 +- .../configs/allow_remote_node.xml | 5 + .../dictionaries/conflict_name_dictionary.xml | 41 +++++ .../dictionaries/lazy_load_dictionary.xml | 4 + .../configs/enable_dictionaries.xml | 3 + .../configs/user_admin.xml | 36 ++++ .../integration/test_dictionaries_ddl/test.py | 14 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_dependency_xml/test.py | 6 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_mysql/test.py | 5 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_null_value/test.py | 6 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_select_all/test.py | 6 +- .../configs/enable_dictionaries.xml | 4 + .../test.py | 5 +- .../configs/enable_dictionaries.xml | 4 + .../test_default_reading.py | 6 +- .../test_default_string.py | 6 +- .../test_dict_get.py | 5 +- .../test_dict_get_or_default.py | 5 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionary_custom_settings/test.py | 7 +- .../test_dictionary_ddl_on_cluster/test.py | 8 +- .../test_disk_types/configs/storage.xml | 16 ++ tests/integration/test_disk_types/test.py | 3 +- .../test_distributed_ddl/cluster.py | 15 +- .../configs_secure/config.d/ssl_conf.xml | 5 +- .../test_distributed_ddl_password/test.py | 12 +- .../test_distributed_format/test.py | 2 +- .../configs_secure/config.d/ssl_conf.xml | 5 +- .../test.py | 12 +- .../test.py | 2 +- .../test_enabling_access_management/test.py | 2 +- .../test_extreme_deduplication/test.py | 4 +- .../configs/dhparam.pem | 8 + .../configs/ssl_conf.xml | 6 +- .../test_https_replication/test.py | 12 +- .../test_log_family_s3/configs/minio.xml | 13 ++ .../test_log_family_s3/configs/ssl.xml | 12 ++ tests/integration/test_log_family_s3/test.py | 2 +- .../test.py | 10 +- tests/integration/test_merge_tree_s3/test.py | 4 +- .../configs/config.d/query_log.xml | 9 + .../configs/config.d/ssl_conf.xml | 12 ++ .../test_merge_tree_s3_with_cache/test.py | 4 +- tests/integration/test_multiple_disks/test.py | 6 +- .../test_mysql_database_engine/test.py | 3 +- .../test_mysql_protocol/configs/log_conf.xml | 10 ++ .../test_mysql_protocol/configs/mysql.xml | 4 + .../test_mysql_protocol/configs/ssl_conf.xml | 18 ++ tests/integration/test_mysql_protocol/test.py | 17 +- .../configs/enable_dictionaries.xml | 4 + .../configs/odbc_logging.xml | 8 + .../test_odbc_interaction/configs/openssl.xml | 12 ++ .../integration/test_odbc_interaction/test.py | 5 +- tests/integration/test_old_versions/test.py | 15 +- .../test_polymorphic_parts/test.py | 16 +- .../configs/default_passwd.xml | 13 ++ .../test_postgresql_protocol/configs/log.xml | 10 ++ .../configs/postresql.xml | 4 + .../configs/ssl_conf.xml | 18 ++ .../test_postgresql_protocol/test.py | 5 +- .../test_profile_events_s3/configs/log.xml | 10 ++ .../configs/query_log.xml | 9 + .../configs/ssl_conf.xml | 12 ++ .../test_profile_events_s3/test.py | 2 +- tests/integration/test_quorum_inserts/test.py | 9 +- tests/integration/test_quota/test.py | 170 +++++++++--------- tests/integration/test_random_inserts/test.py | 4 +- .../configs/max_table_size_to_drop.xml | 5 + .../test.py | 5 +- tests/integration/test_rename_column/test.py | 5 +- .../test_replicated_merge_tree_s3/test.py | 6 +- .../configs/users.d/another_user.xml | 13 ++ .../any_join_distinct_right_table_keys.xml | 8 + tests/integration/test_row_policy/test.py | 10 +- .../configs/config.d/ssl.xml | 12 ++ tests/integration/test_s3_with_https/test.py | 2 +- tests/integration/test_s3_with_proxy/test.py | 2 +- .../test_settings_constraints/test.py | 3 +- .../test.py | 8 +- tests/integration/test_storage_hdfs/test.py | 2 +- .../integration/test_storage_rabbitmq/test.py | 1 - .../dictionary_clickhouse_cache.xml | 4 +- .../dictionary_clickhouse_flat.xml | 4 +- tests/integration/test_system_queries/test.py | 7 +- tests/integration/test_text_log_level/test.py | 2 +- tests/integration/test_tmp_policy/test.py | 2 +- tests/integration/test_ttl_move/test.py | 8 +- .../test_user_ip_restrictions/test.py | 18 +- .../test_user_zero_database_access.py | 2 +- .../configs_secure/conf.d/ssl_conf.xml | 4 +- .../integration/test_zookeeper_config/test.py | 22 +-- 117 files changed, 922 insertions(+), 439 deletions(-) create mode 100644 tests/integration/helpers/0_common_enable_dictionaries.xml create mode 120000 tests/integration/helpers/dictonaries create mode 100644 tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml create mode 100644 tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/user_admin.xml create mode 100644 tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_disk_types/configs/storage.xml create mode 100644 tests/integration/test_https_replication/configs/dhparam.pem create mode 100644 tests/integration/test_log_family_s3/configs/minio.xml create mode 100644 tests/integration/test_log_family_s3/configs/ssl.xml create mode 100644 tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml create mode 100644 tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml create mode 100644 tests/integration/test_mysql_protocol/configs/log_conf.xml create mode 100644 tests/integration/test_mysql_protocol/configs/mysql.xml create mode 100644 tests/integration/test_mysql_protocol/configs/ssl_conf.xml create mode 100644 tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_odbc_interaction/configs/odbc_logging.xml create mode 100644 tests/integration/test_odbc_interaction/configs/openssl.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/default_passwd.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/log.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/postresql.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/ssl_conf.xml create mode 100644 tests/integration/test_profile_events_s3/configs/log.xml create mode 100644 tests/integration/test_profile_events_s3/configs/query_log.xml create mode 100644 tests/integration/test_profile_events_s3/configs/ssl_conf.xml create mode 100644 tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml create mode 100644 tests/integration/test_row_policy/configs/users.d/another_user.xml create mode 100644 tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml create mode 100644 tests/integration/test_s3_with_https/configs/config.d/ssl.xml diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index c38260279ed..9abf3bde53d 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -19,7 +19,8 @@ set -e echo "Start tests" export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse -export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config +export CLICKHOUSE_TESTS_CONFIG_DIR=/clickhouse-config +export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-base-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} diff --git a/tests/integration/CMakeLists.txt b/tests/integration/CMakeLists.txt index 8280464051f..f57ade79471 100644 --- a/tests/integration/CMakeLists.txt +++ b/tests/integration/CMakeLists.txt @@ -18,7 +18,7 @@ if(MAKE_STATIC_LIBRARIES AND DOCKER_CMD) if(NOT INTEGRATION_USE_RUNNER AND DOCKER_COMPOSE_CMD AND PYTEST_CMD) # To run one test with debug: # cmake . -DPYTEST_OPT="-ss;test_cluster_copier" - add_test(NAME integration-pytest WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/programs/server/" ${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}) + add_test(NAME integration-pytest WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/programs/server/" "CLICKHOUSE_TESTS_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/tests/config/" ${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}) message(STATUS "Using tests in docker DOCKER=${DOCKER_CMD}; DOCKER_COMPOSE=${DOCKER_COMPOSE_CMD}; PYTEST=${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}") endif() endif() diff --git a/tests/integration/helpers/0_common_enable_dictionaries.xml b/tests/integration/helpers/0_common_enable_dictionaries.xml new file mode 100644 index 00000000000..b6e52983db2 --- /dev/null +++ b/tests/integration/helpers/0_common_enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/dictionaries/*.xml + diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index 0ca6a977868..d88a21fbe46 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -71,7 +71,7 @@ class CommandRequest: self.stderr_file = tempfile.TemporaryFile() self.ignore_error = ignore_error - #print " ".join(command) + print " ".join(command) # we suppress stderror on client becase sometimes thread sanitizer # can print some debug information there diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f421f979947..69db0c0fb10 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1,25 +1,25 @@ import base64 +import cassandra.cluster import distutils.dir_util +import docker import errno +import httplib +import logging import os import os.path as p +import pprint +import psycopg2 import pwd +import pymongo +import pymysql import re +import requests import shutil import socket import subprocess import time import urllib -import httplib -import requests import xml.dom.minidom -import logging -import docker -import pprint -import psycopg2 -import pymongo -import pymysql -import cassandra.cluster from dicttoxml import dicttoxml from kazoo.client import KazooClient from kazoo.exceptions import KazooException @@ -40,6 +40,7 @@ SANITIZER_SIGN = "==================" def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): full_path = os.path.join(path, fname) with open(full_path, 'w') as f: + f.write('TSAN_OPTIONS="external_symbolizer_path=/usr/bin/llvm-symbolizer"\n') for var, value in variables.items(): f.write("=".join([var, value]) + "\n") return full_path @@ -88,12 +89,16 @@ class ClickHouseCluster: these directories will contain logs, database files, docker-compose config, ClickHouse configs etc. """ - def __init__(self, base_path, name=None, base_configs_dir=None, server_bin_path=None, client_bin_path=None, + def __init__(self, base_path, name=None, base_config_dir=None, config_dir=None, server_bin_path=None, client_bin_path=None, odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): + for param in os.environ.keys(): + print "ENV %40s %s" % (param,os.environ[param]) self.base_dir = p.dirname(base_path) self.name = name if name is not None else '' - self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', + self.base_config_dir = base_config_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', + '/etc/clickhouse-server/') + self.config_dir = config_dir or os.environ.get('CLICKHOUSE_TESTS_CONFIG_DIR', '/etc/clickhouse-server/') self.server_bin_path = p.realpath( server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) @@ -154,6 +159,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False + print "CLUSTER INIT base_config_dir:{} config_dir:{}".format(self.base_config_dir, self.config_dir) def get_client_cmd(self): cmd = self.client_bin_path @@ -161,7 +167,7 @@ class ClickHouseCluster: cmd += " client" return cmd - def add_instance(self, name, config_dir=None, main_configs=None, user_configs=None, macros=None, + def add_instance(self, name, base_config_dir=None, config_dir=None, main_configs=None, user_configs=None, dictionaries = None, macros=None, with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -172,6 +178,7 @@ class ClickHouseCluster: name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. config_dir - a directory with config files which content will be copied to /etc/clickhouse-server/ directory + base_config_dir - a directory with config.xml and users.xml files which will be copied to /etc/clickhouse-server/ directory main_configs - a list of config files that will be added to config.d/ directory user_configs - a list of config files that will be added to users.d/ directory with_zookeeper - if True, add ZooKeeper configuration to configs and ZooKeeper instances to the cluster. @@ -184,11 +191,11 @@ class ClickHouseCluster: raise Exception("Can\'t add instance `%s': there is already an instance with the same name!" % name) instance = ClickHouseInstance( - self, self.base_dir, name, config_dir, main_configs or [], user_configs or [], macros or {}, - with_zookeeper, + self, self.base_dir, name, base_config_dir if base_config_dir else self.base_config_dir, + config_dir if config_dir else self.config_dir, main_configs or [], user_configs or [], dictionaries or [], + macros or {}, with_zookeeper, self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, - self.base_configs_dir, self.server_bin_path, - self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, + self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables or {}, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, tmpfs=tmpfs or []) @@ -458,19 +465,19 @@ class ClickHouseCluster: try: minio_client.list_buckets() - logging.info("Connected to Minio.") + print("Connected to Minio.") if minio_client.bucket_exists(self.minio_bucket): minio_client.remove_bucket(self.minio_bucket) minio_client.make_bucket(self.minio_bucket) - logging.info("S3 bucket '%s' created", self.minio_bucket) + print("S3 bucket '%s' created", self.minio_bucket) self.minio_client = minio_client return except Exception as ex: - logging.warning("Can't connect to Minio: %s", str(ex)) + print("Can't connect to Minio: %s", str(ex)) time.sleep(1) raise Exception("Can't wait Minio to start") @@ -482,10 +489,10 @@ class ClickHouseCluster: try: sr_client._send_request(sr_client.url) self.schema_registry_client = sr_client - logging.info("Connected to SchemaRegistry") + print("Connected to SchemaRegistry") return except Exception as ex: - logging.warning("Can't connect to SchemaRegistry: %s", str(ex)) + print("Can't connect to SchemaRegistry: %s", str(ex)) time.sleep(1) def wait_cassandra_to_start(self, timeout=30): @@ -501,25 +508,27 @@ class ClickHouseCluster: time.sleep(1) def start(self, destroy_dirs=True): + print "Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs) if self.is_up: return # Just in case kill unstopped containers from previous launch try: - logging.info("Trying to kill unstopped containers...") + print("Trying to kill unstopped containers...") if not subprocess_call(['docker-compose', 'kill']): subprocess_call(['docker-compose', 'down', '--volumes']) - logging.info("Unstopped containers killed") + print("Unstopped containers killed") except: pass try: if destroy_dirs and p.exists(self.instances_dir): - logging.info("Removing instances dir %s", self.instances_dir) + print("Removing instances dir %s", self.instances_dir) shutil.rmtree(self.instances_dir) for instance in self.instances.values(): + print('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs)) instance.create_dir(destroy_dir=destroy_dirs) self.docker_client = docker.from_env(version=self.docker_api_version) @@ -527,6 +536,7 @@ class ClickHouseCluster: common_opts = ['up', '-d', '--force-recreate'] if self.with_zookeeper and self.base_zookeeper_cmd: + print('Setup ZooKeeper') env = os.environ.copy() if not self.zookeeper_use_tmpfs: env['ZK_FS'] = 'bind' @@ -545,14 +555,17 @@ class ClickHouseCluster: self.wait_zookeeper_to_start(120) if self.with_mysql and self.base_mysql_cmd: + print('Setup MySQL') subprocess_check_call(self.base_mysql_cmd + common_opts) self.wait_mysql_to_start(120) if self.with_postgres and self.base_postgres_cmd: + print('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(120) if self.with_kafka and self.base_kafka_cmd: + print('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.kafka_docker_id = self.get_instance_docker_id('kafka1') self.wait_schema_registry_to_start(120) @@ -562,14 +575,17 @@ class ClickHouseCluster: self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') if self.with_hdfs and self.base_hdfs_cmd: + print('Setup HDFS') subprocess_check_call(self.base_hdfs_cmd + common_opts) self.wait_hdfs_to_start(120) if self.with_mongo and self.base_mongo_cmd: + print('Setup Mongo') subprocess_check_call(self.base_mongo_cmd + common_opts) self.wait_mongo_to_start(30) if self.with_redis and self.base_redis_cmd: + print('Setup Redis') subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate']) time.sleep(10) @@ -608,18 +624,19 @@ class ClickHouseCluster: self.wait_cassandra_to_start() clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] - logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd))) + print("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd))) subprocess_check_call(clickhouse_start_cmd) - logging.info("ClickHouse instance created") + print("ClickHouse instance created") + start_deadline = time.time() + 20.0 # seconds for instance in self.instances.itervalues(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) - logging.info("Waiting for ClickHouse start...") + print("Waiting for ClickHouse start...") instance.wait_for_start(start_deadline) - logging.info("ClickHouse started") + print("ClickHouse started") instance.client = Client(instance.ip_address, command=self.client_bin_path) @@ -633,7 +650,10 @@ class ClickHouseCluster: def shutdown(self, kill=True): sanitizer_assert_instance = None with open(self.docker_logs_path, "w+") as f: - subprocess.check_call(self.base_cmd + ['logs'], stdout=f) + try: + subprocess.check_call(self.base_cmd + ['logs'], stdout=f) + except Exception as e: + print "Unable to get logs from docker." f.seek(0) for line in f: if SANITIZER_SIGN in line: @@ -641,8 +661,15 @@ class ClickHouseCluster: break if kill: - subprocess_check_call(self.base_cmd + ['kill']) - subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + try: + subprocess_check_call(self.base_cmd + ['kill']) + except Exception as e: + print "Kill command failed durung shutdown. {}".format(repr(e)) + + try: + subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + except Exception as e: + print "Down + remove orphans failed durung shutdown. {}".format(repr(e)) self.is_up = False @@ -707,7 +734,7 @@ services: image: {image} hostname: {hostname} volumes: - - {configs_dir}:/etc/clickhouse-server/ + - {instance_config_dir}:/etc/clickhouse-server/ - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ {binary_volume} @@ -723,6 +750,9 @@ services: - {env_file} security_opt: - label:disable + dns_opt: + - timeout:1 + - attempts:3 {networks} {app_net} {ipv4_address} @@ -735,9 +765,9 @@ services: class ClickHouseInstance: def __init__( - self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, - with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, - base_configs_dir, server_bin_path, odbc_bridge_bin_path, + self, cluster, base_path, name, base_config_dir, config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, + macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, + with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None): @@ -749,15 +779,16 @@ class ClickHouseInstance: self.hostname = hostname if hostname is not None else self.name self.tmpfs = tmpfs or [] - self.custom_config_dir = p.abspath(p.join(base_path, custom_config_dir)) if custom_config_dir else None + self.base_config_dir = p.abspath(p.join(base_path, base_config_dir)) if base_config_dir else None + self.config_dir = p.abspath(p.join(base_path, config_dir)) if config_dir else None self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs] self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] + self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries] self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None self.macros = macros if macros is not None else {} self.with_zookeeper = with_zookeeper self.zookeeper_config_path = zookeeper_config_path - self.base_configs_dir = base_configs_dir self.server_bin_path = server_bin_path self.odbc_bridge_bin_path = odbc_bridge_bin_path @@ -773,7 +804,7 @@ class ClickHouseInstance: self.docker_compose_path = p.join(self.path, 'docker_compose.yml') self.env_variables = env_variables or {} if with_odbc_drivers: - self.odbc_ini_path = os.path.dirname(self.docker_compose_path) + "/odbc.ini:/etc/odbc.ini" + self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini" self.with_mysql = True else: self.odbc_ini_path = "" @@ -975,7 +1006,7 @@ class ClickHouseInstance: time_left = deadline - current_time if deadline is not None and current_time >= deadline: raise Exception("Timed out while waiting for instance `{}' with ip address {} to start. " - "Container status: {}".format(self.name, self.ip_address, status)) + "Container status: {}, logs: {}".format(self.name, self.ip_address, status, handle.logs())) # Repeatedly poll the instance address until there is something that listens there. # Usually it means that ClickHouse is ready to accept queries. @@ -1057,40 +1088,50 @@ class ClickHouseInstance: os.makedirs(self.path) - configs_dir = p.abspath(p.join(self.path, 'configs')) - os.mkdir(configs_dir) + instance_config_dir = p.abspath(p.join(self.path, 'configs')) + os.makedirs(instance_config_dir) - shutil.copy(p.join(self.base_configs_dir, 'config.xml'), configs_dir) - shutil.copy(p.join(self.base_configs_dir, 'users.xml'), configs_dir) + print "Copy common default production configuration from {}".format(self.base_config_dir) + shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml')) + shutil.copyfile(p.join(self.base_config_dir, 'users.xml'), p.join(instance_config_dir, 'users.xml')) + print "Create directory for configuration generated in this helper" # used by all utils with any config - conf_d_dir = p.abspath(p.join(configs_dir, 'conf.d')) - # used by server with main config.xml - self.config_d_dir = p.abspath(p.join(configs_dir, 'config.d')) - users_d_dir = p.abspath(p.join(configs_dir, 'users.d')) + conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d')) os.mkdir(conf_d_dir) - os.mkdir(self.config_d_dir) - os.mkdir(users_d_dir) + print "Create directory for common tests configuration" + # used by server with main config.xml + self.config_d_dir = p.abspath(p.join(instance_config_dir, 'config.d')) + os.mkdir(self.config_d_dir) + users_d_dir = p.abspath(p.join(instance_config_dir, 'users.d')) + os.mkdir(users_d_dir) + dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) + os.mkdir(dictionaries_dir) + + print "Copy common configuration from helpers" # The file is named with 0_ prefix to be processed before other configuration overloads. shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir) shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir) + if len(self.custom_dictionaries_paths): + shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir) - # Generate and write macros file + print "Generate and write macros file" macros = self.macros.copy() macros['instance'] = self.name - with open(p.join(self.config_d_dir, 'macros.xml'), 'w') as macros_config: + with open(p.join(conf_d_dir, 'macros.xml'), 'w') as macros_config: macros_config.write(self.dict_to_xml({"macros": macros})) # Put ZooKeeper config if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - # Copy config dir - if self.custom_config_dir: - distutils.dir_util.copy_tree(self.custom_config_dir, configs_dir) + # print "Copy config dir {} to {}".format(self.config_dir, instance_config_dir) + # if self.config_dir: + # distutils.dir_util.copy_tree(self.config_dir, instance_config_dir) # Copy config.d configs + print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) for path in self.custom_main_config_paths: shutil.copy(path, self.config_d_dir) @@ -1098,12 +1139,21 @@ class ClickHouseInstance: for path in self.custom_user_config_paths: shutil.copy(path, users_d_dir) + + self.config_dir + # Copy dictionaries configs to configs/dictionaries + for path in self.custom_dictionaries_paths: + shutil.copy(path, dictionaries_dir) + db_dir = p.abspath(p.join(self.path, 'database')) + print "Setup database dir {}".format(db_dir) os.mkdir(db_dir) if self.clickhouse_path_dir is not None: + print "Database files taken from {}".format(self.clickhouse_path_dir) distutils.dir_util.copy_tree(self.clickhouse_path_dir, db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) + print "Setup logs dir {}".format(logs_dir) os.mkdir(logs_dir) depends_on = [] @@ -1128,6 +1178,8 @@ class ClickHouseInstance: env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) + print "Env {} stored in {}".format(self.env_variables, env_file) + odbc_ini_path = "" if self.odbc_ini_path: self._create_odbc_config_file() @@ -1138,6 +1190,8 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND + print "Entrypoint cmd: {}".format(entrypoint_cmd) + networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" if self.ipv4_address is not None or self.ipv6_address is not None or self.hostname != self.name: networks = "networks:" @@ -1157,6 +1211,7 @@ class ClickHouseInstance: binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh" odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh" + with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, @@ -1164,7 +1219,7 @@ class ClickHouseInstance: hostname=self.hostname, binary_volume=binary_volume, odbc_bridge_volume=odbc_bridge_volume, - configs_dir=configs_dir, + instance_config_dir=instance_config_dir, config_d_dir=self.config_d_dir, db_dir=db_dir, tmpfs=str(self.tmpfs), diff --git a/tests/integration/helpers/dictonaries b/tests/integration/helpers/dictonaries new file mode 120000 index 00000000000..b33ab3b1e87 --- /dev/null +++ b/tests/integration/helpers/dictonaries @@ -0,0 +1 @@ +../../config/dict_examples/ \ No newline at end of file diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index 93265d280df..67ca025c58a 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -11,6 +11,9 @@ class TSV: raw_lines = contents.splitlines(True) elif isinstance(contents, list): raw_lines = ['\t'.join(map(str, l)) if isinstance(l, list) else str(l) for l in contents] + elif isinstance(contents, TSV): + self.lines = contents.lines + return else: raise TypeError("contents must be either file or string or list, actual type: " + type(contents).__name__) self.lines = [l.strip() for l in raw_lines if l.strip()] diff --git a/tests/integration/runner b/tests/integration/runner index e5d6eabe794..0b084d88f9a 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -53,7 +53,7 @@ def check_args_and_update_paths(args): logging.info("base_configs_dir: {}, binary: {}, cases_dir: {} ".format(args.base_configs_dir, args.binary, args.cases_dir)) - for path in [args.binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: + for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: if not os.path.exists(path): raise Exception("Path {} doesn't exist".format(path)) diff --git a/tests/integration/test_access_control_on_cluster/test.py b/tests/integration/test_access_control_on_cluster/test.py index 07c72e94be0..9f053afb607 100644 --- a/tests/integration/test_access_control_on_cluster/test.py +++ b/tests/integration/test_access_control_on_cluster/test.py @@ -4,9 +4,9 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True) -ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True) -ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True) +ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True) +ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True) +ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True) @pytest.fixture(scope="module", autouse=True) def started_cluster(): diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 671cb5a672b..9feef62f799 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -9,23 +9,23 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', with_installed_binary=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', with_installed_binary=True) +node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', with_installed_binary=True) -node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', with_installed_binary=True) +node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node7 = cluster.add_instance('node7', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) -node8 = cluster.add_instance('node8', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node7 = cluster.add_instance('node7', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) +node8 = cluster.add_instance('node8', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) -node9 = cluster.add_instance('node9', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) -node10 = cluster.add_instance('node10', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) +node9 = cluster.add_instance('node9', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node10 = cluster.add_instance('node10', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) -node11 = cluster.add_instance('node11', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) -node12 = cluster.add_instance('node12', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node11 = cluster.add_instance('node11', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node12 = cluster.add_instance('node12', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) def prepare_single_pair_with_setting(first_node, second_node, group): diff --git a/tests/integration/test_allowed_client_hosts/test.py b/tests/integration/test_allowed_client_hosts/test.py index 23f7f0a4abd..f187b6d889c 100644 --- a/tests/integration/test_allowed_client_hosts/test.py +++ b/tests/integration/test_allowed_client_hosts/test.py @@ -4,7 +4,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -server = cluster.add_instance('server', config_dir="configs") +server = cluster.add_instance('server', user_configs=["configs/users.d/network.xml"]) clientA1 = cluster.add_instance('clientA1', hostname = 'clientA1.com') clientA2 = cluster.add_instance('clientA2', hostname = 'clientA2.com') @@ -20,7 +20,12 @@ clientD2 = cluster.add_instance('clientD2', hostname = 'xxx.clientD0002.ru') clientD3 = cluster.add_instance('clientD3', hostname = 'clientD0003.ru') +def check_clickhouse_is_ok(client_node, server_node): + assert client_node.exec_in_container(["bash", "-c", "/usr/bin/curl -s {}:8123 ".format(server_node.hostname)]) == "Ok.\n" + + def query_from_one_node_to_another(client_node, server_node, query): + check_clickhouse_is_ok(client_node, server_node) return client_node.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host {} --query {!r}".format(server_node.hostname, query)]) @@ -56,5 +61,6 @@ def test_allowed_host(): for client_node in expected_to_fail: with pytest.raises(Exception) as e: - query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") + result = query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") + print("Client node: {} Server node: {} Result: {}".format(client_node, server_node, result)) assert "default: Authentication failed" in str(e) diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 688f94cb058..2a666e4e2ec 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -40,7 +40,7 @@ def test_config_with_only_regexp_hosts(start_cluster): assert node3.query("CREATE TABLE table_test_3_1 (word String) Engine=URL('https://host:80', HDFS)") == "" assert node3.query("CREATE TABLE table_test_3_2 (word String) Engine=URL('https://yandex.ru', CSV)") == "" assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_3 (word String) Engine=URL('https://host', CSV)") - assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)") + assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)") def test_config_without_allowed_hosts(start_cluster): assert node4.query("CREATE TABLE table_test_4_1 (word String) Engine=URL('https://host:80', CSV)") == "" @@ -49,18 +49,18 @@ def test_config_without_allowed_hosts(start_cluster): assert node4.query("CREATE TABLE table_test_4_4 (word String) Engine=URL('ftp://something.com', S3)") == "" def test_table_function_remote(start_cluster): + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-1', system, events", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) assert node6.query("SELECT * FROM remote('localhost', system, events)") != "" assert node6.query("SELECT * FROM remoteSecure('localhost', system, metrics)") != "" assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remoteSecure('localhost:800', system, events)") assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remote('localhost:800', system, metrics)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-1', system, events") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)") - assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)") - assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)") - assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)") def test_redirect(start_cluster): hdfs_api = HDFSApi("root") diff --git a/tests/integration/test_atomic_drop_table/test.py b/tests/integration/test_atomic_drop_table/test.py index 279d13ac4da..ee79a3ff080 100644 --- a/tests/integration/test_atomic_drop_table/test.py +++ b/tests/integration/test_atomic_drop_table/test.py @@ -6,7 +6,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=["configs/config.d/zookeeper_session_timeout.xml", "configs/remote_servers.xml"], with_zookeeper=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml index 54a8822fa98..632ab84d6a2 100644 --- a/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml +++ b/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml @@ -1,80 +1,74 @@ + - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - - true - - s0_1_0 - 9000 - - - - - - - true - - s1_0_0 - 9000 - - - s1_0_1 - 9000 - - - - true - - s1_1_0 - 9000 - - - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - - - - - - s0_0_0 - 9000 - - - - - - - - - s1_0_0 - 9000 - - - - - + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + + true + + s0_1_0 + 9000 + + + + + + true + + s1_0_0 + 9000 + + + s1_0_1 + 9000 + + + + true + + s1_1_0 + 9000 + + + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + + + + + s0_0_0 + 9000 + + + + + + + s1_0_0 + 9000 + + + + diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 983cac596dc..3f9ca8a053c 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -54,7 +54,8 @@ def started_cluster(): for replica_name in replicas: name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) cluster.add_instance(name, - config_dir="configs", + main_configs=["configs/conf.d/query_log.xml", "configs/conf.d/ddl.xml", "configs/conf.d/clusters.xml"], + user_configs=["configs/users.xml"], macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, with_zookeeper=True) @@ -226,6 +227,7 @@ def execute_task(task, cmd_options): zk.ensure_path(zk_task_path) zk.create(zk_task_path + "/description", task.copier_task_config) + # Run cluster-copier processes on each node docker_api = docker.from_env().api copiers_exec_ids = [] @@ -241,9 +243,11 @@ def execute_task(task, cmd_options): for instance_name in copiers: instance = cluster.instances[instance_name] container = instance.get_docker_handle() + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") + print "Copied copier config to {}".format(instance.name) exec_id = docker_api.exec_create(container.id, cmd, stderr=True) - docker_api.exec_start(exec_id, detach=True) - + output = docker_api.exec_start(exec_id).decode('utf8') + print(output) copiers_exec_ids.append(exec_id) print "Copier for {} ({}) has started".format(instance.name, instance.ip_address) diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 70c66653cb2..1697f8bbdfa 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -34,7 +34,7 @@ def started_cluster(): for replica_name in replicas: name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) cluster.add_instance(name, - config_dir="configs", + main_configs=[], user_configs=[], macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, with_zookeeper=True) diff --git a/tests/integration/test_config_corresponding_root/test.py b/tests/integration/test_config_corresponding_root/test.py index fd5d3eae3ff..1c714654820 100644 --- a/tests/integration/test_config_corresponding_root/test.py +++ b/tests/integration/test_config_corresponding_root/test.py @@ -4,10 +4,9 @@ import pytest from helpers.cluster import ClickHouseCluster SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir = config_dir) +node = cluster.add_instance('node', main_configs=["configs/config.d/bad.xml"]) caught_exception = "" @pytest.fixture(scope="module") @@ -19,4 +18,5 @@ def start_cluster(): caught_exception = str(e) def test_work(start_cluster): + print(caught_exception) assert caught_exception.find("Root element doesn't have the corresponding root element as the config file.") != -1 diff --git a/tests/integration/test_custom_settings/test.py b/tests/integration/test_custom_settings/test.py index 444a4d21881..62c765a6ba0 100644 --- a/tests/integration/test_custom_settings/test.py +++ b/tests/integration/test_custom_settings/test.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir='configs') +node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"], user_configs=["configs/users.d/custom_settings.xml"]) @pytest.fixture(scope="module", autouse=True) diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml new file mode 100644 index 00000000000..dc9958934d2 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + \ No newline at end of file diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py index f4b0ba9c1e4..4c35f9725a8 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py @@ -181,12 +181,18 @@ def setup_module(module): if not (field.is_key or field.is_range or field.is_range_key): DICTIONARIES_KV.append(get_dict(source, layout, field_keys + [field], field.name)) + cluster = ClickHouseCluster(__file__) + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + cluster.add_instance('clickhouse1', main_configs=main_configs) + + dictionaries = [] for fname in os.listdir(dict_configs_path): - main_configs.append(os.path.join(dict_configs_path, fname)) - cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) - node = cluster.add_instance('node', main_configs=main_configs, with_mysql=True, with_mongo=True, with_redis=True, with_cassandra=True) - cluster.add_instance('clickhouse1') + dictionaries.append(os.path.join(dict_configs_path, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True, with_mongo=True, with_redis=True, with_cassandra=True) @pytest.fixture(scope="module") @@ -238,8 +244,8 @@ def remove_mysql_dicts(): TODO remove this when open ssl will be fixed or thread sanitizer will be suppressed """ - global DICTIONARIES - DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")] + #global DICTIONARIES + #DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")] @pytest.mark.parametrize("fold", list(range(10))) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..46d148ad9b9 --- /dev/null +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml @@ -0,0 +1,5 @@ + + + /etc/clickhouse-server/config.d/complex_key_cache_string.xml + /etc/clickhouse-server/config.d/ssd_complex_key_cache_string.xml + diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index 2a62d66a5f8..8c676841f16 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -5,13 +5,12 @@ from helpers.cluster import ClickHouseCluster @pytest.fixture(scope="function") def cluster(request): SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) - + cluster = ClickHouseCluster(__file__) try: if request.param == "memory": - node = cluster.add_instance('node', main_configs=['configs/dictionaries/complex_key_cache_string.xml']) + node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/complex_key_cache_string.xml']) if request.param == "ssd": - node = cluster.add_instance('node', main_configs=['configs/dictionaries/ssd_complex_key_cache_string.xml']) + node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/ssd_complex_key_cache_string.xml']) cluster.start() node.query("create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") diff --git a/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml b/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml new file mode 100644 index 00000000000..5e616865fef --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml @@ -0,0 +1,5 @@ + + + node1 + + diff --git a/tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml b/tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml new file mode 100644 index 00000000000..75e6f8953eb --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml @@ -0,0 +1,41 @@ + + + test.conflicting_dictionary + + + localhost + 9000 + default + + test + xml_dictionary_table
+
+ + + + 0 + 0 + + + + 128 + + + + + id + + + SomeValue1 + UInt8 + 1 + + + + SomeValue2 + String + '' + + +
+
diff --git a/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml b/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml new file mode 100644 index 00000000000..d01f7a0155b --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml @@ -0,0 +1,4 @@ + + false + + diff --git a/tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..ddb049db2a4 --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml @@ -0,0 +1,3 @@ + + /etc/clickhouse-server/config.d/*dictionary.xml + diff --git a/tests/integration/test_dictionaries_ddl/configs/user_admin.xml b/tests/integration/test_dictionaries_ddl/configs/user_admin.xml new file mode 100644 index 00000000000..3e53e05aee1 --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/user_admin.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + ::/0 + + default + default + + default + test + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 220aeb6998a..ff252401928 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -7,10 +7,10 @@ import warnings SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml']) -node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml']) -node3 = cluster.add_instance('node3', main_configs=['configs/dictionaries/dictionary_with_conflict_name.xml']) +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/simple_dictionary.xml'], user_configs=['configs/user_admin.xml']) +node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/allow_remote_node.xml','configs/enable_dictionaries.xml','configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load_dictionary.xml'], user_configs=['configs/user_admin.xml']) +node3 = cluster.add_instance('node3', main_configs=['configs/allow_remote_node.xml','configs/enable_dictionaries.xml','configs/dictionaries/conflict_name_dictionary.xml'], user_configs=['configs/user_admin.xml']) def create_mysql_conn(user, password, hostname, port): @@ -49,7 +49,7 @@ def started_cluster(): (node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), ]) -def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout): +def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(name)) @@ -93,8 +93,8 @@ def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout): for i in range(172, 200): assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 3 + '\n' - stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip() - value = float(stroka) + string = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip() + value = float(string) assert int(value) == int(i * 2.718) clickhouse.query("select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))") == "17\n" diff --git a/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..89a4c99ef7a --- /dev/null +++ b/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/dep_*.xml + diff --git a/tests/integration/test_dictionaries_dependency_xml/test.py b/tests/integration/test_dictionaries_dependency_xml/test.py index c0ce0af0313..da1146cd54c 100644 --- a/tests/integration/test_dictionaries_dependency_xml/test.py +++ b/tests/integration/test_dictionaries_dependency_xml/test.py @@ -3,11 +3,11 @@ import os from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml'] -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES,) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..76ed6af89ba --- /dev/null +++ b/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/mysql_dict*.xml + diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 647e36c71b3..4d2a063e91d 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -8,10 +8,9 @@ import pymysql.cursors from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml'] - -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +CONFIG_FILES += ['configs/enable_dictionaries.xml'] +cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql = True) create_table_mysql_template = """ diff --git a/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionaries_null_value/test.py b/tests/integration/test_dictionaries_null_value/test.py index bb840d8f8f7..c4ad3782498 100644 --- a/tests/integration/test_dictionaries_null_value/test.py +++ b/tests/integration/test_dictionaries_null_value/test.py @@ -3,11 +3,11 @@ import os from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV, assert_eq_with_retry -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache.xml'] -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..fa26ed7ec3d --- /dev/null +++ b/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/dictionary*.xml + diff --git a/tests/integration/test_dictionaries_select_all/test.py b/tests/integration/test_dictionaries_select_all/test.py index 8bad8a9b214..7dc93b2df44 100644 --- a/tests/integration/test_dictionaries_select_all/test.py +++ b/tests/integration/test_dictionaries_select_all/test.py @@ -19,12 +19,12 @@ def setup_module(module): structure = generate_structure() dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure) - cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) - instance = cluster.add_instance('instance', main_configs=dictionary_files) + cluster = ClickHouseCluster(__file__) + instance = cluster.add_instance('instance', main_configs=dictionary_files+['configs/enable_dictionaries.xml']) test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv')) -@pytest.fixture(scope="module") +@pytest.fixture(scope="module", autouse=True) def started_cluster(): try: cluster.start() diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 5e5c6514dd2..762fd3adc28 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -6,10 +6,11 @@ from helpers.client import QueryTimeoutExceedException from helpers.test_tools import assert_eq_with_retry SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache_xypairs.xml', 'configs/dictionaries/executable.xml', 'configs/dictionaries/file.xml', 'configs/dictionaries/file.txt', 'configs/dictionaries/slow.xml'] -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml b/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 8da882679bd..b6b742c1de8 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -8,11 +8,11 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', + 'configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index 7d762db2a6d..d6517379086 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -9,10 +9,10 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_strings_default_settings.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/cache_ints_dictionary.xml','configs/dictionaries/cache_strings_default_settings.xml']) def get_random_string(string_length=8): @@ -26,7 +26,7 @@ def started_cluster(): dictionary_node.query("CREATE DATABASE IF NOT EXISTS test;") dictionary_node.query("DROP TABLE IF EXISTS test.strings;") dictionary_node.query(""" - CREATE TABLE test.strings + CREATE TABLE test.strings (key UInt64, value String) ENGINE = Memory; """) diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py index 6b0e1936259..44698b380e3 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py @@ -9,11 +9,10 @@ from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager from helpers.network import PartitionManagerDisabler -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index 3fce7b7398d..e0b546aae24 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -8,11 +8,10 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml b/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 97874879525..e58b40df527 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -3,8 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -config_dir = os.path.join(SCRIPT_DIR, './configs') +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = [ 'configs/dictionaries/FileSourceConfig.xml', 'configs/dictionaries/ExecutableSourceConfig.xml', @@ -13,8 +12,8 @@ DICTIONARY_FILES = [ 'configs/dictionaries/ClickHouseSourceConfig.xml' ] -cluster = ClickHouseCluster(__file__, base_configs_dir=config_dir) -instance = cluster.add_instance('node', main_configs=DICTIONARY_FILES, config_dir=config_dir) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('node', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES) def prepare(): node = instance diff --git a/tests/integration/test_dictionary_ddl_on_cluster/test.py b/tests/integration/test_dictionary_ddl_on_cluster/test.py index 909d2e06377..6239fda1752 100644 --- a/tests/integration/test_dictionary_ddl_on_cluster/test.py +++ b/tests/integration/test_dictionary_ddl_on_cluster/test.py @@ -4,10 +4,10 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True) -ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True) -ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True) -ch4 = cluster.add_instance('ch4', config_dir="configs", with_zookeeper=True) +ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) +ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) +ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) +ch4 = cluster.add_instance('ch4', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_disk_types/configs/storage.xml b/tests/integration/test_disk_types/configs/storage.xml new file mode 100644 index 00000000000..2bf9a2e363a --- /dev/null +++ b/tests/integration/test_disk_types/configs/storage.xml @@ -0,0 +1,16 @@ + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + memory + + + + diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index 04346388b47..3c65315a7e3 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -1,3 +1,4 @@ + import pytest from helpers.cluster import ClickHouseCluster @@ -12,7 +13,7 @@ disk_types = { def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True) cluster.start() yield cluster finally: diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index 082a76cd88d..d7cb3d81c82 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,10 +17,23 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: + main_configs = [os.path.join(self.test_config_dir, "config.d/clusters.xml"), + os.path.join(self.test_config_dir, "config.d/zookeeper_session_timeout.xml"), + os.path.join(self.test_config_dir, "config.d/macro.xml"), + os.path.join(self.test_config_dir, "config.d/query_log.xml"), + os.path.join(self.test_config_dir, "config.d/ddl.xml")] + user_configs = [os.path.join(self.test_config_dir, "users.d/restricted_user.xml"), + os.path.join(self.test_config_dir, "users.d/query_log.xml")] + if self.test_config_dir == "configs_secure": + main_configs += [os.path.join(self.test_config_dir, "server.crt"), + os.path.join(self.test_config_dir, "server.key"), + os.path.join(self.test_config_dir, "dhparam.pem"), + os.path.join(self.test_config_dir, "config.d/ssl_conf.xml")] for i in xrange(4): self.add_instance( 'ch{}'.format(i+1), - config_dir=self.test_config_dir, + main_configs=main_configs, + user_configs=user_configs, macros={"layer": 0, "shard": i/2 + 1, "replica": i%2 + 1}, with_zookeeper=True) diff --git a/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml b/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml index 696695ddc69..fe39e3712b8 100644 --- a/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml +++ b/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml @@ -1,8 +1,9 @@ - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + /etc/clickhouse-server/config.d/dhparam.pem none true diff --git a/tests/integration/test_distributed_ddl_password/test.py b/tests/integration/test_distributed_ddl_password/test.py index f957f001df1..961b60857dd 100644 --- a/tests/integration/test_distributed_ddl_password/test.py +++ b/tests/integration/test_distributed_ddl_password/test.py @@ -6,12 +6,12 @@ from helpers.test_tools import assert_eq_with_retry from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) -node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", with_zookeeper=True) -node5 = cluster.add_instance('node5', config_dir="configs", with_zookeeper=True) -node6 = cluster.add_instance('node6', config_dir="configs", with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node4 = cluster.add_instance('node4', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node6 = cluster.add_instance('node6', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 291db89ae4c..251ec766b74 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -9,7 +9,7 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs", main_configs=['configs/remote_servers.xml']) +node = cluster.add_instance('node', main_configs=['configs/remote_servers.xml']) cluster_param = pytest.mark.parametrize("cluster", [ ('test_cluster'), diff --git a/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml b/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml index 696695ddc69..fe39e3712b8 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml +++ b/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml @@ -1,8 +1,9 @@ - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + /etc/clickhouse-server/config.d/dhparam.pem none true diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index ba760e90412..dc5168bfdad 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -1,6 +1,6 @@ import itertools import timeit - +import os.path import pytest from helpers.cluster import ClickHouseCluster @@ -91,8 +91,16 @@ def started_cluster(request): cluster = ClickHouseCluster(__file__) cluster.__with_ssl_config = request.param == "configs_secure" + main_configs = [] + main_configs += [os.path.join(request.param, "config.d/remote_servers.xml")] + if cluster.__with_ssl_config: + main_configs += [os.path.join(request.param, "server.crt")] + main_configs += [os.path.join(request.param, "server.key")] + main_configs += [os.path.join(request.param, "dhparam.pem")] + main_configs += [os.path.join(request.param, "config.d/ssl_conf.xml")] + user_configs = [os.path.join(request.param, "users.d/set_distributed_defaults.xml")] for name in NODES: - NODES[name] = cluster.add_instance(name, config_dir=request.param) + NODES[name] = cluster.add_instance(name, main_configs=main_configs, user_configs=user_configs) try: cluster.start() diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index 8dfaab659cb..716dd3e3075 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -9,7 +9,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', - config_dir='configs', + main_configs=["configs/config.d/storage_configuration.xml"], tmpfs=['/disk1:size=100M', '/disk2:size=100M']) @pytest.fixture(scope='module') diff --git a/tests/integration/test_enabling_access_management/test.py b/tests/integration/test_enabling_access_management/test.py index abb8cd6c07a..4a6ad59f0bb 100644 --- a/tests/integration/test_enabling_access_management/test.py +++ b/tests/integration/test_enabling_access_management/test.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', config_dir="configs") +instance = cluster.add_instance('instance', user_configs=["configs/users.d/extra_users.xml"]) @pytest.fixture(scope="module", autouse=True) def started_cluster(): diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 5c1ae389857..a7e6f10c1f6 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -12,8 +12,8 @@ from helpers.client import QueryTimeoutExceedException cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) -node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) +node1 = cluster.add_instance('node1', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) +node2 = cluster.add_instance('node2', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) nodes = [node1, node2] @pytest.fixture(scope="module") diff --git a/tests/integration/test_https_replication/configs/dhparam.pem b/tests/integration/test_https_replication/configs/dhparam.pem new file mode 100644 index 00000000000..2e6cee0798d --- /dev/null +++ b/tests/integration/test_https_replication/configs/dhparam.pem @@ -0,0 +1,8 @@ +-----BEGIN DH PARAMETERS----- +MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR +XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5 +Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb +9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf +slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb +dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg== +-----END DH PARAMETERS----- diff --git a/tests/integration/test_https_replication/configs/ssl_conf.xml b/tests/integration/test_https_replication/configs/ssl_conf.xml index 237bbc6af1c..ad7b874ebd3 100644 --- a/tests/integration/test_https_replication/configs/ssl_conf.xml +++ b/tests/integration/test_https_replication/configs/ssl_conf.xml @@ -1,8 +1,9 @@ - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + /etc/clickhouse-server/config.d/dhparam.pem none true @@ -15,4 +16,5 @@ 9010 + diff --git a/tests/integration/test_https_replication/test.py b/tests/integration/test_https_replication/test.py index a34c5faeccc..4974da850b4 100644 --- a/tests/integration/test_https_replication/test.py +++ b/tests/integration/test_https_replication/test.py @@ -23,8 +23,8 @@ def _fill_nodes(nodes, shard): '''.format(shard=shard, replica=node.name)) cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True) @pytest.fixture(scope="module") def both_https_cluster(): @@ -78,8 +78,8 @@ def test_replication_after_partition(both_https_cluster): -node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) +node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def both_http_cluster(): @@ -104,8 +104,8 @@ def test_both_http(both_http_cluster): assert_eq_with_retry(node3, "SELECT id FROM test_table order by id", '111\n222') assert_eq_with_retry(node4, "SELECT id FROM test_table order by id", '111\n222') -node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) -node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True) +node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def mixed_protocol_cluster(): diff --git a/tests/integration/test_log_family_s3/configs/minio.xml b/tests/integration/test_log_family_s3/configs/minio.xml new file mode 100644 index 00000000000..6c9329a2bbc --- /dev/null +++ b/tests/integration/test_log_family_s3/configs/minio.xml @@ -0,0 +1,13 @@ + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + diff --git a/tests/integration/test_log_family_s3/configs/ssl.xml b/tests/integration/test_log_family_s3/configs/ssl.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_log_family_s3/configs/ssl.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index 50e5b2ad19e..3b0d847967b 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -11,7 +11,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/minio.xml", "configs/ssl.xml", "configs/config.d/log_conf.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_max_http_connections_for_replication/test.py b/tests/integration/test_max_http_connections_for_replication/test.py index c421d36c315..0317aa19cc3 100644 --- a/tests/integration/test_max_http_connections_for_replication/test.py +++ b/tests/integration/test_max_http_connections_for_replication/test.py @@ -22,8 +22,8 @@ def _fill_nodes(nodes, shard, connections_count): '''.format(shard=shard, replica=node.name, connections=connections_count)) cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def start_small_cluster(): @@ -68,9 +68,9 @@ def test_keepalive_timeout(start_small_cluster): assert not node2.contains_in_log("No message received"), "Found 'No message received' in clickhouse-server.log" -node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node4 = cluster.add_instance('node4', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def start_big_cluster(): diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 01923293b21..de8ec8374e9 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -14,7 +14,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml", "configs/config.d/log_conf.xml"], user_configs=[], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -55,7 +55,7 @@ def create_table(cluster, table_name, additional_settings=None): ORDER BY (dt, id) SETTINGS storage_policy='s3', - old_parts_lifetime=0, + old_parts_lifetime=0, index_granularity=512 """.format(table_name) diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml new file mode 100644 index 00000000000..afcc8ba5c67 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml @@ -0,0 +1,9 @@ + + + + system + query_log
+ toYYYYMM(event_date) + 1000 +
+
diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_merge_tree_s3_with_cache/test.py b/tests/integration/test_merge_tree_s3_with_cache/test.py index 72c7d97cfed..25c08777ae5 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/test.py +++ b/tests/integration/test_merge_tree_s3_with_cache/test.py @@ -11,7 +11,9 @@ logging.getLogger().addHandler(logging.StreamHandler()) def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/log_conf.xml", "configs/config.d/storage_conf.xml", + "configs/config.d/ssl_conf.xml", "configs/config.d/query_log.xml"], + user_configs=["configs/config.d/users.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index d00450bf245..9034892ba83 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -13,16 +13,14 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'], with_zookeeper=True, stay_alive=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'], with_zookeeper=True, stay_alive=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 2791cc7b382..efbbe6d4104 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -127,7 +127,6 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster): with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: with pytest.raises(QueryRuntimeException) as exception: mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") - + clickhouse_node.query("CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") assert 'Database engine MySQL requested literal argument.' in str(exception.value) mysql_node.query("DROP DATABASE test_bad_arguments") diff --git a/tests/integration/test_mysql_protocol/configs/log_conf.xml b/tests/integration/test_mysql_protocol/configs/log_conf.xml new file mode 100644 index 00000000000..0346e43c81d --- /dev/null +++ b/tests/integration/test_mysql_protocol/configs/log_conf.xml @@ -0,0 +1,10 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_mysql_protocol/configs/mysql.xml b/tests/integration/test_mysql_protocol/configs/mysql.xml new file mode 100644 index 00000000000..a3ebc6e8576 --- /dev/null +++ b/tests/integration/test_mysql_protocol/configs/mysql.xml @@ -0,0 +1,4 @@ + + + 9001 + diff --git a/tests/integration/test_mysql_protocol/configs/ssl_conf.xml b/tests/integration/test_mysql_protocol/configs/ssl_conf.xml new file mode 100644 index 00000000000..5938b80ccb8 --- /dev/null +++ b/tests/integration/test_mysql_protocol/configs/ssl_conf.xml @@ -0,0 +1,18 @@ + + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + /etc/clickhouse-server/config.d/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index a31961dbd16..6e1ef39d2ca 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -17,9 +17,10 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) DOCKER_COMPOSE_PATH = get_docker_compose_path() -config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) +node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "configs/ssl_conf.xml", "configs/mysql.xml", + "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], + user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) server_port = 9001 @@ -36,7 +37,7 @@ def server_address(): @pytest.fixture(scope='module') def mysql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') @@ -62,28 +63,28 @@ def mysql_server(mysql_client): @pytest.fixture(scope='module') def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_java1_1') @@ -329,7 +330,7 @@ def test_python_client(server_address): def test_golang_client(server_address, golang_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR,'golang.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'golang.reference')) as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user default --password 123 --database ' diff --git a/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml b/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..93780125e8e --- /dev/null +++ b/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*dictionary.xml + diff --git a/tests/integration/test_odbc_interaction/configs/odbc_logging.xml b/tests/integration/test_odbc_interaction/configs/odbc_logging.xml new file mode 100644 index 00000000000..029275eb09c --- /dev/null +++ b/tests/integration/test_odbc_interaction/configs/odbc_logging.xml @@ -0,0 +1,8 @@ + + + + /var/log/clickhouse-server/clickhouse-odbc-bridge.log + /var/log/clickhouse-server/clickhouse-odbc-bridge.err.log + trace + + diff --git a/tests/integration/test_odbc_interaction/configs/openssl.xml b/tests/integration/test_odbc_interaction/configs/openssl.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_odbc_interaction/configs/openssl.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 46845802083..33b024363cb 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -7,10 +7,9 @@ import psycopg2 from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.cluster import ClickHouseCluster -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='yandex/clickhouse-integration-test', main_configs=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='yandex/clickhouse-integration-test', main_configs=['configs/openssl.xml','configs/odbc_logging.xml','configs/enable_dictionaries.xml','configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml','configs/dictionaries/sqlite3_odbc_cached_dictionary.xml','configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( diff --git a/tests/integration/test_old_versions/test.py b/tests/integration/test_old_versions/test.py index d77b4af016a..a1770333ba7 100644 --- a/tests/integration/test_old_versions/test.py +++ b/tests/integration/test_old_versions/test.py @@ -1,3 +1,4 @@ + import time import os import pytest @@ -9,13 +10,13 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True, config_dir="configs") -node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True, config_dir="configs") -node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True, config_dir="configs") -node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True, config_dir="configs") -node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', with_installed_binary=True, config_dir="configs") -node19_13 = cluster.add_instance('node19_13', image='yandex/clickhouse-server:19.13.7.57', with_installed_binary=True, config_dir="configs") -node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', with_installed_binary=True, config_dir="configs") +node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_13 = cluster.add_instance('node19_13', image='yandex/clickhouse-server:19.13.7.57', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) old_nodes = [node18_14, node19_1, node19_4, node19_8, node19_11, node19_13, node19_16] new_node = cluster.add_instance('node_new') diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index d3ebbd8c7a8..7fd29216680 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -53,21 +53,21 @@ def create_tables_old_format(name, nodes, shard): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}', date, id, 64) '''.format(name=name, shard=shard, repl=i)) -node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) settings_default = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings_compact_only = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 1000000, 'min_rows_for_compact_part' : 0} settings_not_adaptive = {'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} -node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/no_leader.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) +node4 = cluster.add_instance('node4', user_configs=["configs/users.d/not_optimize_count.xml"], main_configs=['configs/no_leader.xml'], with_zookeeper=True) settings_compact = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings_wide = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_rows_for_compact_part' : 0} -node5 = cluster.add_instance('node5', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) -node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) +node6 = cluster.add_instance('node6', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) settings_in_memory = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 256} @@ -213,8 +213,8 @@ def test_different_part_types_on_replicas(start_cluster, table, part_type): "WHERE table = '{}' AND active GROUP BY part_type ORDER BY part_type".format(table))) == TSV(expected) -node7 = cluster.add_instance('node7', config_dir="configs_old", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True) -node8 = cluster.add_instance('node8', config_dir="configs", with_zookeeper=True) +node7 = cluster.add_instance('node7', user_configs=["configs_old/users.d/not_optimize_count.xml"], with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True) +node8 = cluster.add_instance('node8', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) settings7 = {'index_granularity_bytes' : 10485760} settings8 = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} diff --git a/tests/integration/test_postgresql_protocol/configs/default_passwd.xml b/tests/integration/test_postgresql_protocol/configs/default_passwd.xml new file mode 100644 index 00000000000..86f5b6657c2 --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/default_passwd.xml @@ -0,0 +1,13 @@ + + + + + + + + + + 123 + + + diff --git a/tests/integration/test_postgresql_protocol/configs/log.xml b/tests/integration/test_postgresql_protocol/configs/log.xml new file mode 100644 index 00000000000..7f6380b0393 --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/log.xml @@ -0,0 +1,10 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_postgresql_protocol/configs/postresql.xml b/tests/integration/test_postgresql_protocol/configs/postresql.xml new file mode 100644 index 00000000000..aedfb59bedb --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/postresql.xml @@ -0,0 +1,4 @@ + + + 5433 + diff --git a/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml b/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml new file mode 100644 index 00000000000..271cb987218 --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml @@ -0,0 +1,18 @@ + + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + /etc/clickhouse-server/config.d/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 527c652229e..939e8231931 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -19,11 +19,12 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path psycopg2.extras.register_uuid() SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -config_dir = os.path.join(SCRIPT_DIR, './configs') DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) +node = cluster.add_instance('node', main_configs=["configs/postresql.xml", "configs/log.xml", "configs/ssl_conf.xml", + "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], + user_configs=["configs/default_passwd.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) server_port = 5433 diff --git a/tests/integration/test_profile_events_s3/configs/log.xml b/tests/integration/test_profile_events_s3/configs/log.xml new file mode 100644 index 00000000000..0346e43c81d --- /dev/null +++ b/tests/integration/test_profile_events_s3/configs/log.xml @@ -0,0 +1,10 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_profile_events_s3/configs/query_log.xml b/tests/integration/test_profile_events_s3/configs/query_log.xml new file mode 100644 index 00000000000..afcc8ba5c67 --- /dev/null +++ b/tests/integration/test_profile_events_s3/configs/query_log.xml @@ -0,0 +1,9 @@ + + + + system + query_log
+ toYYYYMM(event_date) + 1000 +
+
diff --git a/tests/integration/test_profile_events_s3/configs/ssl_conf.xml b/tests/integration/test_profile_events_s3/configs/ssl_conf.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_profile_events_s3/configs/ssl_conf.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index f98505757bf..e2cb10499e7 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -17,7 +17,7 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/log.xml", "configs/query_log.xml", "configs/ssl_conf.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index f490c13ca27..e89611c0d99 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -7,18 +7,15 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -zero = cluster.add_instance("zero", - config_dir="configs", +zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"], macros={"cluster": "anime", "shard": "0", "replica": "zero"}, with_zookeeper=True) -first = cluster.add_instance("first", - config_dir="configs", +first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"], macros={"cluster": "anime", "shard": "0", "replica": "first"}, with_zookeeper=True) -second = cluster.add_instance("second", - config_dir="configs", +second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"], macros={"cluster": "anime", "shard": "0", "replica": "second"}, with_zookeeper=True) diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 27aa353b9b1..4c97d127ad0 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -6,28 +6,38 @@ import re import time cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', - config_dir="configs") +instance = cluster.add_instance('instance', user_configs=["configs/users.d/assign_myquota.xml", "configs/users.d/drop_default_quota.xml", "configs/users.d/quota.xml"]) -def system_quotas(): - return TSV(instance.query("SELECT * FROM system.quotas ORDER BY name")) +def check_system_quotas(canonical): + canonical_tsv = TSV(canonical) + r = TSV(instance.query("SELECT * FROM system.quotas ORDER BY name")) + print("system_quotas: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv -def system_quota_limits(): - return TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration")) +def system_quota_limits(canonical): + canonical_tsv = TSV(canonical) + r = TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration")) + print("system_quota_limits: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv -def system_quota_usage(): +def system_quota_usage(canonical): + canonical_tsv = TSV(canonical) query = "SELECT quota_name, quota_key, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows,"\ "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\ "FROM system.quota_usage ORDER BY duration" - return TSV(instance.query(query)) + r = TSV(instance.query(query)) + print("system_quota_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv -def system_quotas_usage(): +def system_quotas_usage(canonical): + canonical_tsv = TSV(canonical) query = "SELECT quota_name, quota_key, is_current, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows, "\ "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\ "FROM system.quotas_usage ORDER BY quota_name, quota_key, duration" - return TSV(instance.query(query)) - + r = TSV(instance.query(query)) + print("system_quotas_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv def copy_quota_xml(local_file_name, reload_immediately = True): script_dir = os.path.dirname(os.path.realpath(__file__)) @@ -40,7 +50,7 @@ def copy_quota_xml(local_file_name, reload_immediately = True): def started_cluster(): try: cluster.start() - + instance.query("CREATE TABLE test_table(x UInt32) ENGINE = MergeTree ORDER BY tuple()") instance.query("INSERT INTO test_table SELECT number FROM numbers(50)") @@ -61,141 +71,141 @@ def reset_quotas_and_usage_info(): def test_quota_from_users_xml(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]) instance.query("SELECT COUNT() from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 51, "\N", 208, "\N", 50, 1000, 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 0, "\N", 51, "\N", 208, "\N", 50, 1000, 200, "\N", "\N"]]) def test_simpliest_quota(): # Simpliest quota doesn't even track usage. copy_quota_xml('simpliest.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]] - assert system_quota_limits() == "" - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]) + system_quota_limits("") + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) def test_tracking_quota(): # Now we're tracking usage. copy_quota_xml('tracking.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]]) instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, "\N", 0, "\N", 50, "\N", 200, "\N", 50, "\N", 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 1, "\N", 0, "\N", 50, "\N", 200, "\N", 50, "\N", 200, "\N", "\N"]]) instance.query("SELECT COUNT() from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, "\N", 0, "\N", 51, "\N", 208, "\N", 50, "\N", 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, "\N", 0, "\N", 51, "\N", 208, "\N", 50, "\N", 200, "\N", "\N"]]) def test_exceed_quota(): # Change quota, now the limits are tiny so we will exceed the quota. copy_quota_xml('tiny_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]]) assert re.search("Quota.*has\ been\ exceeded", instance.query_and_get_error("SELECT * from test_table")) - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\N", 50, 1, 0, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\N", 50, 1, 0, "\N", "\N"]]) # Change quota, now the limits are enough to execute queries. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 1, "\N", 50, "\N", 200, "\N", 100, 1000, 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 1, "\N", 50, "\N", 200, "\N", 100, 1000, 200, "\N", "\N"]]) def test_add_remove_interval(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Add interval. copy_quota_xml('two_intervals.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], - ["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"], - ["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], + ["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"], + ["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"], - ["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]] + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"], + ["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]]) # Remove interval. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 100, "\N", 400, "\N", 100, 1000, 400, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 0, "\N", 100, "\N", 400, "\N", 100, 1000, 400, "\N", "\N"]]) # Remove all intervals. copy_quota_xml('simpliest.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]] - assert system_quota_limits() == "" - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]) + system_quota_limits("") + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) # Add one interval back. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) def test_add_remove_quota(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Add quota. copy_quota_xml('two_quotas.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"], - ["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"], + ["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], ["myQuota2", 3600, 1, "\N", "\N", 4000, 400000, 4000, 400000, 60], - ["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + ["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Drop quota. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Drop all quotas. copy_quota_xml('no_quotas.xml') - assert system_quotas() == "" - assert system_quota_limits() == "" - assert system_quotas_usage() == "" + check_system_quotas("") + system_quota_limits("") + system_quotas_usage("") # Add one quota back. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) def test_reload_users_xml_by_timer(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) time.sleep(1) # The modification time of the 'quota.xml' file should be different, # because config files are reload by timer only when the modification time is changed. @@ -246,7 +256,7 @@ def test_dcl_introspection(): def test_dcl_management(): copy_quota_xml('no_quotas.xml') assert instance.query("SHOW QUOTA") == "" - + instance.query("CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER") assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA FOR INTERVAL 5 quarter MAX queries = 123 TO default\n" assert re.match("qA\\t\\t.*\\t39446190\\t0\\t123\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t.*\\t\\\\N\n", diff --git a/tests/integration/test_random_inserts/test.py b/tests/integration/test_random_inserts/test.py index eb644a7a19c..4e3d8db7e53 100644 --- a/tests/integration/test_random_inserts/test.py +++ b/tests/integration/test_random_inserts/test.py @@ -14,8 +14,8 @@ from helpers.client import CommandRequest cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) -node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) +node1 = cluster.add_instance('node1', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml" ], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) +node2 = cluster.add_instance('node2', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml" ], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) nodes = [node1, node2] @pytest.fixture(scope="module") diff --git a/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml b/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml new file mode 100644 index 00000000000..03d5e33646f --- /dev/null +++ b/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml @@ -0,0 +1,5 @@ + + + 1 + 1 + diff --git a/tests/integration/test_reload_max_table_size_to_drop/test.py b/tests/integration/test_reload_max_table_size_to_drop/test.py index 3959b383fc5..9d0bc244521 100644 --- a/tests/integration/test_reload_max_table_size_to_drop/test.py +++ b/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -1,3 +1,4 @@ + import time import pytest import os @@ -6,10 +7,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs") +node = cluster.add_instance('node', main_configs=["configs/max_table_size_to_drop.xml"]) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.xml') +CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.d/max_table_size_to_drop.xml') @pytest.fixture(scope="module") diff --git a/tests/integration/test_rename_column/test.py b/tests/integration/test_rename_column/test.py index 029d140d0ed..9a108583347 100644 --- a/tests/integration/test_rename_column/test.py +++ b/tests/integration/test_rename_column/test.py @@ -12,8 +12,9 @@ from helpers.test_tools import TSV node_options = dict( with_zookeeper=True, - main_configs=['configs/remote_servers.xml'], - config_dir='configs', + main_configs=["configs/remote_servers.xml", "configs/config.d/instant_moves.xml", + "configs/config.d/part_log.xml", "configs/config.d/zookeeper_session_timeout.xml", + "configs/config.d/storage_configuration.xml"], tmpfs=['/external:size=200M', '/internal:size=1M']) cluster = ClickHouseCluster(__file__) diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 69d41b1ce11..9e617506d29 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -15,9 +15,9 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node1", config_dir="configs", macros={'cluster': 'test1'}, with_minio=True, with_zookeeper=True) - cluster.add_instance("node2", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) - cluster.add_instance("node3", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) + cluster.add_instance("node1", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_minio=True, with_zookeeper=True) + cluster.add_instance("node2", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_zookeeper=True) + cluster.add_instance("node3", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_zookeeper=True) logging.info("Starting cluster...") cluster.start() diff --git a/tests/integration/test_row_policy/configs/users.d/another_user.xml b/tests/integration/test_row_policy/configs/users.d/another_user.xml new file mode 100644 index 00000000000..fb9608e5313 --- /dev/null +++ b/tests/integration/test_row_policy/configs/users.d/another_user.xml @@ -0,0 +1,13 @@ + + + + + + + ::/0 + + default + default + + + \ No newline at end of file diff --git a/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml b/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml new file mode 100644 index 00000000000..413e64ba3dc --- /dev/null +++ b/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml @@ -0,0 +1,8 @@ + + + + + 1 + + + diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 15796ff0c83..dd0495df237 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -6,8 +6,8 @@ import re import time cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs", with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) +node = cluster.add_instance('node', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/row_policy.xml", "configs/users.d/another_user.xml", "configs/users.d/any_join_distinct_right_table_keys.xml"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/row_policy.xml", "configs/users.d/another_user.xml", "configs/users.d/any_join_distinct_right_table_keys.xml"], with_zookeeper=True) nodes = [node, node2] @@ -42,7 +42,7 @@ def started_cluster(): CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.`.filtered_table4` values (0, 0), (0, 1), (1, 0), (1, 1); - + CREATE TABLE mydb.local (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; ''') @@ -185,7 +185,7 @@ def test_introspection(): def test_dcl_introspection(): assert node.query("SHOW POLICIES") == TSV(["another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3", "another ON mydb.local", "default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3", "default ON mydb.local"]) - + assert node.query("SHOW POLICIES ON mydb.filtered_table1") == TSV([ "another", "default" ]) assert node.query("SHOW POLICIES ON mydb.local") == TSV([ "another", "default" ]) assert node.query("SHOW POLICIES ON mydb.*") == TSV([ "another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3", "another ON mydb.local", "default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3", "default ON mydb.local" ]) @@ -195,7 +195,7 @@ def test_dcl_introspection(): assert node.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n" assert node.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n" assert node.query("SHOW CREATE POLICY default ON mydb.local") == "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n" - + assert node.query("SHOW CREATE POLICY default") == TSV([ "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default", "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default", "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default", "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default" ]) assert node.query("SHOW CREATE POLICIES ON mydb.filtered_table1") == TSV([ "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another", "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default" ]) assert node.query("SHOW CREATE POLICIES ON mydb.*") == TSV([ "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another", "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default", "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default", "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default", "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default" ]) diff --git a/tests/integration/test_s3_with_https/configs/config.d/ssl.xml b/tests/integration/test_s3_with_https/configs/config.d/ssl.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_s3_with_https/configs/config.d/ssl.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 81e57106afc..2b40e02e701 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -18,7 +18,7 @@ def check_proxy_logs(cluster, proxy_instance): def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True, minio_certs_dir='minio_certs') + cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/log_conf.xml", "configs/config.d/ssl.xml"], with_minio=True, minio_certs_dir='minio_certs') logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 0642cd88fe7..daf53c2e27b 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -21,7 +21,7 @@ def run_resolver(cluster): def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/log_conf.xml", "configs/config.d/storage_conf.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_settings_constraints/test.py b/tests/integration/test_settings_constraints/test.py index 1c8e91484ca..b2dcd80448f 100644 --- a/tests/integration/test_settings_constraints/test.py +++ b/tests/integration/test_settings_constraints/test.py @@ -2,8 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', - config_dir="configs") +instance = cluster.add_instance('instance', user_configs=["configs/users.xml"]) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index 7f0f8868bcf..94afa0d6d2d 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -8,9 +8,9 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs") -node2 = cluster.add_instance('node2', config_dir="configs") -distributed = cluster.add_instance('distributed', config_dir="configs", stay_alive=True) +node1 = cluster.add_instance('node1', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/allow_introspection_functions.xml"]) +node2 = cluster.add_instance('node2', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/allow_introspection_functions.xml"]) +distributed = cluster.add_instance('distributed', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/allow_introspection_functions.xml"], stay_alive=True) @pytest.fixture(scope="module", autouse=True) @@ -56,7 +56,7 @@ def test_select_clamps_settings(): assert distributed.query(query, user = 'normal') == '2\n' assert distributed.query(query, user = 'wasteful') == '2\n' assert distributed.query(query, user = 'readonly') == '2\n' - + assert distributed.query(query, settings={"max_memory_usage": 40000000, "readonly": 2}) == '2\n' assert distributed.query(query, settings={"max_memory_usage": 3000000000, "readonly": 2}) == '2\n' diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index d65b0efc334..20613bde1bc 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -12,7 +12,7 @@ import subprocess SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_hdfs=True, config_dir="configs", main_configs=['configs/log_conf.xml']) +node1 = cluster.add_instance('node1', with_hdfs=True, user_configs=[], main_configs=['configs/log_conf.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 42b7101f9c6..5ebde084de7 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -20,7 +20,6 @@ from google.protobuf.internal.encoder import _VarintBytes cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', - config_dir='configs', main_configs=['configs/rabbitmq.xml','configs/log_conf.xml'], with_rabbitmq=True) rabbitmq_id = '' diff --git a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml index a149c2ba774..806a59debca 100644 --- a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml +++ b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml @@ -1,4 +1,4 @@ - + clickhouse_cache @@ -34,4 +34,4 @@ - +
\ No newline at end of file diff --git a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml index feb01b27d1b..e7d32590a39 100644 --- a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml +++ b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml @@ -1,4 +1,4 @@ - + clickhouse_flat @@ -34,4 +34,4 @@ - +
\ No newline at end of file diff --git a/tests/integration/test_system_queries/test.py b/tests/integration/test_system_queries/test.py index 6f36a13b184..db9cf5ccf3c 100644 --- a/tests/integration/test_system_queries/test.py +++ b/tests/integration/test_system_queries/test.py @@ -18,13 +18,14 @@ def started_cluster(): global instance try: cluster = ClickHouseCluster(__file__) - cluster.add_instance('ch1', config_dir="configs") + cluster.add_instance('ch1', main_configs=["configs/config.d/clusters_config.xml", "configs/config.d/query_log.xml"], + dictionaries=["configs/dictionaries/dictionary_clickhouse_cache.xml", "configs/dictionaries/dictionary_clickhouse_flat.xml"]) cluster.start() instance = cluster.instances['ch1'] instance.query('CREATE DATABASE dictionaries ENGINE = Dictionary') instance.query('CREATE TABLE dictionary_source (id UInt64, value UInt8) ENGINE = Memory') - #print instance.query('SELECT * FROM system.dictionaries FORMAT Vertical') + print instance.query('SELECT * FROM system.dictionaries FORMAT Vertical') print "Started ", instance.ip_address yield cluster @@ -90,7 +91,7 @@ def test_RELOAD_CONFIG_AND_MACROS(started_cluster): instance.exec_in_container(['bash', '-c', create_macros], privileged=True, user='root') instance.query("SYSTEM RELOAD CONFIG") - assert TSV(instance.query("select * from system.macros")) == TSV("mac\tro\n") + assert TSV(instance.query("select * from system.macros")) == TSV("instance\tch1\nmac\tro\n") def test_SYSTEM_FLUSH_LOGS(started_cluster): diff --git a/tests/integration/test_text_log_level/test.py b/tests/integration/test_text_log_level/test.py index d7cf72fd9ea..799ae9021cb 100644 --- a/tests/integration/test_text_log_level/test.py +++ b/tests/integration/test_text_log_level/test.py @@ -8,7 +8,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir='configs') +node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"]) @pytest.fixture(scope='module') def start_cluster(): diff --git a/tests/integration/test_tmp_policy/test.py b/tests/integration/test_tmp_policy/test.py index 5c5900cc9dc..728c62d82fb 100644 --- a/tests/integration/test_tmp_policy/test.py +++ b/tests/integration/test_tmp_policy/test.py @@ -8,7 +8,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', - config_dir='configs', + main_configs=["configs/config.d/storage_configuration.xml"], tmpfs=['/disk1:size=100M', '/disk2:size=100M']) @pytest.fixture(scope='module') diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index eedcb01ee3a..d0db52287ca 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -14,15 +14,13 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', "configs/config.d/instant_moves.xml", "configs/config.d/storage_configuration.xml", "configs/config.d/cluster.xml",], with_zookeeper=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', "configs/config.d/instant_moves.xml", "configs/config.d/storage_configuration.xml", "configs/config.d/cluster.xml",], with_zookeeper=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], macros={"shard": 0, "replica": 2} ) @@ -173,7 +171,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() """.format(name=name, engine=engine)) - + node1.query("""ALTER TABLE {name} MODIFY SETTING storage_policy='default_with_small_jbod_with_external'""".format(name=name)) # Second expression is preferred because d1 > now()-3600. diff --git a/tests/integration/test_user_ip_restrictions/test.py b/tests/integration/test_user_ip_restrictions/test.py index 731f2bd7fa8..aee0819fe95 100644 --- a/tests/integration/test_user_ip_restrictions/test.py +++ b/tests/integration/test_user_ip_restrictions/test.py @@ -7,16 +7,16 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node_ipv4 = cluster.add_instance('node_ipv4', config_dir="configs", user_configs=['configs/users_ipv4.xml'], ipv4_address='10.5.172.77') -client_ipv4_ok = cluster.add_instance('client_ipv4_ok', config_dir="configs", ipv4_address='10.5.172.10') -client_ipv4_ok_direct = cluster.add_instance('client_ipv4_ok_direct', config_dir="configs", ipv4_address='10.5.173.1') -client_ipv4_ok_full_mask = cluster.add_instance('client_ipv4_ok_full_mask', config_dir="configs", ipv4_address='10.5.175.77') -client_ipv4_bad = cluster.add_instance('client_ipv4_bad', config_dir="configs", ipv4_address='10.5.173.10') +node_ipv4 = cluster.add_instance('node_ipv4', main_configs=[], user_configs=['configs/users_ipv4.xml'], ipv4_address='10.5.172.77') +client_ipv4_ok = cluster.add_instance('client_ipv4_ok', main_configs=[], user_configs=[], ipv4_address='10.5.172.10') +client_ipv4_ok_direct = cluster.add_instance('client_ipv4_ok_direct', main_configs=[], user_configs=[], ipv4_address='10.5.173.1') +client_ipv4_ok_full_mask = cluster.add_instance('client_ipv4_ok_full_mask', main_configs=[], user_configs=[], ipv4_address='10.5.175.77') +client_ipv4_bad = cluster.add_instance('client_ipv4_bad', main_configs=[], user_configs=[], ipv4_address='10.5.173.10') -node_ipv6 = cluster.add_instance('node_ipv6', config_dir="configs", main_configs=["configs/config_ipv6.xml"], user_configs=['configs/users_ipv6.xml'], ipv6_address='2001:3984:3989::1:1000') -client_ipv6_ok = cluster.add_instance('client_ipv6_ok', config_dir="configs", ipv6_address='2001:3984:3989::5555') -client_ipv6_ok_direct = cluster.add_instance('client_ipv6_ok_direct', config_dir="configs", ipv6_address='2001:3984:3989::1:1111') -client_ipv6_bad = cluster.add_instance('client_ipv6_bad', config_dir="configs", ipv6_address='2001:3984:3989::1:1112') +node_ipv6 = cluster.add_instance('node_ipv6', main_configs=["configs/config_ipv6.xml"], user_configs=['configs/users_ipv6.xml'], ipv6_address='2001:3984:3989::1:1000') +client_ipv6_ok = cluster.add_instance('client_ipv6_ok', main_configs=[], user_configs=[], ipv6_address='2001:3984:3989::5555') +client_ipv6_ok_direct = cluster.add_instance('client_ipv6_ok_direct', main_configs=[], user_configs=[], ipv6_address='2001:3984:3989::1:1111') +client_ipv6_bad = cluster.add_instance('client_ipv6_bad', main_configs=[], user_configs=[], ipv6_address='2001:3984:3989::1:1112') @pytest.fixture(scope="module") diff --git a/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py b/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py index f3d57e2e174..3af5c18544a 100644 --- a/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py +++ b/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs") +node = cluster.add_instance('node', user_configs=["configs/users.xml"]) @pytest.fixture(scope="module") diff --git a/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml b/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml index 5e6f5f37624..50303fb70cc 100644 --- a/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml +++ b/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml @@ -1,8 +1,8 @@ - /etc/clickhouse-server/client.crt - /etc/clickhouse-server/client.key + /etc/clickhouse-server/config.d/client.crt + /etc/clickhouse-server/config.d/client.key true true sslv2,sslv3 diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 5ee6a8af021..086b9ac0c73 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -12,8 +12,8 @@ def test_chroot_with_same_root(): cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') - node1 = cluster_1.add_instance('node1', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) + node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) + node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) nodes = [node1, node2] def create_zk_root(zk): @@ -51,8 +51,8 @@ def test_chroot_with_different_root(): cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_b.xml') - node1 = cluster_1.add_instance('node1', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) + node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) + node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) nodes = [node1, node2] def create_zk_roots(zk): @@ -90,8 +90,8 @@ def test_identity(): cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_password.xml') cluster_2 = ClickHouseCluster(__file__) - node1 = cluster_1.add_instance('node1', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) + node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_with_password.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) + node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) try: cluster_1.start() @@ -145,10 +145,12 @@ def test_secure_connection(): ) docker_compose.close() - node1 = cluster.add_instance('node1', config_dir='configs_secure', with_zookeeper=True, - zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) - node2 = cluster.add_instance('node2', config_dir='configs_secure', with_zookeeper=True, - zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) + node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], + with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) + node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], + with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) try: cluster.start() From ad740fc7daddafd90d59a87a48fe80da904f25b8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 12:26:01 +0300 Subject: [PATCH 041/535] copy dictionaries --- tests/integration/helpers/dictonaries | 1 - .../dictonaries/decimals_dictionary.xml | 197 +++++++ .../helpers/dictonaries/ints_dictionary.xml | 514 ++++++++++++++++++ .../dictonaries/strings_dictionary.xml | 209 +++++++ 4 files changed, 920 insertions(+), 1 deletion(-) delete mode 120000 tests/integration/helpers/dictonaries create mode 100644 tests/integration/helpers/dictonaries/decimals_dictionary.xml create mode 100644 tests/integration/helpers/dictonaries/ints_dictionary.xml create mode 100644 tests/integration/helpers/dictonaries/strings_dictionary.xml diff --git a/tests/integration/helpers/dictonaries b/tests/integration/helpers/dictonaries deleted file mode 120000 index b33ab3b1e87..00000000000 --- a/tests/integration/helpers/dictonaries +++ /dev/null @@ -1 +0,0 @@ -../../config/dict_examples/ \ No newline at end of file diff --git a/tests/integration/helpers/dictonaries/decimals_dictionary.xml b/tests/integration/helpers/dictonaries/decimals_dictionary.xml new file mode 100644 index 00000000000..f728fa774a7 --- /dev/null +++ b/tests/integration/helpers/dictonaries/decimals_dictionary.xml @@ -0,0 +1,197 @@ + + + flat_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + hashed_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + cache_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + 1000 + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + complex_hashed_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + + + + + + key + UInt64 + + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + complex_cache_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+
diff --git a/tests/integration/helpers/dictonaries/ints_dictionary.xml b/tests/integration/helpers/dictonaries/ints_dictionary.xml new file mode 100644 index 00000000000..a22dab8933c --- /dev/null +++ b/tests/integration/helpers/dictonaries/ints_dictionary.xml @@ -0,0 +1,514 @@ + + + flat_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + hashed_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + hashed_sparse_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + cache_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + 1000 + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + complex_hashed_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + + key + UInt64 + + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + complex_cache_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + +one_cell_cache_ints + + + localhost + 9000 + default + + test_01054 + ints
+
+ +0 + + 1 + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + + one_cell_cache_ints_overflow + + + localhost + 9000 + default + + test_01054_overflow + ints
+
+ + 0 + + 1 + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ +
\ No newline at end of file diff --git a/tests/integration/helpers/dictonaries/strings_dictionary.xml b/tests/integration/helpers/dictonaries/strings_dictionary.xml new file mode 100644 index 00000000000..c5643eecb68 --- /dev/null +++ b/tests/integration/helpers/dictonaries/strings_dictionary.xml @@ -0,0 +1,209 @@ + + + flat_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + key + + + str + String + + + +
+ + + hashed_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + key + + + str + String + + + +
+ + + cache_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + 1000 + + + + key + + + str + String + + + +
+ + + complex_hashed_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + + key + UInt64 + + + + str + String + + + +
+ + + complex_cache_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + str + String + + + +
+ + + complex_hashed_strings_key + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + + str + String + + + + key + UInt64 + 0 + + +
+ + + complex_cache_strings_key + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + 1000 + + + + + str + String + + + + key + UInt64 + 0 + + +
+
From 730056a9f0167c1b41f57493acc7c1914eb76c77 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 15:22:31 +0300 Subject: [PATCH 042/535] fix --- docker/test/integration/runner/dockerd-entrypoint.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 9abf3bde53d..c38260279ed 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -19,8 +19,7 @@ set -e echo "Start tests" export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse -export CLICKHOUSE_TESTS_CONFIG_DIR=/clickhouse-config -export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-base-config +export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} From 73e9f4d4210449d943dcd461bacf4524b211d0cd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 13 Aug 2020 21:20:47 +0300 Subject: [PATCH 043/535] update test_materialize_mysql_database test --- .../runner/compose/docker_compose_mysql.yml | 1 + .../compose/docker_compose_mysql_8_0.yml | 0 .../composes/mysql_5_7_compose.yml | 10 ------- .../materialize_with_ddl.py | 6 ++-- .../test_materialize_mysql_database/test.py | 30 +++++++++---------- 5 files changed, 19 insertions(+), 28 deletions(-) rename tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml (100%) delete mode 100644 tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 2e3afce117d..cef781f95c4 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -7,3 +7,4 @@ services: MYSQL_ROOT_PASSWORD: clickhouse ports: - 3308:3306 + command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml similarity index 100% rename from tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml deleted file mode 100644 index bfc5b6a9538..00000000000 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml +++ /dev/null @@ -1,10 +0,0 @@ -version: '2.3' -services: - mysql5_7: - image: mysql:5.7 - restart: always - environment: - MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 33307:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index cc3a8f82fe1..26f8e9416ba 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -177,7 +177,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ("0" if service_name == "mysql5_7" else "(id)")) + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ("0" if service_name == "mysql1" else "(id)")) # create mapping clickhouse_node.query( @@ -193,9 +193,9 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql5_7" else "(id)")) + "0" if service_name == "mysql1" else "(id)")) - default_expression = "DEFAULT\t0" if service_name == "mysql5_7" else "DEFAULT\tid" + default_expression = "DEFAULT\t0" if service_name == "mysql1" else "DEFAULT\tid" check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 08baf87e69f..dceacc1d1e2 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -6,12 +6,12 @@ import pymysql.cursors import pytest import materialize_with_ddl -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) -clickhouse_node = cluster.add_instance('node1', config_dir="configs", with_mysql=False) +clickhouse_node = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False) @pytest.fixture(scope="module") @@ -61,8 +61,8 @@ class MySQLNodeInstance: @pytest.fixture(scope="module") def started_mysql_5_7(): - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33307) - docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_5_7_compose.yml') + mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308) + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql.yml') try: subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) @@ -76,7 +76,7 @@ def started_mysql_5_7(): @pytest.fixture(scope="module") def started_mysql_8_0(): mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308) - docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_8_0_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0.yml') try: subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) @@ -88,7 +88,7 @@ def started_mysql_8_0(): def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0): @@ -96,15 +96,15 @@ def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") # mysql 5.7 cannot support alter rename column - # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0): From a79c3175a7bda80ceb541ed60c8580de08a7445b Mon Sep 17 00:00:00 2001 From: it1804 Date: Fri, 14 Aug 2020 02:00:12 +0500 Subject: [PATCH 044/535] Allow authenticate Redis with requirepass option --- src/Dictionaries/RedisDictionarySource.cpp | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 8794f0620e2..030ee2b1a06 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -51,12 +51,14 @@ namespace DB const String & host_, UInt16 port_, UInt8 db_index_, + const String & password_, RedisStorageType storage_type_, const Block & sample_block_) : dict_struct{dict_struct_} , host{host_} , port{port_} , db_index{db_index_} + , password{password_} , storage_type{storage_type_} , sample_block{sample_block_} , client{std::make_shared(host, port)} @@ -77,16 +79,22 @@ namespace DB ErrorCodes::INVALID_CONFIG_PARAMETER}; // suppose key[0] is primary key, key[1] is secondary key } + if (!password.empty()) + { + RedisCommand command("AUTH"); + command << password; + String reply = client->execute(command); + if (reply != "OK") + throw Exception{"Authentication failed with reason " + + reply, ErrorCodes::INTERNAL_REDIS_ERROR}; + } if (db_index != 0) { RedisCommand command("SELECT"); - // Use poco's Int64, because it is defined as long long, and on - // MacOS, for the purposes of template instantiation, this type is - // distinct from int64_t, which is our Int64. - command << static_cast(db_index); + command << std::to_string(db_index); String reply = client->execute(command); - if (reply != "+OK\r\n") + if (reply != "OK") throw Exception{"Selecting database with index " + DB::toString(db_index) + " failed with reason " + reply, ErrorCodes::INTERNAL_REDIS_ERROR}; } @@ -103,6 +111,7 @@ namespace DB config_.getString(config_prefix_ + ".host"), config_.getUInt(config_prefix_ + ".port"), config_.getUInt(config_prefix_ + ".db_index", 0), + config_.getString(config_prefix_ + ".password",""), parseStorageType(config_.getString(config_prefix_ + ".storage_type", "")), sample_block_) { @@ -114,6 +123,7 @@ namespace DB other.host, other.port, other.db_index, + other.password, other.storage_type, other.sample_block} { From a1c0c52c5bdda2358139d712352c706f4dd20086 Mon Sep 17 00:00:00 2001 From: it1804 Date: Fri, 14 Aug 2020 02:01:25 +0500 Subject: [PATCH 045/535] Allow authenticate Redis with requirepass option --- src/Dictionaries/RedisDictionarySource.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index b30c428cb2d..75dcc2fb081 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -41,6 +41,7 @@ namespace ErrorCodes const std::string & host, UInt16 port, UInt8 db_index, + const std::string & password, RedisStorageType storage_type, const Block & sample_block); @@ -91,6 +92,7 @@ namespace ErrorCodes const std::string host; const UInt16 port; const UInt8 db_index; + const std::string password; const RedisStorageType storage_type; Block sample_block; From fac881a6f0f50005fffa95e4ef77c071bb2c5d0e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 Aug 2020 18:51:28 +0300 Subject: [PATCH 046/535] finally remove config_dir --- tests/integration/helpers/cluster.py | 21 +++++-------------- .../test_distributed_ddl/cluster.py | 16 +++++--------- .../test_polymorphic_parts/test.py | 8 +++---- .../test.py | 2 -- tests/integration/test_system_merges/test.py | 2 -- 5 files changed, 14 insertions(+), 35 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 69db0c0fb10..9ce84478c7a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -89,7 +89,7 @@ class ClickHouseCluster: these directories will contain logs, database files, docker-compose config, ClickHouse configs etc. """ - def __init__(self, base_path, name=None, base_config_dir=None, config_dir=None, server_bin_path=None, client_bin_path=None, + def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): for param in os.environ.keys(): print "ENV %40s %s" % (param,os.environ[param]) @@ -98,8 +98,6 @@ class ClickHouseCluster: self.base_config_dir = base_config_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/') - self.config_dir = config_dir or os.environ.get('CLICKHOUSE_TESTS_CONFIG_DIR', - '/etc/clickhouse-server/') self.server_bin_path = p.realpath( server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) self.odbc_bridge_bin_path = p.realpath(odbc_bridge_bin_path or get_odbc_bridge_path()) @@ -159,7 +157,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False - print "CLUSTER INIT base_config_dir:{} config_dir:{}".format(self.base_config_dir, self.config_dir) + print "CLUSTER INIT base_config_dir:{}".format(self.base_config_dir) def get_client_cmd(self): cmd = self.client_bin_path @@ -167,7 +165,7 @@ class ClickHouseCluster: cmd += " client" return cmd - def add_instance(self, name, base_config_dir=None, config_dir=None, main_configs=None, user_configs=None, dictionaries = None, macros=None, + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries = None, macros=None, with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -177,7 +175,6 @@ class ClickHouseCluster: """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. - config_dir - a directory with config files which content will be copied to /etc/clickhouse-server/ directory base_config_dir - a directory with config.xml and users.xml files which will be copied to /etc/clickhouse-server/ directory main_configs - a list of config files that will be added to config.d/ directory user_configs - a list of config files that will be added to users.d/ directory @@ -192,8 +189,7 @@ class ClickHouseCluster: instance = ClickHouseInstance( self, self.base_dir, name, base_config_dir if base_config_dir else self.base_config_dir, - config_dir if config_dir else self.config_dir, main_configs or [], user_configs or [], dictionaries or [], - macros or {}, with_zookeeper, + main_configs or [], user_configs or [], dictionaries or [], macros or {}, with_zookeeper, self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables or {}, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, @@ -765,7 +761,7 @@ services: class ClickHouseInstance: def __init__( - self, cluster, base_path, name, base_config_dir, config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, + self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, @@ -780,7 +776,6 @@ class ClickHouseInstance: self.tmpfs = tmpfs or [] self.base_config_dir = p.abspath(p.join(base_path, base_config_dir)) if base_config_dir else None - self.config_dir = p.abspath(p.join(base_path, config_dir)) if config_dir else None self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs] self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries] @@ -1126,10 +1121,6 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - # print "Copy config dir {} to {}".format(self.config_dir, instance_config_dir) - # if self.config_dir: - # distutils.dir_util.copy_tree(self.config_dir, instance_config_dir) - # Copy config.d configs print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) for path in self.custom_main_config_paths: @@ -1139,8 +1130,6 @@ class ClickHouseInstance: for path in self.custom_user_config_paths: shutil.copy(path, users_d_dir) - - self.config_dir # Copy dictionaries configs to configs/dictionaries for path in self.custom_dictionaries_paths: shutil.copy(path, dictionaries_dir) diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index d7cb3d81c82..b3a0513b799 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,18 +17,12 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: - main_configs = [os.path.join(self.test_config_dir, "config.d/clusters.xml"), - os.path.join(self.test_config_dir, "config.d/zookeeper_session_timeout.xml"), - os.path.join(self.test_config_dir, "config.d/macro.xml"), - os.path.join(self.test_config_dir, "config.d/query_log.xml"), - os.path.join(self.test_config_dir, "config.d/ddl.xml")] - user_configs = [os.path.join(self.test_config_dir, "users.d/restricted_user.xml"), - os.path.join(self.test_config_dir, "users.d/query_log.xml")] + main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml"), + "query_log.xml","ddl.xml"] + main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files)] + user_configs = [os.path.join(self.test_config_dir, "users.d", f) for f in ["restricted_user.xml", "query_log.xml"]] if self.test_config_dir == "configs_secure": - main_configs += [os.path.join(self.test_config_dir, "server.crt"), - os.path.join(self.test_config_dir, "server.key"), - os.path.join(self.test_config_dir, "dhparam.pem"), - os.path.join(self.test_config_dir, "config.d/ssl_conf.xml")] + main_configs += [os.path.join(self.test_config_dir, f) for i in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] for i in xrange(4): self.add_instance( 'ch{}'.format(i+1), diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 7fd29216680..e6c093ad414 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -71,11 +71,11 @@ node6 = cluster.add_instance('node6', main_configs=['configs/compact_parts.xml'] settings_in_memory = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 256} -node9 = cluster.add_instance('node9', config_dir="configs", with_zookeeper=True, stay_alive=True) -node10 = cluster.add_instance('node10', config_dir="configs", with_zookeeper=True) +node9 = cluster.add_instance('node9', with_zookeeper=True, stay_alive=True) +node10 = cluster.add_instance('node10', with_zookeeper=True) -node11 = cluster.add_instance('node11', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) -node12 = cluster.add_instance('node12', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) +node11 = cluster.add_instance('node11', main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) +node12 = cluster.add_instance('node12', main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) @pytest.fixture(scope="module") def start_cluster(): diff --git a/tests/integration/test_reloading_storage_configuration/test.py b/tests/integration/test_reloading_storage_configuration/test.py index c9effcdd67a..a30d4029d7c 100644 --- a/tests/integration/test_reloading_storage_configuration/test.py +++ b/tests/integration/test_reloading_storage_configuration/test.py @@ -14,7 +14,6 @@ import helpers.cluster cluster = helpers.cluster.ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, stay_alive=True, @@ -22,7 +21,6 @@ node1 = cluster.add_instance('node1', macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, stay_alive=True, diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 8e3714bc23b..15e5b1c0835 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -6,13 +6,11 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, macros={"shard": 0, "replica": 2} ) From c6fdeb6c021b0d9724608925513c3ef657e5a232 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 15 Aug 2020 06:50:53 +0000 Subject: [PATCH 047/535] Better --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 8 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 6 +- .../ReadBufferFromRabbitMQConsumer.cpp | 122 +++--- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 51 ++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 88 ++-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 12 + .../WriteBufferToRabbitMQProducer.cpp | 104 +++-- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 10 +- .../integration/test_storage_rabbitmq/test.py | 396 ++++++++---------- 9 files changed, 419 insertions(+), 378 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 589f5b39d2e..e26645a1168 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -22,7 +22,7 @@ RabbitMQBlockInputStream::RabbitMQBlockInputStream( , column_names(columns) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header(metadata_snapshot->getSampleBlockForColumns( - {"_exchange_name", "_consumer_tag", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) + {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) { } @@ -128,16 +128,16 @@ Block RabbitMQBlockInputStream::readImpl() if (new_rows) { auto exchange_name = storage.getExchange(); - auto consumer_tag = buffer->getConsumerTag(); + auto channel_id = buffer->getChannelID(); auto delivery_tag = buffer->getDeliveryTag(); auto redelivered = buffer->getRedelivered(); - buffer->updateNextDeliveryTag(delivery_tag); + buffer->updateAckTracker({delivery_tag, channel_id}); for (size_t i = 0; i < new_rows; ++i) { virtual_columns[0]->insert(exchange_name); - virtual_columns[1]->insert(consumer_tag); + virtual_columns[1]->insert(channel_id); virtual_columns[2]->insert(delivery_tag); virtual_columns[3]->insert(redelivered); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index d6b6ab440b2..835ded1718c 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -20,18 +20,16 @@ RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : ///Method that is called when the connection ends up in an error state. void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * message) { - connection_running.store(false); LOG_ERROR(log, "Library error report: {}", message); - + connection_running.store(false); if (connection) connection->close(); } void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) { - connection_running.store(true); LOG_TRACE(log, "Connection is ready"); - + connection_running.store(true); loop_state.store(Loop::RUN); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index d12d08fad25..833382f354b 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -21,7 +21,8 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - size_t channel_id_, + size_t channel_id_base_, + const String & channel_base_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, @@ -34,14 +35,15 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , setup_channel(setup_channel_) , event_handler(event_handler_) , exchange_name(exchange_name_) - , channel_id(channel_id_) + , channel_base(channel_base_) + , channel_id_base(channel_id_base_) , queue_base(queue_base_) , hash_exchange(hash_exchange_) , num_queues(num_queues_) + , deadletter_exchange(deadletter_exchange_) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) - , deadletter_exchange(deadletter_exchange_) , received(QUEUE_SIZE * num_queues) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) @@ -49,27 +51,32 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( consumer_channel->onReady([&]() { + channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + LOG_TRACE(log, "Channel {} is created", channel_id); + consumer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Consumer {} error: {}", channel_id, message); + LOG_ERROR(log, "Channel {} error: {}", channel_id, message); channel_error.store(true); }); + updateAckTracker(AckTracker()); subscribe(); + + channel_error.store(false); }); } ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { - consumer_channel->close(); BufferBase::set(nullptr, 0, 0); } void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) { - std::atomic bindings_created = false, bindings_error = false; + std::atomic binding_created = false; auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) { @@ -83,22 +90,20 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for * fanout exchange it can be arbitrary. */ - setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id)) + setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id_base)) .onSuccess([&] { - bindings_created = true; + binding_created = true; }) .onError([&](const char * message) { - bindings_error = true; - LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); + throw Exception("Failed to create queue binding. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); }; auto error_callback([&](const char * message) { - bindings_error = true; - LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); + throw Exception("Failed to declare queue. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); AMQP::Table queue_settings; @@ -110,10 +115,10 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) /* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one * specific queue when its name is specified in queue_base setting. */ - const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); + const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(queue_id); setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - while (!bindings_created && !bindings_error) + while (!binding_created) { iterateEventLoop(); } @@ -125,11 +130,9 @@ void ReadBufferFromRabbitMQConsumer::subscribe() for (const auto & queue_name : queues) { consumer_channel->consume(queue_name) - .onSuccess([&](const std::string & consumer) + .onSuccess([&](const std::string & /* consumer_tag */) { - if (consumer_tag.empty()) - consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { @@ -139,12 +142,12 @@ void ReadBufferFromRabbitMQConsumer::subscribe() if (row_delimiter != '\0') message_received += row_delimiter; - received.push({delivery_tag, message_received, redelivered}); + received.push({message_received, redelivered, AckTracker(delivery_tag, channel_id)}); } }) .onError([&](const char * message) { - LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); + LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); }); } } @@ -152,16 +155,63 @@ void ReadBufferFromRabbitMQConsumer::subscribe() void ReadBufferFromRabbitMQConsumer::ackMessages() { - UInt64 delivery_tag = last_inserted_delivery_tag; - if (delivery_tag && delivery_tag > prev_tag) + /* Delivery tags are scoped per channel, so if channel fails, then all previous delivery tags become invalid. Also this check ensures + * that there is no data race with onReady callback in restoreChannel() (they can be called at the same time from different threads). + * And there is no need to synchronize this method with updateAckTracker() as they are not supposed to be called at the same time. + */ + if (channel_error.load()) + return; + + AckTracker record = last_inserted_record; + + /// Do not send ack to server if message's channel is not the same as current running channel. + if (record.channel_id == channel_id && record.delivery_tag && record.delivery_tag > prev_tag) { - prev_tag = delivery_tag; - consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", channel_id, prev_tag); + consumer_channel->ack(record.delivery_tag, AMQP::multiple); /// Will ack all up to last tag starting from last acked. + prev_tag = record.delivery_tag; + + LOG_TRACE(log, "Consumer acknowledged messages with deliveryTags up to {} on the channel {}", record.delivery_tag, channel_id); } } +void ReadBufferFromRabbitMQConsumer::updateAckTracker(AckTracker record) +{ + /* This method can be called from readImpl and from channel->onError() callback, but channel_error check ensures that it is not done + * at the same time, so no synchronization needed. + */ + if (record.delivery_tag && channel_error.load()) + return; + + if (!record.delivery_tag) + prev_tag = 0; + + last_inserted_record = record; +} + + +void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) +{ + consumer_channel = std::move(new_channel); + consumer_channel->onReady([&]() + { + channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + LOG_TRACE(log, "Channel {} is created", channel_id); + + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Channel {} error: {}", channel_id, message); + channel_error.store(true); + }); + + updateAckTracker(AckTracker()); + subscribe(); + + channel_error.store(false); + }); +} + + void ReadBufferFromRabbitMQConsumer::iterateEventLoop() { event_handler->iterateLoop(); @@ -185,26 +235,4 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() return false; } - -void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) -{ - if (consumer_channel->usable()) - return; - - consumer_channel = std::move(new_channel); - consumer_channel->onReady([&]() - { - LOG_TRACE(log, "Channel {} is restored", channel_id); - channel_error.store(false); - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); - channel_error.store(true); - }); - - subscribe(); - }); -} - - } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index d3f560fad3b..c5643cb59f4 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -27,7 +27,8 @@ public: ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - size_t channel_id_, + size_t channel_id_base_, + const String & channel_base_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, @@ -38,53 +39,65 @@ public: ~ReadBufferFromRabbitMQConsumer() override; - struct MessageData + struct AckTracker { UInt64 delivery_tag; + String channel_id; + + AckTracker() : delivery_tag(0), channel_id("") {} + AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} + }; + + struct MessageData + { String message; bool redelivered; + AckTracker track; }; void allowNext() { allowed = true; } // Allow to read next message. bool channelUsable() { return !channel_error.load(); } void restoreChannel(ChannelPtr new_channel); - void updateNextDeliveryTag(UInt64 delivery_tag) { last_inserted_delivery_tag = delivery_tag; } - void ackMessages(); - auto getConsumerTag() const { return consumer_tag; } - auto getDeliveryTag() const { return current.delivery_tag; } + void ackMessages(); + void updateAckTracker(AckTracker record); + + auto getChannelID() const { return current.track.channel_id; } + auto getDeliveryTag() const { return current.track.delivery_tag; } auto getRedelivered() const { return current.redelivered; } private: + bool nextImpl() override; + + void bindQueue(size_t queue_id); + void subscribe(); + void iterateEventLoop(); + ChannelPtr consumer_channel; ChannelPtr setup_channel; HandlerPtr event_handler; const String exchange_name; - const size_t channel_id; + const String channel_base; + const size_t channel_id_base; const String queue_base; const bool hash_exchange; const size_t num_queues; + const String deadletter_exchange; Poco::Logger * log; char row_delimiter; bool allowed = true; const std::atomic & stopped; - const String deadletter_exchange; - std::atomic channel_error = false; - - String consumer_tag; - ConcurrentBoundedQueue received; - UInt64 last_inserted_delivery_tag = 0, prev_tag = 0; - MessageData current; + String channel_id; + std::atomic channel_error = true; std::vector queues; + ConcurrentBoundedQueue received; + MessageData current; - bool nextImpl() override; - - void bindQueue(size_t queue_id); - void subscribe(); - void iterateEventLoop(); + AckTracker last_inserted_record; + UInt64 prev_tag = 0, channel_id_counter = 0; }; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index a477477fd63..f85f7d6b59c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -110,7 +110,8 @@ StorageRabbitMQ::StorageRabbitMQ( if (!connection->ready()) { - uv_loop_close(loop.get()); + if (!connection->closed()) + connection->close(true); throw Exception("Cannot connect to RabbitMQ", ErrorCodes::CANNOT_CONNECT_RABBITMQ); } @@ -145,7 +146,9 @@ StorageRabbitMQ::StorageRabbitMQ( if (queue_base.empty()) { - /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name + /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to + * be table_name and not just a random string, because local exchanges should be declared the same for same tables. + */ sharding_exchange = exchange_name + "_" + table_name; /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better @@ -159,13 +162,19 @@ StorageRabbitMQ::StorageRabbitMQ( { /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need - * to share sharding exchange. + * to share sharding exchange and bridge exchange. */ sharding_exchange = exchange_name + "_" + queue_base; } bridge_exchange = sharding_exchange + "_bridge"; + /* Generate a random string, which will be used for channelID's, which must be unique to tables and to channels within each table. + * (Cannot use table_name here because it must be a different string if table was restored) + */ + unique_strbase = getRandomName(); + + /// One looping task for all consumers as they share the same connection == the same handler == the same event loop looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); looping_task->deactivate(); @@ -195,21 +204,21 @@ void StorageRabbitMQ::loopingFunc() void StorageRabbitMQ::initExchange() { - /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which - * will evenly distribute messages between all consumers. + /* Binding scheme is the following: client's exchange -> key bindings by routing key list -> bridge exchange (fanout) -> + * -> sharding exchange (only if needed) -> queues. */ setup_channel->declareExchange(exchange_name, exchange_type, AMQP::durable) .onError([&](const char * message) { throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " - + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + + std::string(message), ErrorCodes::LOGICAL_ERROR); }); - /// Bridge exchange is needed to easily disconnect consumer queues. Also simplifies queue bindings a lot. + /// Bridge exchange is needed to easily disconnect consumer queues and also simplifies queue bindings a lot. setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) .onError([&](const char * message) { - throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); if (!hash_exchange) @@ -218,20 +227,23 @@ void StorageRabbitMQ::initExchange() return; } - /// Declare exchange for sharding. + /* Change hash property because by default it will be routing key, which has to be an integer, but with support for any exchange + * type - routing keys will not be such. + */ AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; + /// Declare exchange for sharding. setup_channel->declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) .onError([&](const char * message) { - throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); setup_channel->bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); consumer_exchange = sharding_exchange; @@ -260,7 +272,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) @@ -272,7 +284,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } else @@ -288,7 +300,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } } @@ -302,6 +314,16 @@ void StorageRabbitMQ::bindExchange() void StorageRabbitMQ::unbindExchange() { + /* This is needed because with RabbitMQ can't (without special adjustments) use the same table for reading and writing (alternating them), + * because publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in + * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as + * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible + * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local + * queues, then messages will go both ways and in one of them they will remain not consumed. Therefore, if insert query is called, need + * to desconnect local consumers, but then MV cannot be afterwards created on the same table. It can be reverted to allow alternating + * these queries, but it will be ugly and seems pointless because probably nobody uses tables alternating INSERT and MV queries on the + * same table. + */ std::call_once(flag, [&]() { setup_channel->removeExchange(bridge_exchange) @@ -333,8 +355,9 @@ bool StorageRabbitMQ::restoreConnection() /// This lock is to synchronize with getChannel(). std::lock_guard lk(connection_mutex); - if (!connection->usable() || !connection->ready()) + if (!event_handler->connectionRunning()) { + /// Stopping loop now and not right after connection error, because need to run it to let it properly close connection. if (event_handler->getLoopState() == Loop::RUN) { event_handler->updateLoopState(Loop::STOP); @@ -342,12 +365,12 @@ bool StorageRabbitMQ::restoreConnection() heartbeat_task->deactivate(); } - /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) - * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP - * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + /* connection->close() is called in onError() method (which is called by the AMQP library when a fatal error occurs on the + * connection) inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then + * an AMQP closing-handshake is performed). But cannot open a new connection untill previous one is properly closed. */ size_t cnt_retries = 0; - while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + while (!connection->closed() && ++cnt_retries != RETRIES_MAX) event_handler->iterateLoop(); /// This will force immediate closure if not yet closed. @@ -355,7 +378,8 @@ bool StorageRabbitMQ::restoreConnection() connection->close(true); LOG_TRACE(log, "Trying to restore consumer connection"); - connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + connection = std::make_shared(event_handler.get(), + AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) @@ -386,6 +410,7 @@ ChannelPtr StorageRabbitMQ::getChannel() { std::lock_guard lk(connection_mutex); ChannelPtr new_channel = std::make_shared(connection.get()); + return new_channel; } @@ -466,12 +491,21 @@ void StorageRabbitMQ::shutdown() streaming_task->deactivate(); heartbeat_task->deactivate(); - for (size_t i = 0; i < num_created_consumers; ++i) + connection->close(); + + size_t cnt_retries = 0; + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) { - popReadBuffer(); + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP >> 3)); } - connection->close(); + /// Should actually force closure, if not yet closed, but it generates distracting error logs. + //if (!connection->closed()) + // connection->close(true); + + for (size_t i = 0; i < num_created_consumers; ++i) + popReadBuffer(); } @@ -514,8 +548,8 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, setup_channel, event_handler, consumer_exchange, - ++consumer_id, queue_base, log, row_delimiter, hash_exchange, num_queues, + consumer_channel, setup_channel, event_handler, consumer_exchange, ++consumer_id, + unique_strbase, queue_base, log, row_delimiter, hash_exchange, num_queues, deadletter_exchange, stream_cancelled); } @@ -524,7 +558,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - ++producer_id, use_transactional_channel, persistent, log, + ++producer_id, unique_strbase, use_transactional_channel, persistent, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } @@ -862,7 +896,7 @@ NamesAndTypesList StorageRabbitMQ::getVirtuals() const { return NamesAndTypesList{ {"_exchange_name", std::make_shared()}, - {"_consumer_tag", std::make_shared()}, + {"_channel_id", std::make_shared()}, {"_delivery_tag", std::make_shared()}, {"_redelivered", std::make_shared()} }; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index cfdbf1c3d94..0960e35d3bf 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -9,8 +9,10 @@ #include #include #include +#include #include #include +#include namespace DB @@ -111,6 +113,7 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers + String unique_strbase; String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; size_t producer_id = 0, consumer_id = 0; @@ -135,6 +138,15 @@ private: void pingConnection() { connection->heartbeat(); } bool streamToViews(); bool checkDependencies(const StorageID & table_id); + + String getRandomName() + { + std::uniform_int_distribution distribution('a', 'z'); + String random_str(32, ' '); + for (auto & c : random_str) + c = distribution(thread_local_rng); + return random_str; + } }; } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 78920bc13c6..2b818f0341f 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -15,7 +15,8 @@ namespace DB static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -static const auto BATCH = 512; +static const auto BATCH = 10000; +static const auto RETURNED_LIMIT = 50000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address_, @@ -24,8 +25,9 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const Names & routing_keys_, const String & exchange_name_, const AMQP::ExchangeType exchange_type_, - const size_t channel_id_, - const bool use_tx_, + const size_t channel_id_base_, + const String channel_base_, + const bool use_txn_, const bool persistent_, Poco::Logger * log_, std::optional delimiter, @@ -37,11 +39,12 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , routing_keys(routing_keys_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) - , channel_id(std::to_string(channel_id_)) - , use_tx(use_tx_) + , channel_id_base(std::to_string(channel_id_base_)) + , channel_base(channel_base_) + , use_txn(use_txn_) , persistent(persistent_) , payloads(BATCH) - , returned(BATCH << 6) + , returned(RETURNED_LIMIT) , log(log_) , delim(delimiter) , max_rows(rows_per_message) @@ -52,14 +55,14 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( uv_loop_init(loop.get()); event_handler = std::make_unique(loop.get(), log); - /* New coonection for each producer buffer because cannot publish from different threads with the same connection. - * (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) - */ if (setupConnection(false)) setupChannel(); - writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); - writing_task->deactivate(); + if (!use_txn) + { + writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); + writing_task->deactivate(); + } if (exchange_type == AMQP::ExchangeType::headers) { @@ -77,6 +80,14 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() { writing_task->deactivate(); connection->close(); + + size_t cnt_retries = 0; + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + { + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP >> 3)); + } + assert(rows == 0 && chunks.empty()); } @@ -103,7 +114,7 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - if (!use_tx) + if (!use_txn) { /// "publisher confirms" will be used, this is default. ++payload_counter; @@ -125,7 +136,7 @@ bool WriteBufferToRabbitMQProducer::setupConnection(bool reconnecting) { /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP - * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed. */ while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) event_handler->iterateLoop(); @@ -154,17 +165,19 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Producer error: {}", message); + LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) producer_channel->close(); - /// Records that have not received ack/nack from server before channel closure. + /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once + * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid. + */ for (const auto & record : delivery_record) returned.tryPush(record.second); - LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} waiting to be published, {} will be republished", - delivery_record.size(), payloads.size(), returned.size()); + LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", + channel_id, delivery_record.size(), payloads.size()); /// Delivery tags are scoped per channel. delivery_record.clear(); @@ -173,9 +186,10 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->onReady([&]() { - LOG_DEBUG(log, "Producer channel is ready"); + channel_id = channel_base + "_" + channel_id_base + std::to_string(channel_id_counter++); + LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); - if (use_tx) + if (use_txn) { producer_channel->startTransaction(); } @@ -238,27 +252,31 @@ void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) { std::pair payload; - while (!messages.empty() && producer_channel->usable()) + + /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged + * messages cannot exceed returned.size(), because they all might end up there. + */ + while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) { messages.pop(payload); AMQP::Envelope envelope(payload.second.data(), payload.second.size()); - /// if headers exchange - routing keys are added here via headers, else - it is just empty. + /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty. AMQP::Table message_settings = key_arguments; /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the - * server, then it means that publisher will never now whether those messages were delivered or not, and therefore those records + * server, then it means that publisher will never know whether those messages were delivered or not, and therefore those records * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata. */ message_settings["republished"] = std::to_string(republishing); - envelope.setHeaders(message_settings); - /* Adding here a message_id property to message metadata. - * (https://stackoverflow.com/questions/59384305/rabbitmq-how-to-handle-unwanted-duplicate-un-ack-message-after-connection-lost) + /* Adding here a messageID property to message metadata. Since RabbitMQ does not guarantee excatly-once delivery, then on the + * consumer side "republished" field of message metadata can be checked and, if it set to 1, consumer might also check "messageID" + * property. This way detection of duplicates is guaranteed. */ - envelope.setMessageID(channel_id + "-" + std::to_string(payload.first)); + envelope.setMessageID(std::to_string(payload.first)); /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) @@ -277,10 +295,11 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueuepublish(exchange_name, routing_keys[0], envelope); } + /// This is needed for "publisher confirms", which guarantees at-least-once delivery. ++delivery_tag; delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - /// Need to break at some point to let event loop run, because no publishing actually happend before looping. + /// Need to break at some point to let event loop run, because no publishing actually happens before looping. if (delivery_tag % BATCH == 0) break; } @@ -291,33 +310,30 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable()) - publish(returned, true); - else if (!payloads.empty() && producer_channel->usable()) - publish(payloads, false); - } + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished + * as fast as possible and no new publishes are made before returned messages are handled. Also once payloads.queue lacks space + * - push attemt will block thread in countRow() - this is intended. + */ + if (!returned.empty() && producer_channel->usable()) + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); iterateEventLoop(); + /* wait_num != 0 if there will be no new payloads pushed to payloads.queue in countRow(), delivery_record is empty if there are + * no more pending acknowldgements from the server (if receieved ack(), records are deleted, if received nack(), records are pushed + * to returned.queue and deleted, because server will attach new delivery tags to them). + */ if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) wait_all = false; else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection(true))) setupChannel(); } - LOG_DEBUG(log, "Processing ended"); + LOG_DEBUG(log, "Prodcuer on channel {} completed", channel_id); } @@ -353,7 +369,7 @@ void WriteBufferToRabbitMQProducer::commit() * RabbitMQ transactions seem not trust-worthy at all - see https://www.rabbitmq.com/semantics.html. Seems like its best to always * use "publisher confirms" rather than transactions (and by default it is so). Probably even need to delete this option. */ - if (!use_tx || !producer_channel->usable()) + if (!use_txn || !producer_channel->usable()) return; std::atomic answer_received = false, wait_rollback = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 95d505bafd5..85c90cd0ce9 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -25,7 +25,8 @@ public: const String & exchange_name_, const AMQP::ExchangeType exchange_type_, const size_t channel_id_, - const bool use_tx_, + const String channel_base_, + const bool use_txn_, const bool persistent_, Poco::Logger * log_, std::optional delimiter, @@ -55,8 +56,9 @@ private: const Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - const String channel_id; - const bool use_tx; + const String channel_id_base; + const String channel_base; + const bool use_txn; const bool persistent; AMQP::Table key_arguments; @@ -67,12 +69,14 @@ private: std::unique_ptr connection; std::unique_ptr producer_channel; + String channel_id; ConcurrentBoundedQueue> payloads, returned; UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; UInt64 payload_counter = 0; std::map> delivery_record; + UInt64 channel_id_counter = 0; Poco::Logger * log; const std::optional delim; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index a670ea8ab54..bb65319a3be 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -496,7 +496,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): DROP TABLE test.view; ''') - assert int(result) == rabbitmq_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == rabbitmq_messages * batch_messages, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) @@ -516,12 +516,12 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): rabbitmq_row_delimiter = '\\n'; DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) ENGINE = MergeTree ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq; + SELECT *, _channel_id AS channel_id FROM test.rabbitmq; ''') i = [0] @@ -541,8 +541,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): for message in messages: current += 1 mes_id = str(current) - channel.basic_publish(exchange='test_sharding', routing_key='', - properties=pika.BasicProperties(message_id=mes_id), body=message) + channel.basic_publish(exchange='test_sharding', routing_key='', properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -561,7 +560,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): if int(result1) == messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + result2 = instance.query("SELECT count(DISTINCT channel_id) FROM test.view") for thread in threads: thread.join() @@ -1153,7 +1152,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): def test_rabbitmq_hash_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.destination(key UInt64, value UInt64, channel_id String) ENGINE = MergeTree() ORDER BY key; ''') @@ -1175,7 +1174,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS - SELECT key, value, _consumer_tag AS consumer_tag FROM test.{0}; + SELECT key, value, _channel_id AS channel_id FROM test.{0}; '''.format(table_name)) i = [0] @@ -1215,7 +1214,8 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): time.sleep(1) if int(result1) == messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") + + result2 = instance.query("SELECT count(DISTINCT channel_id) FROM test.destination") for consumer_id in range(num_tables): table_name = 'rabbitmq_consumer{}'.format(consumer_id) @@ -1413,7 +1413,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): rabbitmq_exchange_name = 'virtuals', rabbitmq_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; + SELECT value, key, _exchange_name, _channel_id, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; ''') credentials = pika.PlainCredentials('root', 'clickhouse') @@ -1422,11 +1422,11 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): channel = connection.channel() message_num = 10 - i = [0] + i = 0 messages = [] for _ in range(message_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 for message in messages: channel.basic_publish(exchange='virtuals', routing_key='', body=message) @@ -1439,33 +1439,28 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): connection.close() - result = instance.query("SELECT count(DISTINCT _delivery_tag) FROM test.view") - assert int(result) == 10 - - result = instance.query("SELECT count(DISTINCT _consumer_tag) FROM test.view") - assert int(result) == 1 - result = instance.query(''' - SELECT key, value, _exchange_name, SUBSTRING(_consumer_tag, 1, 8), _delivery_tag, _redelivered - FROM test.view - ORDER BY key + SELECT key, value, _exchange_name, SUBSTRING(_channel_id, 34, 3), _delivery_tag, _redelivered + FROM test.view ORDER BY key ''') expected = '''\ -0 0 virtuals amq.ctag 1 0 -1 1 virtuals amq.ctag 2 0 -2 2 virtuals amq.ctag 3 0 -3 3 virtuals amq.ctag 4 0 -4 4 virtuals amq.ctag 5 0 -5 5 virtuals amq.ctag 6 0 -6 6 virtuals amq.ctag 7 0 -7 7 virtuals amq.ctag 8 0 -8 8 virtuals amq.ctag 9 0 -9 9 virtuals amq.ctag 10 0 +0 0 virtuals 1_0 1 0 +1 1 virtuals 1_0 2 0 +2 2 virtuals 1_0 3 0 +3 3 virtuals 1_0 4 0 +4 4 virtuals 1_0 5 0 +5 5 virtuals 1_0 6 0 +6 6 virtuals 1_0 7 0 +7 7 virtuals 1_0 8 0 +8 8 virtuals 1_0 9 0 +9 9 virtuals 1_0 10 0 ''' + instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv ''') + assert TSV(result) == TSV(expected) @@ -1480,10 +1475,10 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): rabbitmq_exchange_name = 'virtuals_mv', rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value UInt64, - exchange_name String, consumer_tag String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() + exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _exchange_name as exchange_name, _consumer_tag as consumer_tag, _delivery_tag as delivery_tag, _redelivered as redelivered + SELECT *, _exchange_name as exchange_name, _channel_id as channel_id, _delivery_tag as delivery_tag, _redelivered as redelivered FROM test.rabbitmq_virtuals_mv; ''') @@ -1493,11 +1488,11 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): channel = connection.channel() message_num = 10 - i = [0] + i = 0 messages = [] for _ in range(message_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 for message in messages: channel.basic_publish(exchange='virtuals_mv', routing_key='', body=message) @@ -1510,24 +1505,18 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): connection.close() - result = instance.query("SELECT count(DISTINCT delivery_tag) FROM test.view") - assert int(result) == 10 - - result = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") - assert int(result) == 1 - - result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(channel_id, 34, 3), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") expected = '''\ -0 0 virtuals_mv amq.ctag 1 0 -1 1 virtuals_mv amq.ctag 2 0 -2 2 virtuals_mv amq.ctag 3 0 -3 3 virtuals_mv amq.ctag 4 0 -4 4 virtuals_mv amq.ctag 5 0 -5 5 virtuals_mv amq.ctag 6 0 -6 6 virtuals_mv amq.ctag 7 0 -7 7 virtuals_mv amq.ctag 8 0 -8 8 virtuals_mv amq.ctag 9 0 -9 9 virtuals_mv amq.ctag 10 0 +0 0 virtuals_mv 1_0 1 0 +1 1 virtuals_mv 1_0 2 0 +2 2 virtuals_mv 1_0 3 0 +3 3 virtuals_mv 1_0 4 0 +4 4 virtuals_mv 1_0 5 0 +5 5 virtuals_mv 1_0 6 0 +6 6 virtuals_mv 1_0 7 0 +7 7 virtuals_mv 1_0 8 0 +8 8 virtuals_mv 1_0 9 0 +9 9 virtuals_mv 1_0 10 0 ''' instance.query(''' @@ -1540,7 +1529,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_queue_resume_1(rabbitmq_cluster): +def test_rabbitmq_queue_resume(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1548,54 +1537,6 @@ def test_rabbitmq_queue_resume_1(rabbitmq_cluster): rabbitmq_exchange_name = 'queue_resume', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_num_consumers = '2', - rabbitmq_num_queues = '2', - rabbitmq_queue_base = 'queue_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - ''') - - i = [0] - messages_num = 1000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - for message in messages: - channel.basic_publish(exchange='queue_resume', routing_key='queue_resume', body=message, - properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - - threads = [] - threads_num = 10 - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_queue_resume; - ''') - - instance.query(''' - CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'queue_resume', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_num_consumers = '2', - rabbitmq_num_queues = '2', rabbitmq_queue_base = 'queue_resume', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -1608,42 +1549,6 @@ def test_rabbitmq_queue_resume_1(rabbitmq_cluster): SELECT * FROM test.rabbitmq_queue_resume; ''') - while True: - result1 = instance.query('SELECT count() FROM test.view') - time.sleep(1) - if int(result1) == messages_num * threads_num: - break - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_queue_resume; - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; - ''') - - assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - -@pytest.mark.timeout(420) -def test_rabbitmq_queue_resume_2(rabbitmq_cluster): - instance.query(''' - CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'queue_resume', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_queue_base = 'queue_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_queue_resume; - ''') - i = [0] messages_num = 10000 @@ -1696,7 +1601,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): while True: result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) - if int(result1) == messages_num * threads_num: + if int(result1) >= messages_num * threads_num: break instance.query(''' @@ -1705,59 +1610,44 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): DROP TABLE IF EXISTS test.view; ''') - assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) -def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): +def test_rabbitmq_no_loss_on_table_drop(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'consumer_acks', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'consumer_acks', rabbitmq_queue_base = 'consumer_resume', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') - i = [0] - messages_num = 5000 + i = 0 + messages_num = 100000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - for message in messages: - channel.basic_publish(exchange='consumer_acks', routing_key='consumer_acks', body=message, - properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - - threads = [] - threads_num = 20 - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 + for message in messages: + channel.basic_publish(exchange='consumer_acks', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() instance.query(''' DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_consumer_acks; + SELECT * FROM test.rabbitmq_consumer_acks; ''') while int(instance.query('SELECT count() FROM test.view')) == 0: @@ -1779,27 +1669,25 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): ''') while True: - result1 = instance.query('SELECT count() FROM test.view') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - #print("receiived", result1, "collected", collected) - if int(result1) >= messages_num * threads_num: + if int(result) == messages_num: break instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; DROP TABLE IF EXISTS test.consumer; DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; ''') - # >= because at-least-once - assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.destination(key UInt64, value UInt64, channel_id String) ENGINE = MergeTree() ORDER BY key; ''') @@ -1820,7 +1708,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS - SELECT key, value, _consumer_tag as consumer_tag FROM test.many_consumers_{0}; + SELECT key, value, _channel_id as channel_id FROM test.many_consumers_{0}; '''.format(table_id)) i = [0] @@ -1860,7 +1748,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): if int(result1) == messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") + result2 = instance.query("SELECT count(DISTINCT channel_id) FROM test.destination") for thread in threads: thread.join() @@ -1881,7 +1769,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): +def test_rabbitmq_consumer_restore_failed_connection_without_losses(rabbitmq_cluster): instance.query(''' CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1891,32 +1779,21 @@ def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): rabbitmq_row_delimiter = '\\n'; ''') - i = [0] - messages_num = 5000 + i = 0 + messages_num = 100000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - for message in messages: - channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - threads = [] - threads_num = 20 - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 + for message in messages: + channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() instance.query(''' DROP TABLE IF EXISTS test.view; @@ -1929,19 +1806,18 @@ def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): ''') while int(instance.query('SELECT count() FROM test.view')) == 0: - time.sleep(1) + time.sleep(0.1) kill_rabbitmq(); time.sleep(4); revive_rabbitmq(); - collected = int(instance.query('SELECT count() FROM test.view')) + #collected = int(instance.query('SELECT count() FROM test.view')) while True: - result = instance.query('SELECT count() FROM test.view') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - print("receiived", result, "collected", collected) - if int(result) >= messages_num * threads_num: + if int(result) == messages_num: break instance.query(''' @@ -1950,33 +1826,26 @@ def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): DROP TABLE IF EXISTS test.consumer_reconnect; ''') - # >= because at-least-once - assert int(result) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) -def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): - instance.query(''' - DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - ''') - +def test_rabbitmq_producer_restore_failed_connection_without_losses(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.consume; - DROP TABLE IF EXISTS test.consume_mv; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; CREATE TABLE test.consume (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'producer_reconnect', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.consume_mv TO test.destination AS - SELECT key, value FROM test.consume; - ''') - - instance.query(''' + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.consume; DROP TABLE IF EXISTS test.producer_reconnect; CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -2008,7 +1877,7 @@ def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): else: raise - while int(instance.query('SELECT count() FROM test.destination')) == 0: + while int(instance.query('SELECT count() FROM test.view')) == 0: time.sleep(0.1) kill_rabbitmq(); @@ -2016,20 +1885,87 @@ def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): revive_rabbitmq(); while True: - result = instance.query('SELECT count() FROM test.destination') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - print(result, messages_num) - if int(result) >= messages_num: + if int(result) == messages_num: break instance.query(''' - DROP TABLE IF EXISTS test.consume_mv; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consume; DROP TABLE IF EXISTS test.producer_reconnect; - DROP TABLE IF EXISTS test.destination; ''') - assert int(result) >= messages_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) + + +@pytest.mark.timeout(420) +def test_rabbitmq_virtual_columns_2(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.destination; + CREATE TABLE test.destination(key UInt64, value UInt64, + exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() + ORDER BY key; + ''') + + table_num = 3 + for table_id in range(table_num): + print("Setting up table {}".format(table_id)) + instance.query(''' + DROP TABLE IF EXISTS test.virtuals_{0}; + DROP TABLE IF EXISTS test.virtuals_{0}_mv; + CREATE TABLE test.virtuals_{0} (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'virtuals_2', + rabbitmq_num_queues = 2, + rabbitmq_num_consumers = 2, + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS + SELECT *, _exchange_name as exchange_name, _channel_id as channel_id, _delivery_tag as delivery_tag, _redelivered as redelivered + FROM test.virtuals_{0}; + '''.format(table_id)) + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + message_num = 10 + i = 0 + messages = [] + for _ in range(message_num): + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 + + for i in range(message_num): + channel.basic_publish(exchange='virtuals_2', routing_key='', body=messages[i], + properties=pika.BasicProperties(delivery_mode=2, message_id=str(i))) + + #kill_rabbitmq(); + #time.sleep(2); + #revive_rabbitmq(); + + while True: + result = instance.query('SELECT count(DISTINCT concat([channel_id], [toString(delivery_tag)])) FROM test.destination') + print instance.query(''' + SELECT DISTINCT concat([channel_id], [toString(delivery_tag)]) + FROM (SELECT channel_id AS id, delivery_tag AS tag FROM test.destination GROUP BY id ORDER BY tag)''') + time.sleep(1) + if int(result) == message_num * table_num: + break + + connection.close() + + instance.query(''' + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + ''') + + assert int(result) == message_num * table_num if __name__ == '__main__': From 1006c4f11bce91b4fdf82f575f8f427828347805 Mon Sep 17 00:00:00 2001 From: "Ivan A. Torgashov" Date: Sat, 15 Aug 2020 14:18:17 +0500 Subject: [PATCH 048/535] Update tests for Redis dictionary requirepass authorization support --- .../integration/runner/compose/docker_compose_redis.yml | 1 + .../external_sources.py | 9 ++++++--- .../test_dictionaries_all_layouts_and_sources/test.py | 4 ++-- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_redis.yml b/docker/test/integration/runner/compose/docker_compose_redis.yml index 2dc79ed5910..2c9ace96d0c 100644 --- a/docker/test/integration/runner/compose/docker_compose_redis.yml +++ b/docker/test/integration/runner/compose/docker_compose_redis.yml @@ -5,3 +5,4 @@ services: restart: always ports: - 6380:6379 + command: redis-server --requirepass "clickhouse" diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py b/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py index f6985e7de54..fac7dcdea1e 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py @@ -483,23 +483,27 @@ class SourceRedis(ExternalSource): name, internal_hostname, internal_port, docker_hostname, docker_port, user, password ) self.storage_type = storage_type + self.db_index = 1 def get_source_str(self, table_name): return ''' {host} {port} - 0 + {password} + {db_index} {storage_type} '''.format( host=self.docker_hostname, port=self.docker_port, + password=self.password, storage_type=self.storage_type, # simple or hash_map + db_index=self.db_index, ) def prepare(self, structure, table_name, cluster): - self.client = redis.StrictRedis(host=self.internal_hostname, port=self.internal_port) + self.client = redis.StrictRedis(host=self.internal_hostname, port=self.internal_port, db=self.db_index, password=self.password or None) self.prepared = True self.ordered_names = structure.get_ordered_names() @@ -525,7 +529,6 @@ class SourceRedis(ExternalSource): return True return False - class SourceAerospike(ExternalSource): def __init__(self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password): diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py index f4b0ba9c1e4..994d8e5e65d 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py @@ -134,8 +134,8 @@ DICTIONARIES = [] # Key-value dictionaries with only one possible field for key SOURCES_KV = [ - SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "", storage_type="simple"), - SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "", storage_type="hash_map"), + SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", storage_type="simple"), + SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", storage_type="hash_map"), ] DICTIONARIES_KV = [] From 43839a97b6a214cdbeeb5d6fdbf8c9cccfbd5e95 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 15 Aug 2020 21:29:24 +0800 Subject: [PATCH 049/535] ISSUES-4006 add factor with DateTime type --- src/DataTypes/DataTypeDateTime.cpp | 27 ----- src/DataTypes/DataTypeDateTime64.cpp | 61 ---------- src/DataTypes/registerDataTypeDateTime.cpp | 110 ++++++++++++++++++ src/DataTypes/ya.make | 1 + src/Functions/FunctionsConversion.cpp | 1 + src/Functions/FunctionsConversion.h | 33 ++++++ .../01442_date_time_with_params.reference | 4 + .../01442_date_time_with_params.sql | 15 +++ 8 files changed, 164 insertions(+), 88 deletions(-) create mode 100644 src/DataTypes/registerDataTypeDateTime.cpp create mode 100644 tests/queries/0_stateless/01442_date_time_with_params.reference create mode 100644 tests/queries/0_stateless/01442_date_time_with_params.sql diff --git a/src/DataTypes/DataTypeDateTime.cpp b/src/DataTypes/DataTypeDateTime.cpp index c860766406e..9ea698d4fbb 100644 --- a/src/DataTypes/DataTypeDateTime.cpp +++ b/src/DataTypes/DataTypeDateTime.cpp @@ -185,31 +185,4 @@ bool DataTypeDateTime::equals(const IDataType & rhs) const return typeid(rhs) == typeid(*this); } -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -static DataTypePtr create(const ASTPtr & arguments) -{ - if (!arguments) - return std::make_shared(); - - if (arguments->children.size() != 1) - throw Exception("DateTime data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const auto * arg = arguments->children[0]->as(); - if (!arg || arg->value.getType() != Field::Types::String) - throw Exception("Parameter for DateTime data type must be string literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(arg->value.get()); -} - -void registerDataTypeDateTime(DataTypeFactory & factory) -{ - factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive); - factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); -} - } diff --git a/src/DataTypes/DataTypeDateTime64.cpp b/src/DataTypes/DataTypeDateTime64.cpp index 97dd28439d7..ee4139c2b7a 100644 --- a/src/DataTypes/DataTypeDateTime64.cpp +++ b/src/DataTypes/DataTypeDateTime64.cpp @@ -201,65 +201,4 @@ bool DataTypeDateTime64::equals(const IDataType & rhs) const return false; } -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -enum class ArgumentKind -{ - Optional, - Mandatory -}; - -template -std::conditional_t, T> -getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) -{ - using NearestResultType = NearestFieldType; - const auto field_type = Field::TypeToEnum::value; - const ASTLiteral * argument = nullptr; - - auto exception_message = [=](const String & message) - { - return std::string("Parameter #") + std::to_string(argument_index) + " '" - + argument_name + "' for " + context_data_type_name - + message - + ", expected: " + Field::Types::toString(field_type) + " literal."; - }; - - if (!arguments || arguments->children.size() <= argument_index - || !(argument = arguments->children[argument_index]->as())) - { - if constexpr (Kind == ArgumentKind::Optional) - return {}; - else - throw Exception(exception_message(" is missing"), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - if (argument->value.getType() != field_type) - throw Exception(exception_message(String(" has wrong type: ") + argument->value.getTypeName()), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return argument->value.get(); -} - -static DataTypePtr create64(const ASTPtr & arguments) -{ - if (!arguments || arguments->size() == 0) - return std::make_shared(DataTypeDateTime64::default_scale); - - const auto scale = getArgument(arguments, 0, "scale", "DateType64"); - const auto timezone = getArgument(arguments, !!scale, "timezone", "DateType64"); - - return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); -} - -void registerDataTypeDateTime64(DataTypeFactory & factory) -{ - factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); -} - } diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp new file mode 100644 index 00000000000..47487641e09 --- /dev/null +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -0,0 +1,110 @@ + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +enum class ArgumentKind +{ + Optional, + Mandatory +}; + +template +std::conditional_t, T> +getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) +{ + using NearestResultType = NearestFieldType; + const auto field_type = Field::TypeToEnum::value; + const ASTLiteral * argument = nullptr; + + auto exception_message = [=](const String & message) + { + return std::string("Parameter #") + std::to_string(argument_index) + " '" + + argument_name + "' for " + context_data_type_name + + message + + ", expected: " + Field::Types::toString(field_type) + " literal."; + }; + + if (!arguments || arguments->children.size() <= argument_index + || !(argument = arguments->children[argument_index]->as()) + || argument->value.getType() != field_type) + { + if constexpr (Kind == ArgumentKind::Optional) + return {}; + else + throw Exception(exception_message(" is missing"), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + return argument->value.get(); +} + +static DataTypePtr create(const ASTPtr & arguments) +{ + if (!arguments || arguments->size() == 0) + return std::make_shared(); + + const auto scale = getArgument(arguments, 0, "scale", "DateTime"); + const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime"); + + if (scale) + return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); + + return std::make_shared(timezone.value_or(String{})); +} + +static DataTypePtr create32(const ASTPtr & arguments) +{ + if (!arguments || arguments->size() == 0) + return std::make_shared(); + + if (arguments->children.size() != 1) + throw Exception("DateTime32 data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto timezone = getArgument(arguments, 0, "timezone", "DateTime32"); + + return std::make_shared(timezone); +} + +static DataTypePtr create64(const ASTPtr & arguments) +{ + if (!arguments || arguments->size() == 0) + return std::make_shared(DataTypeDateTime64::default_scale); + + if (arguments->children.size() > 2) + throw Exception("DateTime64 data type can optionally have two argument - scale and time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); + const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); + + return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); +} + +void registerDataTypeDateTime(DataTypeFactory & factory) +{ + factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive); + factory.registerDataType("DateTime32", create32, DataTypeFactory::CaseInsensitive); + factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); + + factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); +} + +void registerDataTypeDateTime64(DataTypeFactory & /*factory*/) +{ +// factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); +} + +} diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index 82e9baf76f2..4237ca920ae 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -38,6 +38,7 @@ SRCS( getMostSubtype.cpp IDataType.cpp NestedUtils.cpp + registerDataTypeDateTime.cpp ) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index da42c8a2623..804c16d946d 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -32,6 +32,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4aacafafd96..a8e8ad81ff8 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -968,6 +968,7 @@ struct ConvertImpl /// Declared early because used below. struct NameToDate { static constexpr auto name = "toDate"; }; struct NameToDateTime { static constexpr auto name = "toDateTime"; }; +struct NameToDateTime32 { static constexpr auto name = "toDateTime32"; }; struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; }; struct NameToString { static constexpr auto name = "toString"; }; struct NameToDecimal32 { static constexpr auto name = "toDecimal32"; }; @@ -1027,6 +1028,14 @@ public: { mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } + + if constexpr (std::is_same_v && std::is_same_v) + { + /// toDateTime(value, scale:Integer) + if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) + mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); + } + // toString(DateTime or DateTime64, [timezone: String]) if ((std::is_same_v && arguments.size() > 0 && (isDateTime64(arguments[0].type) || isDateTime(arguments[0].type))) // toUnixTimestamp(value[, timezone : String]) @@ -1076,6 +1085,17 @@ public: scale = static_cast(arguments[1].column->get64(0)); } + if constexpr (std::is_same_v && std::is_same_v) + { + /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 + if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) + { + timezone_arg_position += 1; + scale = static_cast(arguments[1].column->get64(0)); + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + } + } + if constexpr (std::is_same_v) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); else if constexpr (to_datetime64) @@ -1179,6 +1199,18 @@ private: return true; }; + if constexpr (std::is_same_v && std::is_same_v) + { + /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 + if ((arguments.size() == 2 && isUnsignedInteger(block.getByPosition(arguments[1]).type)) || arguments.size() == 3) + { + if (!callOnIndexAndDataType(from_type->getTypeId(), call)) + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return; + } + } + bool done = callOnIndexAndDataType(from_type->getTypeId(), call); if (!done) { @@ -1607,6 +1639,7 @@ using FunctionToFloat32 = FunctionConvert>; using FunctionToDate = FunctionConvert; using FunctionToDateTime = FunctionConvert; +using FunctionToDateTime32 = FunctionConvert; using FunctionToDateTime64 = FunctionConvert; using FunctionToUUID = FunctionConvert>; using FunctionToString = FunctionConvert; diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference new file mode 100644 index 00000000000..a6cb7f7b948 --- /dev/null +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -0,0 +1,4 @@ +2020-01-01 00:00:00 DateTime 2020-01-01 00:01:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:06:00 DateTime(\'Europe/Moscow\') +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') +2020-01-01 00:00:00 DateTime diff --git a/tests/queries/0_stateless/01442_date_time_with_params.sql b/tests/queries/0_stateless/01442_date_time_with_params.sql new file mode 100644 index 00000000000..1e75089bc05 --- /dev/null +++ b/tests/queries/0_stateless/01442_date_time_with_params.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (a DateTime, b DateTime(), c DateTime(2), d DateTime('Europe/Moscow'), e DateTime(3, 'Europe/Moscow'), f DateTime32, g DateTime32('Europe/Moscow')) ENGINE = MergeTree ORDER BY a; + +INSERT INTO test VALUES('2020-01-01 00:00:00', '2020-01-01 00:01:00', '2020-01-01 00:02:00.11', '2020-01-01 00:03:00', '2020-01-01 00:04:00.22', '2020-01-01 00:05:00', '2020-01-01 00:06:00') + +SELECT a, toTypeName(a), b, toTypeName(b), c, toTypeName(c), d, toTypeName(d), e, toTypeName(e), f, toTypeName(f), g, toTypeName(g) FROM test; + +SELECT toDateTime('2020-01-01 00:00:00') AS a, toTypeName(a), toDateTime('2020-01-01 00:02:00.11', 2) AS b, toTypeName(b), toDateTime('2020-01-01 00:03:00', 'Europe/Moscow') AS c, toTypeName(c), toDateTime('2020-01-01 00:04:00.22', 3, 'Europe/Moscow') AS d, toTypeName(d); + +SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-01-01 00:02:00.11', 'DateTime(2)') AS b, toTypeName(b), CAST('2020-01-01 00:03:00', 'DateTime(\'Europe/Moscow\')') AS c, toTypeName(c), CAST('2020-01-01 00:04:00.22', 'DateTime(3, \'Europe/Moscow\')') AS d, toTypeName(d); + +SELECT toDateTime32('2020-01-01 00:00:00') AS a, toTypeName(a); + +DROP TABLE IF EXISTS test; From 4ad267571e5454349e3fca00c9ec34d0c578e794 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 15 Aug 2020 21:43:44 +0800 Subject: [PATCH 050/535] ISSUES-4006 remove unused code --- src/DataTypes/DataTypeFactory.cpp | 1 - src/DataTypes/DataTypeFactory.h | 1 - src/DataTypes/registerDataTypeDateTime.cpp | 5 ----- 3 files changed, 7 deletions(-) diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 664927389b5..9386f4b39f1 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -165,7 +165,6 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDecimal(*this); registerDataTypeDate(*this); registerDataTypeDateTime(*this); - registerDataTypeDateTime64(*this); registerDataTypeString(*this); registerDataTypeFixedString(*this); registerDataTypeEnum(*this); diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 67b72945acc..ea77c50170c 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -82,7 +82,6 @@ void registerDataTypeInterval(DataTypeFactory & factory); void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); -void registerDataTypeDateTime64(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); } diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 47487641e09..c6a79e48335 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -102,9 +102,4 @@ void registerDataTypeDateTime(DataTypeFactory & factory) factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); } -void registerDataTypeDateTime64(DataTypeFactory & /*factory*/) -{ -// factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); -} - } From 1e7eb494812ffb48f77e6cb58e1f9f323050aa11 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 15 Aug 2020 14:38:29 +0000 Subject: [PATCH 051/535] Add one more mes property, support format_schema --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 6 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 18 ++- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 2 + src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 65 +++++--- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 + .../WriteBufferToRabbitMQProducer.cpp | 2 +- .../format_schemas/rabbitmq.proto | 6 + .../test_storage_rabbitmq/rabbitmq_pb2.py | 77 +++++++++ .../integration/test_storage_rabbitmq/test.py | 151 ++++++++++++------ 10 files changed, 257 insertions(+), 74 deletions(-) create mode 100644 tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto create mode 100644 tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index e26645a1168..16ba14094ac 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -22,8 +22,10 @@ RabbitMQBlockInputStream::RabbitMQBlockInputStream( , column_names(columns) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header(metadata_snapshot->getSampleBlockForColumns( - {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) + {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered", "_message_id"}, storage.getVirtuals(), storage.getStorageID())) { + if (!storage.getSchemaName().empty()) + context.setSetting("format_schema", storage.getSchemaName()); } @@ -131,6 +133,7 @@ Block RabbitMQBlockInputStream::readImpl() auto channel_id = buffer->getChannelID(); auto delivery_tag = buffer->getDeliveryTag(); auto redelivered = buffer->getRedelivered(); + auto message_id = buffer->getMessageID(); buffer->updateAckTracker({delivery_tag, channel_id}); @@ -140,6 +143,7 @@ Block RabbitMQBlockInputStream::readImpl() virtual_columns[1]->insert(channel_id); virtual_columns[2]->insert(delivery_tag); virtual_columns[3]->insert(redelivered); + virtual_columns[4]->insert(message_id); } total_rows = total_rows + new_rows; diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 5d15bd5b77d..2416a15f65a 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -14,6 +14,7 @@ namespace DB M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ M(String, rabbitmq_format, "", "The message format.", 0) \ M(Char, rabbitmq_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ + M(String, rabbitmq_schema, "", "Schema identifier (used by schema-based formats) for RabbitMQ engine", 0) \ M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \ M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 833382f354b..197b9f7e057 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static const auto QUEUE_SIZE = 50000; ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( @@ -51,7 +56,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( consumer_channel->onReady([&]() { - channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_TRACE(log, "Channel {} is created", channel_id); consumer_channel->onError([&](const char * message) @@ -142,7 +147,10 @@ void ReadBufferFromRabbitMQConsumer::subscribe() if (row_delimiter != '\0') message_received += row_delimiter; - received.push({message_received, redelivered, AckTracker(delivery_tag, channel_id)}); + if (message.hasMessageID()) + received.push({message_received, message.messageID(), redelivered, AckTracker(delivery_tag, channel_id)}); + else + received.push({message_received, "", redelivered, AckTracker(delivery_tag, channel_id)}); } }) .onError([&](const char * message) @@ -195,7 +203,11 @@ void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) consumer_channel = std::move(new_channel); consumer_channel->onReady([&]() { - channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + /* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer, + * i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that + * channel_id is unique for each table. + */ + channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_TRACE(log, "Channel {} is created", channel_id); consumer_channel->onError([&](const char * message) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index c5643cb59f4..e00e8172509 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -51,6 +51,7 @@ public: struct MessageData { String message; + String message_id; bool redelivered; AckTracker track; }; @@ -65,6 +66,7 @@ public: auto getChannelID() const { return current.track.channel_id; } auto getDeliveryTag() const { return current.track.delivery_tag; } auto getRedelivered() const { return current.redelivered; } + auto getMessageID() const { return current.message_id; } private: bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f85f7d6b59c..f82773ed367 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -69,6 +69,7 @@ StorageRabbitMQ::StorageRabbitMQ( const String & exchange_name_, const String & format_name_, char row_delimiter_, + const String & schema_name_, const String & exchange_type_, size_t num_consumers_, size_t num_queues_, @@ -83,6 +84,7 @@ StorageRabbitMQ::StorageRabbitMQ( , exchange_name(exchange_name_) , format_name(global_context.getMacros()->expand(format_name_)) , row_delimiter(row_delimiter_) + , schema_name(global_context.getMacros()->expand(schema_name_)) , num_consumers(num_consumers_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) @@ -785,13 +787,29 @@ void registerStorageRabbitMQ(StorageFactory & factory) } } - String exchange_type = rabbitmq_settings.rabbitmq_exchange_type.value; + String schema = rabbitmq_settings.rabbitmq_schema.value; if (args_count >= 6) { engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[5], args.local_context); const auto * ast = engine_args[5]->as(); if (ast && ast->value.getType() == Field::Types::String) + { + schema = safeGet(ast->value); + } + else + { + throw Exception("Format schema must be a string", ErrorCodes::BAD_ARGUMENTS); + } + } + + String exchange_type = rabbitmq_settings.rabbitmq_exchange_type.value; + if (args_count >= 7) + { + engine_args[6] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[6], args.local_context); + + const auto * ast = engine_args[6]->as(); + if (ast && ast->value.getType() == Field::Types::String) { exchange_type = safeGet(ast->value); } @@ -802,9 +820,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) } UInt64 num_consumers = rabbitmq_settings.rabbitmq_num_consumers; - if (args_count >= 7) + if (args_count >= 8) { - const auto * ast = engine_args[6]->as(); + const auto * ast = engine_args[7]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { num_consumers = safeGet(ast->value); @@ -816,9 +834,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) } UInt64 num_queues = rabbitmq_settings.rabbitmq_num_queues; - if (args_count >= 8) + if (args_count >= 9) { - const auto * ast = engine_args[7]->as(); + const auto * ast = engine_args[8]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { num_consumers = safeGet(ast->value); @@ -830,9 +848,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) } bool use_transactional_channel = static_cast(rabbitmq_settings.rabbitmq_transactional_channel); - if (args_count >= 9) + if (args_count >= 10) { - const auto * ast = engine_args[8]->as(); + const auto * ast = engine_args[9]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { use_transactional_channel = static_cast(safeGet(ast->value)); @@ -844,33 +862,33 @@ void registerStorageRabbitMQ(StorageFactory & factory) } String queue_base = rabbitmq_settings.rabbitmq_queue_base.value; - if (args_count >= 10) - { - engine_args[9] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[9], args.local_context); - - const auto * ast = engine_args[9]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - queue_base = safeGet(ast->value); - } - } - - String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; if (args_count >= 11) { engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); const auto * ast = engine_args[10]->as(); if (ast && ast->value.getType() == Field::Types::String) + { + queue_base = safeGet(ast->value); + } + } + + String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; + if (args_count >= 12) + { + engine_args[11] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[11], args.local_context); + + const auto * ast = engine_args[11]->as(); + if (ast && ast->value.getType() == Field::Types::String) { deadletter_exchange = safeGet(ast->value); } } bool persistent = static_cast(rabbitmq_settings.rabbitmq_persistent_mode); - if (args_count >= 12) + if (args_count >= 13) { - const auto * ast = engine_args[11]->as(); + const auto * ast = engine_args[12]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { persistent = static_cast(safeGet(ast->value)); @@ -883,7 +901,7 @@ void registerStorageRabbitMQ(StorageFactory & factory) return StorageRabbitMQ::create( args.table_id, args.context, args.columns, - host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, + host_port, routing_keys, exchange, format, row_delimiter, schema, exchange_type, num_consumers, num_queues, use_transactional_channel, queue_base, deadletter_exchange, persistent); }; @@ -898,7 +916,8 @@ NamesAndTypesList StorageRabbitMQ::getVirtuals() const {"_exchange_name", std::make_shared()}, {"_channel_id", std::make_shared()}, {"_delivery_tag", std::make_shared()}, - {"_redelivered", std::make_shared()} + {"_redelivered", std::make_shared()}, + {"_message_id", std::make_shared()} }; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 0960e35d3bf..60bc1aa7157 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -55,6 +55,7 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; + const auto & getSchemaName() const { return schema_name; } const String getExchange() const { return exchange_name; } bool checkBridge() const { return !exchange_removed.load(); } @@ -74,6 +75,7 @@ protected: const String & exchange_name_, const String & format_name_, char row_delimiter_, + const String & schema_name_, const String & exchange_type_, size_t num_consumers_, size_t num_queues_, @@ -92,6 +94,7 @@ private: const String format_name; char row_delimiter; + const String schema_name; size_t num_consumers; size_t num_created_consumers = 0; bool hash_exchange; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 2b818f0341f..8cd769e792f 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -186,7 +186,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->onReady([&]() { - channel_id = channel_base + "_" + channel_id_base + std::to_string(channel_id_counter++); + channel_id = channel_id_base + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); if (use_txn) diff --git a/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto b/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto new file mode 100644 index 00000000000..96b24be4938 --- /dev/null +++ b/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message KeyValuePair { + uint64 key = 1; + string value = 2; +} \ No newline at end of file diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py new file mode 100644 index 00000000000..fb0f1413eac --- /dev/null +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -0,0 +1,77 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: clickhouse_path/format_schemas/rabbitmq.proto +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='clickhouse_path/format_schemas/rabbitmq.proto', + package='', + syntax='proto3', + serialized_options=None, + create_key=_descriptor._internal_create_key, + serialized_pb=b'\n-clickhouse_path/format_schemas/rabbitmq.proto\"*\n\x0cKeyValuePair\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' +) + + + + +_KEYVALUEPAIR = _descriptor.Descriptor( + name='KeyValuePair', + full_name='KeyValuePair', + filename=None, + file=DESCRIPTOR, + containing_type=None, + create_key=_descriptor._internal_create_key, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='KeyValuePair.key', index=0, + number=1, type=4, cpp_type=4, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key), + _descriptor.FieldDescriptor( + name='value', full_name='KeyValuePair.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=b"".decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=49, + serialized_end=91, +) + +DESCRIPTOR.message_types_by_name['KeyValuePair'] = _KEYVALUEPAIR +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +KeyValuePair = _reflection.GeneratedProtocolMessageType('KeyValuePair', (_message.Message,), { + 'DESCRIPTOR' : _KEYVALUEPAIR, + '__module__' : 'clickhouse_path.format_schemas.rabbitmq_pb2' + # @@protoc_insertion_point(class_scope:KeyValuePair) + }) +_sym_db.RegisterMessage(KeyValuePair) + + +# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index bb65319a3be..b8ccbf9ce56 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -16,13 +16,19 @@ from helpers.network import PartitionManager import json import subprocess +import avro.schema +from confluent.schemaregistry.client import CachedSchemaRegistryClient +from confluent.schemaregistry.serializers.MessageSerializer import MessageSerializer from google.protobuf.internal.encoder import _VarintBytes +import rabbitmq_pb2 + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', config_dir='configs', main_configs=['configs/rabbitmq.xml','configs/log_conf.xml'], - with_rabbitmq=True) + with_rabbitmq=True, + clickhouse_path_dir='clickhouse_path') rabbitmq_id = '' @@ -316,6 +322,57 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): rabbitmq_check_result(result, True) +@pytest.mark.timeout(180) +def test_rabbitmq_protobuf(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq (key UInt64, value String) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'pb', + rabbitmq_format = 'Protobuf', + rabbitmq_schema = 'rabbitmq.proto:KeyValuePair'; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + data = '' + for i in range(0, 20): + msg = rabbitmq_pb2.KeyValuePair() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + channel.basic_publish(exchange='pb', routing_key='', body=data) + data = '' + for i in range(20, 21): + msg = rabbitmq_pb2.KeyValuePair() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + channel.basic_publish(exchange='pb', routing_key='', body=data) + data = '' + for i in range(21, 50): + msg = rabbitmq_pb2.KeyValuePair() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + channel.basic_publish(exchange='pb', routing_key='', body=data) + + result = '' + while True: + result += instance.query('SELECT * FROM test.rabbitmq') + if rabbitmq_check_result(result): + break + + connection.close() + rabbitmq_check_result(result, True) + + @pytest.mark.timeout(180) def test_rabbitmq_materialized_view(rabbitmq_cluster): instance.query(''' @@ -451,6 +508,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): DROP TABLE test.view2; ''') + connection.close() rabbitmq_check_result(result1, True) rabbitmq_check_result(result2, True) @@ -1440,7 +1498,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): connection.close() result = instance.query(''' - SELECT key, value, _exchange_name, SUBSTRING(_channel_id, 34, 3), _delivery_tag, _redelivered + SELECT key, value, _exchange_name, SUBSTRING(_channel_id, 1, 3), _delivery_tag, _redelivered FROM test.view ORDER BY key ''') @@ -1505,7 +1563,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): connection.close() - result = instance.query("SELECT key, value, exchange_name, SUBSTRING(channel_id, 34, 3), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(channel_id, 1, 3), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") expected = '''\ 0 0 virtuals_mv 1_0 1 0 1 1 virtuals_mv 1_0 2 0 @@ -1769,7 +1827,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_consumer_restore_failed_connection_without_losses(rabbitmq_cluster): +def test_rabbitmq_consumer_restore_failed_connection_without_losses_1(rabbitmq_cluster): instance.query(''' CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1901,71 +1959,72 @@ def test_rabbitmq_producer_restore_failed_connection_without_losses(rabbitmq_clu @pytest.mark.timeout(420) -def test_rabbitmq_virtual_columns_2(rabbitmq_cluster): +def test_rabbitmq_consumer_restore_failed_connection_without_losses_2(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() - ORDER BY key; + CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'consumer_reconnect', + rabbitmq_num_consumers = 10, + rabbitmq_num_queues = 2, + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; ''') - table_num = 3 - for table_id in range(table_num): - print("Setting up table {}".format(table_id)) - instance.query(''' - DROP TABLE IF EXISTS test.virtuals_{0}; - DROP TABLE IF EXISTS test.virtuals_{0}_mv; - CREATE TABLE test.virtuals_{0} (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'virtuals_2', - rabbitmq_num_queues = 2, - rabbitmq_num_consumers = 2, - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS - SELECT *, _exchange_name as exchange_name, _channel_id as channel_id, _delivery_tag as delivery_tag, _redelivered as redelivered - FROM test.virtuals_{0}; - '''.format(table_id)) + i = 0 + messages_num = 150000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) channel = connection.channel() - - message_num = 10 - i = 0 messages = [] - for _ in range(message_num): + for _ in range(messages_num): messages.append(json.dumps({'key': i, 'value': i})) i += 1 + for i in range(messages_num): + channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=messages[i], + properties=pika.BasicProperties(delivery_mode = 2, message_id=str(i))) + connection.close() - for i in range(message_num): - channel.basic_publish(exchange='virtuals_2', routing_key='', body=messages[i], - properties=pika.BasicProperties(delivery_mode=2, message_id=str(i))) + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.consumer_reconnect; + ''') - #kill_rabbitmq(); - #time.sleep(2); - #revive_rabbitmq(); + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(0.1) + + kill_rabbitmq(); + time.sleep(8); + revive_rabbitmq(); + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(0.1) + + kill_rabbitmq(); + time.sleep(2); + revive_rabbitmq(); while True: - result = instance.query('SELECT count(DISTINCT concat([channel_id], [toString(delivery_tag)])) FROM test.destination') - print instance.query(''' - SELECT DISTINCT concat([channel_id], [toString(delivery_tag)]) - FROM (SELECT channel_id AS id, delivery_tag AS tag FROM test.destination GROUP BY id ORDER BY tag)''') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - if int(result) == message_num * table_num: + if int(result) == messages_num: break - connection.close() - instance.query(''' DROP TABLE IF EXISTS test.consumer; DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + DROP TABLE IF EXISTS test.consumer_reconnect; ''') - assert int(result) == message_num * table_num + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) if __name__ == '__main__': From fb1417db7188a5b83c8a02344993597e054c7db1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Aug 2020 01:08:03 +0800 Subject: [PATCH 052/535] ISSUES-4006 try fix test failure --- src/DataTypes/registerDataTypeDateTime.cpp | 41 ++++++++++++------- src/Functions/FunctionsConversion.h | 25 +++++++---- .../0_stateless/00921_datetime64_basic.sql | 4 +- .../01442_date_time_with_params.reference | 6 +-- .../01442_date_time_with_params.sql | 10 ++--- 5 files changed, 53 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index c6a79e48335..0596b229494 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -22,6 +22,16 @@ enum class ArgumentKind Mandatory }; +String getExceptionMessage( + const String & message, size_t argument_index, const char * argument_name, + const std::string & context_data_type_name, Field::Types::Which field_type) +{ + return std::string("Parameter #") + std::to_string(argument_index) + " '" + + argument_name + "' for " + context_data_type_name + + message + + ", expected: " + Field::Types::toString(field_type) + " literal."; +} + template std::conditional_t, T> getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) @@ -30,14 +40,6 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume const auto field_type = Field::TypeToEnum::value; const ASTLiteral * argument = nullptr; - auto exception_message = [=](const String & message) - { - return std::string("Parameter #") + std::to_string(argument_index) + " '" - + argument_name + "' for " + context_data_type_name - + message - + ", expected: " + Field::Types::toString(field_type) + " literal."; - }; - if (!arguments || arguments->children.size() <= argument_index || !(argument = arguments->children[argument_index]->as()) || argument->value.getType() != field_type) @@ -45,8 +47,8 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume if constexpr (Kind == ArgumentKind::Optional) return {}; else - throw Exception(exception_message(" is missing"), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } return argument->value.get(); @@ -54,21 +56,26 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume static DataTypePtr create(const ASTPtr & arguments) { - if (!arguments || arguments->size() == 0) + if (!arguments || arguments->children.size() == 0) return std::make_shared(); const auto scale = getArgument(arguments, 0, "scale", "DateTime"); const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime"); - if (scale) - return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); + if (!scale && !timezone) + throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + /// If scale is defined, the data type is DateTime when scale = 0 otherwise the data type is DateTime64 + if (scale && scale.value() != 0) + return std::make_shared(scale.value(), timezone.value_or(String{})); return std::make_shared(timezone.value_or(String{})); } static DataTypePtr create32(const ASTPtr & arguments) { - if (!arguments || arguments->size() == 0) + if (!arguments || arguments->children.size() == 0) return std::make_shared(); if (arguments->children.size() != 1) @@ -81,7 +88,7 @@ static DataTypePtr create32(const ASTPtr & arguments) static DataTypePtr create64(const ASTPtr & arguments) { - if (!arguments || arguments->size() == 0) + if (!arguments || arguments->children.size() == 0) return std::make_shared(DataTypeDateTime64::default_scale); if (arguments->children.size() > 2) @@ -90,6 +97,10 @@ static DataTypePtr create64(const ASTPtr & arguments) const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); + if (!scale && !timezone) + throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index a8e8ad81ff8..9e5a781240d 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1029,7 +1029,7 @@ public: mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v) { /// toDateTime(value, scale:Integer) if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) @@ -1085,14 +1085,16 @@ public: scale = static_cast(arguments[1].column->get64(0)); } - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) { timezone_arg_position += 1; scale = static_cast(arguments[1].column->get64(0)); - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + if (scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime + return std::make_shared( + scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); } } @@ -1199,15 +1201,22 @@ private: return true; }; - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 if ((arguments.size() == 2 && isUnsignedInteger(block.getByPosition(arguments[1]).type)) || arguments.size() == 3) { - if (!callOnIndexAndDataType(from_type->getTypeId(), call)) - throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return; + const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); + UInt32 scale = extractToDecimalScale(scale_column); + + if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 + { + if (!callOnIndexAndDataType(from_type->getTypeId(), call)) + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return; + } } } diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index 2d7cb975cfc..bc881e3175d 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS A; -SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type +SELECT CAST(1 as DateTime64('abc')); -- { serverError 1000 } # invalid timezone SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone -SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid scale +SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid timezone SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index a6cb7f7b948..03b591a34bb 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -1,4 +1,4 @@ -2020-01-01 00:00:00 DateTime 2020-01-01 00:01:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:06:00 DateTime(\'Europe/Moscow\') -2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') -2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') +2020-01-01 00:00:00 DateTime 2020-01-01 00:01:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:06:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:06:00 DateTime +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:00:00 DateTime diff --git a/tests/queries/0_stateless/01442_date_time_with_params.sql b/tests/queries/0_stateless/01442_date_time_with_params.sql index 1e75089bc05..d2664a4e316 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.sql +++ b/tests/queries/0_stateless/01442_date_time_with_params.sql @@ -1,14 +1,14 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (a DateTime, b DateTime(), c DateTime(2), d DateTime('Europe/Moscow'), e DateTime(3, 'Europe/Moscow'), f DateTime32, g DateTime32('Europe/Moscow')) ENGINE = MergeTree ORDER BY a; +CREATE TABLE test (a DateTime, b DateTime(), c DateTime(2), d DateTime('Europe/Moscow'), e DateTime(3, 'Europe/Moscow'), f DateTime32, g DateTime32('Europe/Moscow'), h DateTime(0)) ENGINE = MergeTree ORDER BY a; -INSERT INTO test VALUES('2020-01-01 00:00:00', '2020-01-01 00:01:00', '2020-01-01 00:02:00.11', '2020-01-01 00:03:00', '2020-01-01 00:04:00.22', '2020-01-01 00:05:00', '2020-01-01 00:06:00') +INSERT INTO test VALUES('2020-01-01 00:00:00', '2020-01-01 00:01:00', '2020-01-01 00:02:00.11', '2020-01-01 00:03:00', '2020-01-01 00:04:00.22', '2020-01-01 00:05:00', '2020-01-01 00:06:00', '2020-01-01 00:06:00'); -SELECT a, toTypeName(a), b, toTypeName(b), c, toTypeName(c), d, toTypeName(d), e, toTypeName(e), f, toTypeName(f), g, toTypeName(g) FROM test; +SELECT a, toTypeName(a), b, toTypeName(b), c, toTypeName(c), d, toTypeName(d), e, toTypeName(e), f, toTypeName(f), g, toTypeName(g), h, toTypeName(h) FROM test; -SELECT toDateTime('2020-01-01 00:00:00') AS a, toTypeName(a), toDateTime('2020-01-01 00:02:00.11', 2) AS b, toTypeName(b), toDateTime('2020-01-01 00:03:00', 'Europe/Moscow') AS c, toTypeName(c), toDateTime('2020-01-01 00:04:00.22', 3, 'Europe/Moscow') AS d, toTypeName(d); +SELECT toDateTime('2020-01-01 00:00:00') AS a, toTypeName(a), toDateTime('2020-01-01 00:02:00.11', 2) AS b, toTypeName(b), toDateTime('2020-01-01 00:03:00', 'Europe/Moscow') AS c, toTypeName(c), toDateTime('2020-01-01 00:04:00.22', 3, 'Europe/Moscow') AS d, toTypeName(d), toDateTime('2020-01-01 00:05:00', 0) AS e, toTypeName(e); -SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-01-01 00:02:00.11', 'DateTime(2)') AS b, toTypeName(b), CAST('2020-01-01 00:03:00', 'DateTime(\'Europe/Moscow\')') AS c, toTypeName(c), CAST('2020-01-01 00:04:00.22', 'DateTime(3, \'Europe/Moscow\')') AS d, toTypeName(d); +SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-01-01 00:02:00.11', 'DateTime(2)') AS b, toTypeName(b), CAST('2020-01-01 00:03:00', 'DateTime(\'Europe/Moscow\')') AS c, toTypeName(c), CAST('2020-01-01 00:04:00.22', 'DateTime(3, \'Europe/Moscow\')') AS d, toTypeName(d), CAST('2020-01-01 00:05:00', 'DateTime(0)') AS e, toTypeName(e); SELECT toDateTime32('2020-01-01 00:00:00') AS a, toTypeName(a); From ade8c19b571f1f0ab1eb47727bd48341c1219f6d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Aug 2020 13:21:38 +0800 Subject: [PATCH 053/535] ISSUES-4006 try fix build & test failure --- src/DataTypes/registerDataTypeDateTime.cpp | 16 ++++++---------- .../0_stateless/00921_datetime64_basic.sql | 6 +++--- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 0596b229494..9b6af5f6e0b 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -34,7 +34,7 @@ String getExceptionMessage( template std::conditional_t, T> -getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) +getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name [[maybe_unused]], const std::string context_data_type_name) { using NearestResultType = NearestFieldType; const auto field_type = Field::TypeToEnum::value; @@ -56,7 +56,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume static DataTypePtr create(const ASTPtr & arguments) { - if (!arguments || arguments->children.size() == 0) + if (!arguments || arguments->children.empty()) return std::make_shared(); const auto scale = getArgument(arguments, 0, "scale", "DateTime"); @@ -75,7 +75,7 @@ static DataTypePtr create(const ASTPtr & arguments) static DataTypePtr create32(const ASTPtr & arguments) { - if (!arguments || arguments->children.size() == 0) + if (!arguments || arguments->children.empty()) return std::make_shared(); if (arguments->children.size() != 1) @@ -88,20 +88,16 @@ static DataTypePtr create32(const ASTPtr & arguments) static DataTypePtr create64(const ASTPtr & arguments) { - if (!arguments || arguments->children.size() == 0) + if (!arguments || arguments->children.empty()) return std::make_shared(DataTypeDateTime64::default_scale); if (arguments->children.size() > 2) throw Exception("DateTime64 data type can optionally have two argument - scale and time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); + const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); - if (!scale && !timezone) - throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); + return std::make_shared(scale, timezone.value_or(String{})); } void registerDataTypeDateTime(DataTypeFactory & factory) diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index bc881e3175d..1fc534d8afd 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS A; -SELECT CAST(1 as DateTime64('abc')); -- { serverError 1000 } # invalid timezone +SELECT CAST(1 as DateTime64('abc')); -- { serverError 42 } # Miss scale parameter type SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale -SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type +SELECT CAST(1 as DateTime64(-1)); -- { serverError 42 } # Miss scale parameter type SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone -SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid timezone +SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # Miss scale parameter type SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone From 405a6fb08fa22a9e063dd5e48e7ee6060f718749 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Mon, 17 Aug 2020 18:20:23 +0800 Subject: [PATCH 054/535] New feature: LineAsString format. #13630 --- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatFactory.h | 1 + .../Impl/LineAsStringRowInputFormat.cpp | 101 ++++++++++++++++++ .../Formats/Impl/LineAsStringRowInputFormat.h | 31 ++++++ src/Processors/ya.make | 1 + 5 files changed, 135 insertions(+) create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.h diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 5256ab2b321..f996e3d8cf2 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -365,6 +365,7 @@ FormatFactory::FormatFactory() registerInputFormatProcessorMsgPack(*this); registerOutputFormatProcessorMsgPack(*this); registerInputFormatProcessorJSONAsString(*this); + registerInputFormatProcessorLineAsString(*this); registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineCSV(*this); diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index ea4004c191f..610cf8105b8 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -210,5 +210,6 @@ void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); void registerInputFormatProcessorCapnProto(FormatFactory & factory); void registerInputFormatProcessorRegexp(FormatFactory & factory); void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorLineAsString(FormatFactory & factory); } diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp new file mode 100644 index 00000000000..a28b3903724 --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -0,0 +1,101 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; +} + +LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : + IRowInputFormat(header_, in_, std::move(params_)), buf(in) +{ + if (header_.columns() > 1 || header_.getDataTypes()[0]->getTypeId() != TypeIndex::String) + { + throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::LOGICAL_ERROR); + } +} + +void LineAsStringRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + buf.reset(); +} + +void LineAsStringRowInputFormat::readLineObject(IColumn & column) +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + size_t balance = 0; + + if (*buf.position() != '"') + throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); + + ++buf.position(); + ++balance; + + char * pos; + + while (balance) + { + if (buf.eof()) + throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); + + pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + buf.position() = pos; + if (buf.position() == buf.buffer().end()) + continue; + else if (*buf.position() == '"') + { + --balance; + ++buf.position(); + } + else if (*buf.position() == '\\') + { + ++buf.position(); + if (!buf.eof()) + { + ++buf.position(); + } + } + + } + buf.makeContinuousMemoryFromCheckpointToPos(); + char * end = buf.position(); + buf.rollbackToCheckpoint(); + column.insertData(buf.position(), end - buf.position()); + buf.position() = end; +} + +bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + skipWhitespaceIfAny(buf); + + if (!buf.eof()) + readLineObject(*columns[0]); + + skipWhitespaceIfAny(buf); + if (!buf.eof() && *buf.position() == ',') + ++buf.position(); + skipWhitespaceIfAny(buf); + + return !buf.eof(); +} + +void registerInputFormatProcessorLineAsString(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("LineAsString", []( + ReadBuffer & buf, + const Block & sample, + const RowInputFormatParams & params, + const FormatSettings &) + { + return std::make_shared(sample, buf, params); + }); +} + +} diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h new file mode 100644 index 00000000000..a31dce1cc4a --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/// This format parses a sequence of Line objects separated by newlines, spaces and/or comma. +/// Each Line object is parsed as a whole to string. +/// This format can only parse a table with single field of type String. + +class LineAsStringRowInputFormat : public IRowInputFormat +{ +public: + LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + String getName() const override { return "LineAsStringRowInputFormat"; } + void resetParser() override; + +private: + void readLineObject(IColumn & column); + + PeekableReadBuffer buf; +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4c25ad5bf3f..081b1d5ba1f 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -23,6 +23,7 @@ SRCS( Formats/Impl/ConstantExpressionTemplate.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/LineAsStringRowInputFormat.cpp Formats/Impl/JSONAsStringRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp From 0451d5729323b7f46d79336fea4f0982bb1662ae Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Tue, 18 Aug 2020 10:35:08 +0800 Subject: [PATCH 055/535] Add new feature: LineAsString Format --- src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp | 6 ++++++ .../0_stateless/01460_line_as_string_format.reference | 1 + tests/queries/0_stateless/01460_line_as_string_format.sql | 5 +++++ 3 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.reference create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index a28b3903724..36844fa700b 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -32,6 +32,12 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) PeekableReadBufferCheckpoint checkpoint{buf}; size_t balance = 0; + if (*buf.position() == ';') { + ++buf.position(); + if(buf.eof()) + return; + } + if (*buf.position() != '"') throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference new file mode 100644 index 00000000000..989f8ac0292 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -0,0 +1 @@ +"I love apple","I love banana","I love pear" diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql new file mode 100644 index 00000000000..e5518a828d0 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS line_as_string; +CREATE TABLE line_as_string (field String) ENGINE = Memory; +INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; +SELECT * FROM line_as_string; +DROP TABLE line_as_string; From adc2c117c8e6b4384fa134988ba2aff19043dec3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 Aug 2020 17:34:04 +0300 Subject: [PATCH 056/535] fixes --- tests/integration/test_alter_codec/test.py | 2 -- tests/integration/test_storage_kafka/test.py | 1 - 2 files changed, 3 deletions(-) diff --git a/tests/integration/test_alter_codec/test.py b/tests/integration/test_alter_codec/test.py index 7e038081110..4d251f60b16 100644 --- a/tests/integration/test_alter_codec/test.py +++ b/tests/integration/test_alter_codec/test.py @@ -6,11 +6,9 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', main_configs=['configs/logs_config.xml']) node2 = cluster.add_instance('node2', - config_dir='configs', main_configs=['configs/logs_config.xml']) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8961fce173f..6d38a7368ea 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -40,7 +40,6 @@ import kafka_pb2 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', - config_dir='configs', main_configs=['configs/kafka.xml', 'configs/log_conf.xml', 'configs/kafka_macros.xml' ], with_kafka=True, with_zookeeper=True, From 26020cdf6840961e99ee4784307afef68ecee3e3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 Aug 2020 17:38:16 +0300 Subject: [PATCH 057/535] typo --- tests/integration/test_distributed_ddl/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index b3a0513b799..b8c9527441e 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,9 +17,9 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: - main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml"), + main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml", "query_log.xml","ddl.xml"] - main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files)] + main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files] user_configs = [os.path.join(self.test_config_dir, "users.d", f) for f in ["restricted_user.xml", "query_log.xml"]] if self.test_config_dir == "configs_secure": main_configs += [os.path.join(self.test_config_dir, f) for i in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] From e9be2f14ea8ac45f11c7c65b6c36646b64a5b390 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Wed, 19 Aug 2020 11:50:43 +0800 Subject: [PATCH 058/535] fix implementation for \n separated lines --- .../Impl/LineAsStringRowInputFormat.cpp | 45 ++++++------------- .../01460_line_as_string_format.reference | 7 ++- .../01460_line_as_string_format.sh | 19 ++++++++ .../01460_line_as_string_format.sql | 5 --- 4 files changed, 38 insertions(+), 38 deletions(-) create mode 100755 tests/queries/0_stateless/01460_line_as_string_format.sh delete mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 36844fa700b..27bc71d764d 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -30,35 +30,22 @@ void LineAsStringRowInputFormat::resetParser() void LineAsStringRowInputFormat::readLineObject(IColumn & column) { PeekableReadBufferCheckpoint checkpoint{buf}; - size_t balance = 0; - - if (*buf.position() == ';') { - ++buf.position(); - if(buf.eof()) - return; - } - - if (*buf.position() != '"') - throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); - - ++buf.position(); - ++balance; + bool newline = true; + bool over = false; char * pos; - while (balance) + while (newline) { - if (buf.eof()) - throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); - - pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end()); buf.position() = pos; - if (buf.position() == buf.buffer().end()) - continue; - else if (*buf.position() == '"') + if (buf.position() == buf.buffer().end()) { + over = true; + break; + } + else if (*buf.position() == '\n') { - --balance; - ++buf.position(); + newline = false; } else if (*buf.position() == '\\') { @@ -70,25 +57,19 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) } } + buf.makeContinuousMemoryFromCheckpointToPos(); - char * end = buf.position(); + char * end = over ? buf.position(): ++buf.position(); buf.rollbackToCheckpoint(); - column.insertData(buf.position(), end - buf.position()); + column.insertData(buf.position(), end - (over ? 0 : 1) - buf.position()); buf.position() = end; } bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { - skipWhitespaceIfAny(buf); - if (!buf.eof()) readLineObject(*columns[0]); - skipWhitespaceIfAny(buf); - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); - skipWhitespaceIfAny(buf); - return !buf.eof(); } diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference index 989f8ac0292..dec67eb2e0a 100644 --- a/tests/queries/0_stateless/01460_line_as_string_format.reference +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -1 +1,6 @@ -"I love apple","I love banana","I love pear" +"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\\"\\\\", +"array" : [1, 2, 3], + +Finally implement this new feature. diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sh b/tests/queries/0_stateless/01460_line_as_string_format.sh new file mode 100755 index 00000000000..a985bc207a8 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string(field String) ENGINE = Memory"; + +echo '"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\"\\", +"array" : [1, 2, 3], + +Finally implement this new feature.' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string FORMAT LineAsString"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string" + diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql deleted file mode 100644 index e5518a828d0..00000000000 --- a/tests/queries/0_stateless/01460_line_as_string_format.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS line_as_string; -CREATE TABLE line_as_string (field String) ENGINE = Memory; -INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; -SELECT * FROM line_as_string; -DROP TABLE line_as_string; From bdb20738e57f24c84384f78336772cb9efe69ad9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 19 Aug 2020 13:19:36 +0800 Subject: [PATCH 059/535] ISSUES-4006 compatible DateTime64 --- src/DataTypes/registerDataTypeDateTime.cpp | 10 ++++++++-- tests/queries/0_stateless/00921_datetime64_basic.sql | 6 +++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 9b6af5f6e0b..eceb531b892 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -47,8 +47,14 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume if constexpr (Kind == ArgumentKind::Optional) return {}; else - throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (argument->value.getType() != field_type) + throw Exception(getExceptionMessage(String(" has wrong type: ") + argument->value.getTypeName(), + argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + else + throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } } return argument->value.get(); diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index 1fc534d8afd..2d7cb975cfc 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS A; -SELECT CAST(1 as DateTime64('abc')); -- { serverError 42 } # Miss scale parameter type +SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale -SELECT CAST(1 as DateTime64(-1)); -- { serverError 42 } # Miss scale parameter type +SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone -SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # Miss scale parameter type +SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid scale SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone From e77ab608c8e579caca7131cc2036dbac3d32e582 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 Aug 2020 10:45:16 +0300 Subject: [PATCH 060/535] fix typo --- tests/integration/test_distributed_ddl/cluster.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index b8c9527441e..258478de990 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,12 +17,12 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: - main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml", - "query_log.xml","ddl.xml"] + main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml", "query_log.xml","ddl.xml"] main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files] user_configs = [os.path.join(self.test_config_dir, "users.d", f) for f in ["restricted_user.xml", "query_log.xml"]] if self.test_config_dir == "configs_secure": - main_configs += [os.path.join(self.test_config_dir, f) for i in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] + main_configs += [os.path.join(self.test_config_dir, f) for f in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] + for i in xrange(4): self.add_instance( 'ch{}'.format(i+1), From e44975df3b44b5dbaac36256ff5d34225a7aa682 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 19 Aug 2020 23:18:25 +0800 Subject: [PATCH 061/535] ISSUES-4006 try fix test failure --- src/DataTypes/registerDataTypeDateTime.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index eceb531b892..70b89bf7545 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -101,7 +101,7 @@ static DataTypePtr create64(const ASTPtr & arguments) throw Exception("DateTime64 data type can optionally have two argument - scale and time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); - const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); + const auto timezone = getArgument(arguments, 1, "timezone", "DateTime64"); return std::make_shared(scale, timezone.value_or(String{})); } From edeb983eb0d93ec66351238f349ef09a472ae083 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 20 Aug 2020 19:18:29 +0800 Subject: [PATCH 062/535] ISSUES-4006 some refactor --- src/Functions/FunctionsConversion.h | 64 ++++++++++++++--------------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9e5a781240d..5fbcce4bc59 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -994,6 +994,18 @@ DEFINE_NAME_TO_INTERVAL(Year) #undef DEFINE_NAME_TO_INTERVAL +template +static inline bool isDateTime64(const ColumnsWithTypeAndName &arguments) +{ + if constexpr (std::is_same_v) + return true; + else if constexpr (std::is_same_v) + { + return (arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3; + } + + return false; +} template class FunctionConvert : public IFunction @@ -1024,16 +1036,14 @@ public: FunctionArgumentDescriptors mandatory_args = {{"Value", nullptr, nullptr, nullptr}}; FunctionArgumentDescriptors optional_args; - if constexpr (to_decimal || to_datetime64) + if constexpr (to_decimal) { mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } - if constexpr (std::is_same_v) + if (!to_decimal && isDateTime64(arguments)) { - /// toDateTime(value, scale:Integer) - if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) - mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); + mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } // toString(DateTime or DateTime64, [timezone: String]) @@ -1079,29 +1089,22 @@ public: UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale; // DateTime64 requires more arguments: scale and timezone. Since timezone is optional, scale should be first. - if constexpr (to_datetime64) + if (isDateTime64(arguments)) { timezone_arg_position += 1; scale = static_cast(arguments[1].column->get64(0)); - } - if constexpr (std::is_same_v) - { - /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 - if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) - { - timezone_arg_position += 1; - scale = static_cast(arguments[1].column->get64(0)); - if (scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime - return std::make_shared( - scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); - } + if (to_datetime64 || scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime + return std::make_shared(scale, + extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); } if constexpr (std::is_same_v) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); - else if constexpr (to_datetime64) - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + else if constexpr (std::is_same_v) + throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); else return std::make_shared(); } @@ -1201,22 +1204,19 @@ private: return true; }; - if constexpr (std::is_same_v) + if (isDateTime64(block.getColumnsWithTypeAndName())) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 - if ((arguments.size() == 2 && isUnsignedInteger(block.getByPosition(arguments[1]).type)) || arguments.size() == 3) + const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); + UInt32 scale = extractToDecimalScale(scale_column); + + if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 { - const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); - UInt32 scale = extractToDecimalScale(scale_column); + if (!callOnIndexAndDataType(from_type->getTypeId(), call)) + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 - { - if (!callOnIndexAndDataType(from_type->getTypeId(), call)) - throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return; - } + return; } } From 45cc0778a0a65204e3c49653c7db067fa9fc1744 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 20 Aug 2020 22:41:03 +0800 Subject: [PATCH 063/535] ISSUES-4006 support scale with parserDateTime --- src/Functions/FunctionsConversion.h | 92 ++++++++++++++++++----------- 1 file changed, 59 insertions(+), 33 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5fbcce4bc59..e4b990b53f4 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -994,14 +994,22 @@ DEFINE_NAME_TO_INTERVAL(Year) #undef DEFINE_NAME_TO_INTERVAL +struct NameParseDateTimeBestEffort; +struct NameParseDateTimeBestEffortOrZero; +struct NameParseDateTimeBestEffortOrNull; + template -static inline bool isDateTime64(const ColumnsWithTypeAndName &arguments) +static inline bool isDateTime64(const ColumnsWithTypeAndName & arguments, const ColumnNumbers & arguments_index = {}) { if constexpr (std::is_same_v) return true; - else if constexpr (std::is_same_v) + else if constexpr (std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v) { - return (arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3; + if (arguments_index.empty()) + return (arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3; + else + return (arguments_index.size() == 2 && isUnsignedInteger(arguments[arguments_index[1]].type)) || arguments_index.size() == 3; } return false; @@ -1204,7 +1212,7 @@ private: return true; }; - if (isDateTime64(block.getColumnsWithTypeAndName())) + if (isDateTime64(block.getColumnsWithTypeAndName(), arguments)) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); @@ -1273,7 +1281,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { DataTypePtr res; - if constexpr (to_datetime64) + + if (isDateTime64(arguments)) { validateFunctionArgumentTypes(*this, arguments, FunctionArgumentDescriptors{{"string", isStringOrFixedString, nullptr, "String or FixedString"}}, @@ -1283,11 +1292,12 @@ public: {"timezone", isStringOrFixedString, isColumnConst, "const String or FixedString"}, }); - UInt64 scale = DataTypeDateTime64::default_scale; + UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; if (arguments.size() > 1) scale = extractToDecimalScale(arguments[1]); const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0); - res = std::make_shared(scale, timezone); + + res = scale == 0 ? res = std::make_shared(timezone) : std::make_shared(scale, timezone); } else { @@ -1334,6 +1344,8 @@ public: if constexpr (std::is_same_v) res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); + else if constexpr (std::is_same_v) + throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); else if constexpr (to_decimal) { UInt64 scale = extractToDecimalScale(arguments[1]); @@ -1358,42 +1370,53 @@ public: return res; } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override + template + bool executeInternal(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, UInt32 scale = 0) const { const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); - bool ok = true; - if constexpr (to_decimal || to_datetime64) + if (checkAndGetDataType(from_type)) { - const UInt32 scale = assert_cast(*removeNullable(block.getByPosition(result).type)).getScale(); - - if (checkAndGetDataType(from_type)) - { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count, scale); - } - else if (checkAndGetDataType(from_type)) - { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count, scale); - } - else - ok = false; + ConvertThroughParsing::execute( + block, arguments, result, input_rows_count, scale); + return true; } + else if (checkAndGetDataType(from_type)) + { + ConvertThroughParsing::execute( + block, arguments, result, input_rows_count, scale); + return true; + } + + return false; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override + { + bool ok = true; + + if constexpr (to_decimal) + ok = executeInternal(block, arguments, result, input_rows_count, + assert_cast(*removeNullable(block.getByPosition(result).type)).getScale()); else { - if (checkAndGetDataType(from_type)) + if (isDateTime64(block.getColumnsWithTypeAndName(), arguments)) { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count); - } - else if (checkAndGetDataType(from_type)) - { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count); + UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; + if (arguments.size() > 1) + scale = extractToDecimalScale(block.getColumnsWithTypeAndName()[arguments[1]]); + + if (scale == 0) + ok = executeInternal(block, arguments, result, input_rows_count); + else + { + ok = executeInternal(block, arguments, result, input_rows_count, static_cast(scale)); + } } else - ok = false; + { + ok = executeInternal(block, arguments, result, input_rows_count); + } } if (!ok) @@ -1757,6 +1780,9 @@ struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTime struct NameParseDateTimeBestEffortUS { static constexpr auto name = "parseDateTimeBestEffortUS"; }; struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; }; struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; }; +struct NameParseDateTime32BestEffort { static constexpr auto name = "parseDateTime32BestEffort"; }; +struct NameParseDateTime32BestEffortOrZero { static constexpr auto name = "parseDateTime32BestEffortOrZero"; }; +struct NameParseDateTime32BestEffortOrNull { static constexpr auto name = "parseDateTime32BestEffortOrNull"; }; struct NameParseDateTime64BestEffort { static constexpr auto name = "parseDateTime64BestEffort"; }; struct NameParseDateTime64BestEffortOrZero { static constexpr auto name = "parseDateTime64BestEffortOrZero"; }; struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = "parseDateTime64BestEffortOrNull"; }; From ec1572d7be7edc35d044dac603af2544b381b17e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 21 Aug 2020 13:06:06 +0800 Subject: [PATCH 064/535] ISSUES-4006 support parserDateTime32 functions --- src/Functions/FunctionsConversion.cpp | 3 ++ src/Functions/FunctionsConversion.h | 7 +++ .../01442_date_time_with_params.reference | 40 +++++++++++++++ .../01442_date_time_with_params.sql | 50 +++++++++++++++++++ 4 files changed, 100 insertions(+) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 804c16d946d..428c6ba8138 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -82,6 +82,9 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e4b990b53f4..bcafcc3b59f 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1797,6 +1797,13 @@ using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString< using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString< DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffort = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffortOrZero = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffortOrNull = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + using FunctionParseDateTime64BestEffort = FunctionConvertFromString< DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; using FunctionParseDateTime64BestEffortOrZero = FunctionConvertFromString< diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index 03b591a34bb..f38732b3f2f 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -2,3 +2,43 @@ 2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:00:00 DateTime +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 06:37:03.253 DateTime64(3, \'Europe/Minsk\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +\N Nullable(DateTime64(3)) +2020-05-14 03:37:03.000 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 03:37:03.000 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 06:37:03.253 Nullable(DateTime64(3, \'Europe/Minsk\')) +2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) +1970-01-01 08:00:00.000 DateTime64(3) +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 06:37:03.253 DateTime64(3, \'Europe/Minsk\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 06:37:03 DateTime(\'Europe/Minsk\') +2020-05-14 03:37:03 DateTime(\'UTC\') +\N Nullable(DateTime) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 06:37:03 Nullable(DateTime(\'Europe/Minsk\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +1970-01-01 08:00:00 DateTime +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 06:37:03 DateTime(\'Europe/Minsk\') +2020-05-14 03:37:03 DateTime(\'UTC\') diff --git a/tests/queries/0_stateless/01442_date_time_with_params.sql b/tests/queries/0_stateless/01442_date_time_with_params.sql index d2664a4e316..5ae7fe22699 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.sql +++ b/tests/queries/0_stateless/01442_date_time_with_params.sql @@ -12,4 +12,54 @@ SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-0 SELECT toDateTime32('2020-01-01 00:00:00') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('', 3) AS a, toTypeName(a); -- {serverError 6} +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14 03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); + +SELECT parseDateTimeBestEffortOrNull('', 3) AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14 03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); + +SELECT parseDateTimeBestEffortOrZero('', 3) AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14 03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); + + +SELECT parseDateTime32BestEffort('') AS a, toTypeName(a); -- {serverError 6} +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14 03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); + +SELECT parseDateTime32BestEffortOrNull('') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14 03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); + +SELECT parseDateTime32BestEffortOrZero('') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14 03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); + + DROP TABLE IF EXISTS test; From 3318b6ea00c478a0986d1fe526c172860dac1997 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 21 Aug 2020 13:08:45 +0800 Subject: [PATCH 065/535] ISSUES-4006 try fix build failure --- src/DataTypes/registerDataTypeDateTime.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 70b89bf7545..815948c6531 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -48,7 +48,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume return {}; else { - if (argument->value.getType() != field_type) + if (argument && argument->value.getType() != field_type) throw Exception(getExceptionMessage(String(" has wrong type: ") + argument->value.getTypeName(), argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); else From b679b2e30cdf01170352de3007880a01834341b7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 21 Aug 2020 13:16:50 +0800 Subject: [PATCH 066/535] ISSUES-4006 fix toDateTime64 with scale 0 --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index bcafcc3b59f..5539d73d2eb 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1218,7 +1218,7 @@ private: const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); UInt32 scale = extractToDecimalScale(scale_column); - if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 + if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 { if (!callOnIndexAndDataType(from_type->getTypeId(), call)) throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), From ff84040cd5394516a64688fc2701472325c00be6 Mon Sep 17 00:00:00 2001 From: Winter Zhang Date: Fri, 21 Aug 2020 14:42:31 +0800 Subject: [PATCH 067/535] ISSUES-4006 try fix test failure --- .../queries/0_stateless/01442_date_time_with_params.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index f38732b3f2f..f55d095d164 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -15,7 +15,7 @@ 2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) 2020-05-14 06:37:03.253 Nullable(DateTime64(3, \'Europe/Minsk\')) 2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) -1970-01-01 08:00:00.000 DateTime64(3) +1970-01-01 03:00:00.000 DateTime64(3) 2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') 2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') 2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') @@ -35,7 +35,7 @@ 2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) 2020-05-14 06:37:03 Nullable(DateTime(\'Europe/Minsk\')) 2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) -1970-01-01 08:00:00 DateTime +1970-01-01 03:00:00 DateTime 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 03:37:03 DateTime(\'UTC\') From 2a96151516008a7b338346d87a6c88151cc95dae Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 22 Aug 2020 01:14:34 +0300 Subject: [PATCH 068/535] Fix GRANT ALL statement when executed on a non-global level. --- src/Access/AccessFlags.h | 29 +++++++++ src/Access/AccessRights.cpp | 75 ++++++---------------- src/Access/AccessRightsElement.h | 46 +++++++++++++ src/Interpreters/InterpreterGrantQuery.cpp | 2 +- src/Parsers/ASTGrantQuery.cpp | 8 +++ src/Parsers/ASTGrantQuery.h | 1 + src/Parsers/ParserGrantQuery.cpp | 27 ++++++++ 7 files changed, 131 insertions(+), 57 deletions(-) diff --git a/src/Access/AccessFlags.h b/src/Access/AccessFlags.h index 9b801fd88a3..11d39585238 100644 --- a/src/Access/AccessFlags.h +++ b/src/Access/AccessFlags.h @@ -96,6 +96,22 @@ public: /// Returns all the flags related to a dictionary. static AccessFlags allDictionaryFlags(); + /// Returns all the flags which could be granted on the global level. + /// The same as allFlags(). + static AccessFlags allFlagsGrantableOnGlobalLevel(); + + /// Returns all the flags which could be granted on the global level. + /// Returns allDatabaseFlags() | allTableFlags() | allDictionaryFlags() | allColumnFlags(). + static AccessFlags allFlagsGrantableOnDatabaseLevel(); + + /// Returns all the flags which could be granted on the table level. + /// Returns allTableFlags() | allDictionaryFlags() | allColumnFlags(). + static AccessFlags allFlagsGrantableOnTableLevel(); + + /// Returns all the flags which could be granted on the global level. + /// The same as allColumnFlags(). + static AccessFlags allFlagsGrantableOnColumnLevel(); + private: static constexpr size_t NUM_FLAGS = 128; using Flags = std::bitset; @@ -193,6 +209,10 @@ public: const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } + const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); } + const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; } + const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; } + const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); } private: enum NodeType @@ -381,6 +401,9 @@ private: } for (const auto & child : start_node->children) collectAllFlags(child.get()); + + all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; + all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level; } Impl() @@ -431,6 +454,8 @@ private: std::vector access_type_to_flags_mapping; Flags all_flags; Flags all_flags_for_target[static_cast(DICTIONARY) + 1]; + Flags all_flags_grantable_on_database_level; + Flags all_flags_grantable_on_table_level; }; @@ -447,6 +472,10 @@ inline AccessFlags AccessFlags::allDatabaseFlags() { return Impl<>::instance().g inline AccessFlags AccessFlags::allTableFlags() { return Impl<>::instance().getTableFlags(); } inline AccessFlags AccessFlags::allColumnFlags() { return Impl<>::instance().getColumnFlags(); } inline AccessFlags AccessFlags::allDictionaryFlags() { return Impl<>::instance().getDictionaryFlags(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Impl<>::instance().getAllFlagsGrantableOnGlobalLevel(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Impl<>::instance().getAllFlagsGrantableOnDatabaseLevel(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Impl<>::instance().getAllFlagsGrantableOnTableLevel(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Impl<>::instance().getAllFlagsGrantableOnColumnLevel(); } inline AccessFlags operator |(AccessType left, AccessType right) { return AccessFlags(left) | right; } inline AccessFlags operator &(AccessType left, AccessType right) { return AccessFlags(left) & right; } diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 65c78f39e86..8ce71dd8da8 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -8,12 +7,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int INVALID_GRANT; -} - - namespace { using Kind = AccessRightsElementWithOptions::Kind; @@ -214,30 +207,14 @@ namespace COLUMN_LEVEL, }; - AccessFlags getAcceptableFlags(Level level) + AccessFlags getAllGrantableFlags(Level level) { switch (level) { - case GLOBAL_LEVEL: - { - static const AccessFlags res = AccessFlags::allFlags(); - return res; - } - case DATABASE_LEVEL: - { - static const AccessFlags res = AccessFlags::allDatabaseFlags() | AccessFlags::allTableFlags() | AccessFlags::allDictionaryFlags() | AccessFlags::allColumnFlags(); - return res; - } - case TABLE_LEVEL: - { - static const AccessFlags res = AccessFlags::allTableFlags() | AccessFlags::allDictionaryFlags() | AccessFlags::allColumnFlags(); - return res; - } - case COLUMN_LEVEL: - { - static const AccessFlags res = AccessFlags::allColumnFlags(); - return res; - } + case GLOBAL_LEVEL: return AccessFlags::allFlagsGrantableOnGlobalLevel(); + case DATABASE_LEVEL: return AccessFlags::allFlagsGrantableOnDatabaseLevel(); + case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel(); + case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel(); } __builtin_unreachable(); } @@ -276,21 +253,7 @@ public: void grant(const AccessFlags & flags_) { - if (!flags_) - return; - - AccessFlags flags_to_add = flags_ & getAcceptableFlags(); - - if (!flags_to_add) - { - if (level == DATABASE_LEVEL) - throw Exception(flags_.toString() + " cannot be granted on the database level", ErrorCodes::INVALID_GRANT); - else if (level == TABLE_LEVEL) - throw Exception(flags_.toString() + " cannot be granted on the table level", ErrorCodes::INVALID_GRANT); - else if (level == COLUMN_LEVEL) - throw Exception(flags_.toString() + " cannot be granted on the column level", ErrorCodes::INVALID_GRANT); - } - + AccessFlags flags_to_add = flags_ & getAllGrantableFlags(); addGrantsRec(flags_to_add); optimizeTree(); } @@ -456,8 +419,8 @@ public: } private: - AccessFlags getAcceptableFlags() const { return ::DB::getAcceptableFlags(level); } - AccessFlags getChildAcceptableFlags() const { return ::DB::getAcceptableFlags(static_cast(level + 1)); } + AccessFlags getAllGrantableFlags() const { return ::DB::getAllGrantableFlags(level); } + AccessFlags getChildAllGrantableFlags() const { return ::DB::getAllGrantableFlags(static_cast(level + 1)); } Node * tryGetChild(const std::string_view & name) const { @@ -480,7 +443,7 @@ private: Node & new_child = (*children)[*new_child_name]; new_child.node_name = std::move(new_child_name); new_child.level = static_cast(level + 1); - new_child.flags = flags & new_child.getAcceptableFlags(); + new_child.flags = flags & new_child.getAllGrantableFlags(); return new_child; } @@ -496,12 +459,12 @@ private: bool canEraseChild(const Node & child) const { - return ((flags & child.getAcceptableFlags()) == child.flags) && !child.children; + return ((flags & child.getAllGrantableFlags()) == child.flags) && !child.children; } void addGrantsRec(const AccessFlags & flags_) { - if (auto flags_to_add = flags_ & getAcceptableFlags()) + if (auto flags_to_add = flags_ & getAllGrantableFlags()) { flags |= flags_to_add; if (children) @@ -547,7 +510,7 @@ private: const AccessFlags & parent_flags) { auto flags = node.flags; - auto parent_fl = parent_flags & node.getAcceptableFlags(); + auto parent_fl = parent_flags & node.getAllGrantableFlags(); auto revokes = parent_fl - flags; auto grants = flags - parent_fl; @@ -576,9 +539,9 @@ private: const Node * node_go, const AccessFlags & parent_flags_go) { - auto acceptable_flags = ::DB::getAcceptableFlags(static_cast(full_name.size())); - auto parent_fl = parent_flags & acceptable_flags; - auto parent_fl_go = parent_flags_go & acceptable_flags; + auto grantable_flags = ::DB::getAllGrantableFlags(static_cast(full_name.size())); + auto parent_fl = parent_flags & grantable_flags; + auto parent_fl_go = parent_flags_go & grantable_flags; auto flags = node ? node->flags : parent_fl; auto flags_go = node_go ? node_go->flags : parent_fl_go; auto revokes = parent_fl - flags; @@ -672,8 +635,8 @@ private: } max_flags_with_children |= max_among_children; - AccessFlags add_acceptable_flags = getAcceptableFlags() - getChildAcceptableFlags(); - min_flags_with_children &= min_among_children | add_acceptable_flags; + AccessFlags add_flags = getAllGrantableFlags() - getChildAllGrantableFlags(); + min_flags_with_children &= min_among_children | add_flags; } void makeUnionRec(const Node & rhs) @@ -689,7 +652,7 @@ private: for (auto & [lhs_childname, lhs_child] : *children) { if (!rhs.tryGetChild(lhs_childname)) - lhs_child.flags |= rhs.flags & lhs_child.getAcceptableFlags(); + lhs_child.flags |= rhs.flags & lhs_child.getAllGrantableFlags(); } } } @@ -738,7 +701,7 @@ private: if (new_flags != flags) { - new_flags &= getAcceptableFlags(); + new_flags &= getAllGrantableFlags(); flags_added |= static_cast(new_flags - flags); flags_removed |= static_cast(flags - new_flags); flags = new_flags; diff --git a/src/Access/AccessRightsElement.h b/src/Access/AccessRightsElement.h index f9f7c433308..36cb64e6eba 100644 --- a/src/Access/AccessRightsElement.h +++ b/src/Access/AccessRightsElement.h @@ -71,6 +71,8 @@ struct AccessRightsElement { } + bool empty() const { return !access_flags || (!any_column && columns.empty()); } + auto toTuple() const { return std::tie(access_flags, any_database, database, any_table, table, any_column, columns); } friend bool operator==(const AccessRightsElement & left, const AccessRightsElement & right) { return left.toTuple() == right.toTuple(); } friend bool operator!=(const AccessRightsElement & left, const AccessRightsElement & right) { return !(left == right); } @@ -86,6 +88,9 @@ struct AccessRightsElement /// If the database is empty, replaces it with `new_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & new_database); + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -111,6 +116,9 @@ struct AccessRightsElementWithOptions : public AccessRightsElement friend bool operator==(const AccessRightsElementWithOptions & left, const AccessRightsElementWithOptions & right) { return left.toTuple() == right.toTuple(); } friend bool operator!=(const AccessRightsElementWithOptions & left, const AccessRightsElementWithOptions & right) { return !(left == right); } + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -120,9 +128,14 @@ struct AccessRightsElementWithOptions : public AccessRightsElement class AccessRightsElements : public std::vector { public: + bool empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } + /// Replaces the empty database with `new_database`. void replaceEmptyDatabase(const String & new_database); + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -134,6 +147,9 @@ public: /// Replaces the empty database with `new_database`. void replaceEmptyDatabase(const String & new_database); + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -157,4 +173,34 @@ inline void AccessRightsElementsWithOptions::replaceEmptyDatabase(const String & element.replaceEmptyDatabase(new_database); } +inline void AccessRightsElement::removeNonGrantableFlags() +{ + if (!any_column) + access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel(); + else if (!any_table) + access_flags &= AccessFlags::allFlagsGrantableOnTableLevel(); + else if (!any_database) + access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel(); + else + access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel(); +} + +inline void AccessRightsElementWithOptions::removeNonGrantableFlags() +{ + if (kind == Kind::GRANT) + AccessRightsElement::removeNonGrantableFlags(); +} + +inline void AccessRightsElements::removeNonGrantableFlags() +{ + for (auto & element : *this) + element.removeNonGrantableFlags(); +} + +inline void AccessRightsElementsWithOptions::removeNonGrantableFlags() +{ + for (auto & element : *this) + element.removeNonGrantableFlags(); +} + } diff --git a/src/Interpreters/InterpreterGrantQuery.cpp b/src/Interpreters/InterpreterGrantQuery.cpp index 2f468507eb6..57cb701036e 100644 --- a/src/Interpreters/InterpreterGrantQuery.cpp +++ b/src/Interpreters/InterpreterGrantQuery.cpp @@ -29,7 +29,6 @@ namespace current_access.grant(access_to_grant); } - AccessRightsElements getFilteredAccessRightsElementsToRevoke( const AccessRights & current_access, const AccessRightsElements & access_to_revoke, bool grant_option) { @@ -214,6 +213,7 @@ BlockIO InterpreterGrantQuery::execute() auto access = context.getAccess(); auto & access_control = context.getAccessControlManager(); query.replaceEmptyDatabaseWithCurrent(context.getCurrentDatabase()); + query.removeNonGrantableFlags(); RolesOrUsersSet roles_from_query; if (query.roles) diff --git a/src/Parsers/ASTGrantQuery.cpp b/src/Parsers/ASTGrantQuery.cpp index ae9649cdddc..63489e0417f 100644 --- a/src/Parsers/ASTGrantQuery.cpp +++ b/src/Parsers/ASTGrantQuery.cpp @@ -144,4 +144,12 @@ void ASTGrantQuery::replaceCurrentUserTagWithName(const String & current_user_na if (to_roles) to_roles->replaceCurrentUserTagWithName(current_user_name); } + + +void ASTGrantQuery::removeNonGrantableFlags() +{ + if (kind == Kind::GRANT) + access_rights_elements.removeNonGrantableFlags(); +} + } diff --git a/src/Parsers/ASTGrantQuery.h b/src/Parsers/ASTGrantQuery.h index c36e42689a5..5f172fe3298 100644 --- a/src/Parsers/ASTGrantQuery.h +++ b/src/Parsers/ASTGrantQuery.h @@ -33,6 +33,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void replaceEmptyDatabaseWithCurrent(const String & current_database); void replaceCurrentUserTagWithName(const String & current_user_name) const; + void removeNonGrantableFlags(); ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster(clone()); } }; } diff --git a/src/Parsers/ParserGrantQuery.cpp b/src/Parsers/ParserGrantQuery.cpp index 6e42b165b21..7dd721c9af2 100644 --- a/src/Parsers/ParserGrantQuery.cpp +++ b/src/Parsers/ParserGrantQuery.cpp @@ -14,6 +14,7 @@ namespace DB { namespace ErrorCodes { + extern const int INVALID_GRANT; extern const int SYNTAX_ERROR; } @@ -156,6 +157,29 @@ namespace } + void removeNonGrantableFlags(AccessRightsElements & elements) + { + for (auto & element : elements) + { + if (element.empty()) + continue; + auto old_flags = element.access_flags; + element.removeNonGrantableFlags(); + if (!element.empty()) + continue; + + if (!element.any_column) + throw Exception(old_flags.toString() + " cannot be granted on the column level", ErrorCodes::INVALID_GRANT); + else if (!element.any_table) + throw Exception(old_flags.toString() + " cannot be granted on the table level", ErrorCodes::INVALID_GRANT); + else if (!element.any_database) + throw Exception(old_flags.toString() + " cannot be granted on the database level", ErrorCodes::INVALID_GRANT); + else + throw Exception(old_flags.toString() + " cannot be granted", ErrorCodes::INVALID_GRANT); + } + } + + bool parseRoles(IParser::Pos & pos, Expected & expected, Kind kind, bool id_mode, std::shared_ptr & roles) { return IParserBase::wrapParseImpl(pos, [&] @@ -274,6 +298,9 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (admin_option && !elements.empty()) throw Exception("ADMIN OPTION should be specified for roles", ErrorCodes::SYNTAX_ERROR); + if (kind == Kind::GRANT) + removeNonGrantableFlags(elements); + auto query = std::make_shared(); node = query; From 4c8a8d5e67ec613f9d164366279e9e7b81577111 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 22 Aug 2020 01:37:01 +0300 Subject: [PATCH 069/535] Add test. --- tests/integration/test_grant_and_revoke/test.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 92ffb78a1cb..1557e81bce8 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -107,6 +107,15 @@ def test_revoke_requires_grant_option(): assert instance.query("SHOW GRANTS FOR B") == "" +def test_grant_all_on_table(): + instance.query("CREATE USER A, B") + instance.query("GRANT ALL ON test.table TO A WITH GRANT OPTION") + instance.query("GRANT ALL ON test.table TO B", user='A') + assert instance.query("SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" + instance.query("REVOKE ALL ON test.table FROM B", user='A') + assert instance.query("SHOW GRANTS FOR B") == "" + + def test_implicit_show_grants(): instance.query("CREATE USER A") assert instance.query("select count() FROM system.databases WHERE name='test'", user="A") == "0\n" From 4331158d3051437f44c7fa1271e4673272cf8cac Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Aug 2020 16:09:23 +0300 Subject: [PATCH 070/535] merge with master --- src/Disks/DiskDecorator.cpp | 15 +++++++++++++++ src/Disks/DiskDecorator.h | 3 +++ .../MergeTree/MergeTreeDataPartWriterInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterInMemory.h | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 11 ++++++----- 5 files changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index e55534e347f..7f2ea58d7cf 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -165,4 +165,19 @@ void DiskDecorator::truncateFile(const String & path, size_t size) delegate->truncateFile(path, size); } +int DiskDecorator::open(const String & path, mode_t mode) const +{ + return delegate->open(path, mode); +} + +void DiskDecorator::close(int fd) const +{ + delegate->close(fd); +} + +void DiskDecorator::sync(int fd) const +{ + delegate->sync(fd); +} + } diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 71bb100c576..f1ddfff4952 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -42,6 +42,9 @@ public: void setReadOnly(const String & path) override; void createHardLink(const String & src_path, const String & dst_path) override; void truncateFile(const String & path, size_t size) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; const String getType() const override { return delegate->getType(); } protected: diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index a7486158737..f0738a1130a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -70,7 +70,7 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl } } -void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool) { /// If part is empty we still need to initialize block by empty columns. if (!part_in_memory->block) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h index 92e4228a90d..6e59cdd08a9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h @@ -18,7 +18,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; void calculateAndSerializePrimaryIndex(const Block & primary_index_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f3a72657be5..b05b970da3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -251,6 +251,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->minmax_idx = std::move(minmax_idx); new_data_part->is_temp = true; + std::optional sync_guard; if (new_data_part->isStoredOnDisk()) { /// The name could be non-unique in case of stale files from previous runs. @@ -262,12 +263,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - const auto disk = new_data_part->volume->getDisk(); - disk->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); - std::optional sync_guard; - if (data.getSettings()->fsync_part_directory) - sync_guard.emplace(disk, full_path); + if (data.getSettings()->fsync_part_directory) + sync_guard.emplace(disk, full_path); + } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) From 308e094d04401144603fb12a64b4604bb0bde02d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Aug 2020 21:06:21 +0300 Subject: [PATCH 071/535] Fix arrayJoin() capturing in lambda Fixes the following LOGICAL_ERROR: $ clickhouse-client -q 'select arrayFilter((a) -> ((a, arrayJoin([[]])) IN (Null, [Null])), [])' 2020.08.16 00:32:01.967102 [ 1744189 ] {b40a5ebd-d710-4f03-bb18-57db67de1181} : Logical error: 'Lambda captured argument arrayJoin(array(array())) not found in required columns.'. clickhouse-server: ../src/Common/Exception.cpp:45: DB::Exception::Exception(const string&, int): Assertion `false' failed. Since there are multiple input columns for arrayJoin(): (gdb) p captured_names_ $6 = std::vector of length 3, capacity 4 = {"arrayJoin(array(array()))", "arrayJoin(array(array()))", "__set"} While FunctionCaptureOverloadResolver cannot handle non-unique columns. --- src/Interpreters/ActionsVisitor.cpp | 15 ++++++++++++++- src/Interpreters/ActionsVisitor.h | 7 +++++++ .../0_stateless/01407_lambda_arrayJoin.reference | 1 + .../0_stateless/01407_lambda_arrayJoin.sql | 6 ++++++ 4 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01407_lambda_arrayJoin.reference create mode 100644 tests/queries/0_stateless/01407_lambda_arrayJoin.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index f2a1d570773..0df83f11c1f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -447,6 +447,19 @@ void ScopeStack::addAction(const ExpressionAction & action) } } +void ScopeStack::addActionNoInput(const ExpressionAction & action) +{ + size_t level = 0; + Names required = action.getNeededColumns(); + for (const auto & elem : required) + level = std::max(level, getColumnLevel(elem)); + + Names added; + stack[level].actions->add(action, added); + + stack[level].new_columns.insert(added.begin(), added.end()); +} + ExpressionActionsPtr ScopeStack::popLevel() { ExpressionActionsPtr res = stack.back().actions; @@ -549,7 +562,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// It could have been possible to implement arrayJoin which keeps source column, /// but in this case it will always be replicated (as many arrays), which is expensive. String tmp_name = data.getUniqueName("_array_join_" + arg->getColumnName()); - data.addAction(ExpressionAction::copyColumn(arg->getColumnName(), tmp_name)); + data.addActionNoInput(ExpressionAction::copyColumn(arg->getColumnName(), tmp_name)); data.addAction(ExpressionAction::arrayJoin(tmp_name, result_name)); } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index dbcc54c01d6..d8d85f1c0bf 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -12,6 +12,7 @@ namespace DB class Context; class ASTFunction; +struct ExpressionAction; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; @@ -49,6 +50,8 @@ struct ScopeStack size_t getColumnLevel(const std::string & name); void addAction(const ExpressionAction & action); + /// For arrayJoin() to avoid double columns in the input. + void addActionNoInput(const ExpressionAction & action); ExpressionActionsPtr popLevel(); @@ -115,6 +118,10 @@ public: { actions_stack.addAction(action); } + void addActionNoInput(const ExpressionAction & action) + { + actions_stack.addActionNoInput(action); + } const Block & getSampleBlock() const { diff --git a/tests/queries/0_stateless/01407_lambda_arrayJoin.reference b/tests/queries/0_stateless/01407_lambda_arrayJoin.reference new file mode 100644 index 00000000000..fe51488c706 --- /dev/null +++ b/tests/queries/0_stateless/01407_lambda_arrayJoin.reference @@ -0,0 +1 @@ +[] diff --git a/tests/queries/0_stateless/01407_lambda_arrayJoin.sql b/tests/queries/0_stateless/01407_lambda_arrayJoin.sql new file mode 100644 index 00000000000..4f34bb59527 --- /dev/null +++ b/tests/queries/0_stateless/01407_lambda_arrayJoin.sql @@ -0,0 +1,6 @@ +SELECT arrayFilter((a) -> ((a, arrayJoin([])) IN (Null, [Null])), []); +SELECT arrayFilter((a) -> ((a, arrayJoin([[]])) IN (Null, [Null])), []); + +-- simplified from the https://clickhouse-test-reports.s3.yandex.net/10373/6c4748a63e7acde2cc3283d96ffec590aae1e724/fuzzer/fuzzer.log#fail1 +SELECT * FROM system.one ARRAY JOIN arrayFilter((a) -> ((a, arrayJoin([])) IN (NULL)), []) AS arr_x; -- { serverError 43; } +SELECT * FROM numbers(1) LEFT ARRAY JOIN arrayFilter((x_0, x_1) -> (arrayJoin([]) IN (NULL)), [], []) AS arr_x; From c09891b4f8b6c78eebbd1ed9acd08e9a921b5197 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 26 Aug 2020 02:12:51 +0300 Subject: [PATCH 072/535] DOCSUP-203: Update by PR#11558. --- docs/ru/operations/utilities/clickhouse-copier.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/utilities/clickhouse-copier.md b/docs/ru/operations/utilities/clickhouse-copier.md index b05db93b28b..b43f5ccaf7a 100644 --- a/docs/ru/operations/utilities/clickhouse-copier.md +++ b/docs/ru/operations/utilities/clickhouse-copier.md @@ -24,7 +24,7 @@ Утилиту следует запускать вручную следующим образом: ``` bash -$ clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir +$ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir ``` Параметры запуска: From 3f53553522a34e1dd2312d8c7e85d9ae687f9df5 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 26 Aug 2020 02:37:32 +0300 Subject: [PATCH 073/535] DOCSUP-2031: Update by PR#11242. Added temporary_files_codec and join_on_disk_max_files_to_merge settings. --- docs/ru/operations/settings/settings.md | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e8d3f1057df..ab64fb757f1 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -401,12 +401,33 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Устанавливает тип поведения [JOIN](../../sql-reference/statements/select/join.md). При объединении таблиц могут появиться пустые ячейки. ClickHouse заполняет их по-разному в зависимости от настроек. -Возможные значения +Возможные значения: - 0 — пустые ячейки заполняются значением по умолчанию соответствующего типа поля. - 1 — `JOIN` ведёт себя как в стандартном SQL. Тип соответствующего поля преобразуется в [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable), а пустые ячейки заполняются значениями [NULL](../../sql-reference/syntax.md). -Значение по умолчанию: 0. +## join_on_disk_max_files_to_merge {#join_on_disk_max_files_to_merge} + +Устанавливет количество файлов, разрешенных для параллельной сортировки, при выполнении операций MergeJoin на диске. + +Чем больше значение параметра, тем больше оперативной памяти используется и тем меньше используется диск (I/O). + +Возможные значения: + +- Положительное целое число, больше 2. + +Значение по умолчанию: 64. + +## temporary_files_codec {#temporary_files_codec} + +Устанавливает метод сжатия для временных файлов на диске, используемых при сортировки и объединения. + +Возможные значения: + +- LZ4 — применять сжатие, используя алгоритм [LZ4](https://ru.wikipedia.org/wiki/LZ4) +- NONE — не применять сжатие. + +Значение по умолчанию: LZ4. ## max\_block\_size {#setting-max_block_size} From c48d3b9d63f38e6a9f281b39060ab5e7bfbd5dfb Mon Sep 17 00:00:00 2001 From: 243f6a88 85a308d3 <33170174+243f6a8885a308d313198a2e037@users.noreply.github.com> Date: Wed, 26 Aug 2020 10:28:03 +0900 Subject: [PATCH 074/535] fixed Japanese translation for data-types/date.md --- docs/ja/sql-reference/data-types/date.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ja/sql-reference/data-types/date.md b/docs/ja/sql-reference/data-types/date.md index ff6e028e885..528872d61c2 100644 --- a/docs/ja/sql-reference/data-types/date.md +++ b/docs/ja/sql-reference/data-types/date.md @@ -7,8 +7,7 @@ toc_title: "\u65E5\u4ED8" # 日付 {#date} -デートだ 1970-01-01(符号なし)以降の日数として二バイト単位で格納されます。 Unixエポックの開始直後から、コンパイル段階で定数によって定義される上限しきい値までの値を格納できます(現在は2106年までですが、完全にサポート -最小値は1970-01-01として出力されます。 +日付型です。 1970-01-01 からの日数が2バイトの符号なし整数として格納されます。 UNIX時間の開始直後から、変換段階で定数として定義される上限しきい値までの値を格納できます(現在は2106年までですが、一年分を完全にサポートしているのは2105年までです)。 日付値は、タイムゾーンなしで格納されます。 From cdcdb5a2c1f94cd629b2f1103340a6e08750c2fc Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 26 Aug 2020 10:47:00 +0300 Subject: [PATCH 075/535] Update date.md --- docs/ja/sql-reference/data-types/date.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ja/sql-reference/data-types/date.md b/docs/ja/sql-reference/data-types/date.md index 528872d61c2..bcdc8f7224d 100644 --- a/docs/ja/sql-reference/data-types/date.md +++ b/docs/ja/sql-reference/data-types/date.md @@ -1,6 +1,4 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_priority: 47 toc_title: "\u65E5\u4ED8" --- From 0f3351d983775eeee067d5d9d2e538238ed343bf Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 26 Aug 2020 13:22:08 +0300 Subject: [PATCH 076/535] Fix testflows checks. --- .../rbac/tests/syntax/grant_privilege.py | 47 ++++++++++--------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/tests/testflows/rbac/tests/syntax/grant_privilege.py b/tests/testflows/rbac/tests/syntax/grant_privilege.py index cabb3a3780b..82c459f546d 100755 --- a/tests/testflows/rbac/tests/syntax/grant_privilege.py +++ b/tests/testflows/rbac/tests/syntax/grant_privilege.py @@ -20,30 +20,30 @@ def setup(node): node.query("DROP ROLE IF EXISTS role1") @TestOutline(Scenario) -@Examples("privilege on allow_introspection", [ - ("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_DictGet("1.0"))), - ("INTROSPECTION", ("*.*",), True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Introspection("1.0"))), - ("SELECT", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Select("1.0"))), - ("INSERT",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Insert("1.0"))), - ("ALTER",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Alter("1.0"))), - ("CREATE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Create("1.0"))), - ("DROP",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Drop("1.0"))), - ("TRUNCATE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Truncate("1.0"))), - ("OPTIMIZE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Optimize("1.0"))), - ("SHOW",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Show("1.0"))), - ("KILL QUERY",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_KillQuery("1.0"))), - ("ACCESS MANAGEMENT",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_AccessManagement("1.0"))), - ("SYSTEM",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_System("1.0"))), - ("SOURCES",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Sources("1.0"))), - ("ALL",("*.*",), True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), - ("ALL PRIVILEGES",("*.*",), True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), #alias for all +@Examples("privilege on allow_column allow_introspection", [ + ("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_DictGet("1.0"))), + ("INTROSPECTION", ("*.*",), False, True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Introspection("1.0"))), + ("SELECT", ("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Select("1.0"))), + ("INSERT",("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Insert("1.0"))), + ("ALTER",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Alter("1.0"))), + ("CREATE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Create("1.0"))), + ("DROP",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Drop("1.0"))), + ("TRUNCATE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Truncate("1.0"))), + ("OPTIMIZE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Optimize("1.0"))), + ("SHOW",("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Show("1.0"))), + ("KILL QUERY",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_KillQuery("1.0"))), + ("ACCESS MANAGEMENT",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_AccessManagement("1.0"))), + ("SYSTEM",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_System("1.0"))), + ("SOURCES",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Sources("1.0"))), + ("ALL",("*.*",), True, True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), + ("ALL PRIVILEGES",("*.*",), True, True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), #alias for all ],) -def grant_privileges(self, privilege, on, allow_introspection, node="clickhouse1"): - grant_privilege(privilege=privilege, on=on, allow_introspection=allow_introspection, node=node) +def grant_privileges(self, privilege, on, allow_column, allow_introspection, node="clickhouse1"): + grant_privilege(privilege=privilege, on=on, allow_column=allow_column, allow_introspection=allow_introspection, node=node) @TestOutline(Scenario) @Requirements(RQ_SRS_006_RBAC_Grant_Privilege_GrantOption("1.0")) -def grant_privilege(self, privilege, on, allow_introspection, node="clickhouse1"): +def grant_privilege(self, privilege, on, allow_column, allow_introspection, node="clickhouse1"): node = self.context.cluster.node(node) for on_ in on: @@ -58,9 +58,10 @@ def grant_privilege(self, privilege, on, allow_introspection, node="clickhouse1" with When("I grant privilege with grant option"): node.query(f"GRANT {privilege} ON {on_} TO user1 WITH GRANT OPTION", settings=settings) - #grant column specific for some column 'x' - with When("I grant privilege with columns"): - node.query(f"GRANT {privilege}(x) ON {on_} TO user0", settings=settings) + if allow_column and ('*' not in on_): + #grant column specific for some column 'x' + with When("I grant privilege with columns"): + node.query(f"GRANT {privilege}(x) ON {on_} TO user0", settings=settings) @TestFeature @Name("grant privilege") From 7ac4bd7d1efe26a7693e72752696092704483e4a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 21 Aug 2020 18:47:37 +0300 Subject: [PATCH 077/535] Add storages from after ones from and . --- src/Access/AccessControlManager.cpp | 44 ++++++++----------- .../configs/local_directories.xml | 2 + .../test_user_directories/configs/memory.xml | 3 ++ .../configs/mixed_style.xml | 8 ++++ .../configs/old_style.xml | 1 + .../configs/relative_path.xml | 3 ++ .../integration/test_user_directories/test.py | 8 ++++ 7 files changed, 43 insertions(+), 26 deletions(-) create mode 100644 tests/integration/test_user_directories/configs/mixed_style.xml diff --git a/src/Access/AccessControlManager.cpp b/src/Access/AccessControlManager.cpp index 6158be1b603..1fa26c85354 100644 --- a/src/Access/AccessControlManager.cpp +++ b/src/Access/AccessControlManager.cpp @@ -281,41 +281,33 @@ void AccessControlManager::addStoragesFromMainConfig( String config_dir = std::filesystem::path{config_path}.remove_filename().string(); String dbms_dir = config.getString("path", DBMS_DEFAULT_PATH); String include_from_path = config.getString("include_from", "/etc/metrika.xml"); + bool has_user_directories = config.has("user_directories"); - if (config.has("user_directories")) + /// If path to users' config isn't absolute, try guess its root (current) dir. + /// At first, try to find it in dir of main config, after will use current dir. + String users_config_path = config.getString("users_config", ""); + if (users_config_path.empty()) { - if (config.has("users_config")) - LOG_WARNING(getLogger(), " is specified, the path from won't be used: " + config.getString("users_config")); - if (config.has("access_control_path")) - LOG_WARNING(getLogger(), " is specified, the path from won't be used: " + config.getString("access_control_path")); - - addStoragesFromUserDirectoriesConfig( - config, - "user_directories", - config_dir, - dbms_dir, - include_from_path, - get_zookeeper_function); - } - else - { - /// If path to users' config isn't absolute, try guess its root (current) dir. - /// At first, try to find it in dir of main config, after will use current dir. - String users_config_path = config.getString("users_config", ""); - if (users_config_path.empty()) + if (!has_user_directories) users_config_path = config_path; - else if (std::filesystem::path{users_config_path}.is_relative() && std::filesystem::exists(config_dir + users_config_path)) - users_config_path = config_dir + users_config_path; + } + else if (std::filesystem::path{users_config_path}.is_relative() && std::filesystem::exists(config_dir + users_config_path)) + users_config_path = config_dir + users_config_path; + if (!users_config_path.empty()) + { if (users_config_path != config_path) checkForUsersNotInMainConfig(config, config_path, users_config_path, getLogger()); addUsersConfigStorage(users_config_path, include_from_path, dbms_dir, get_zookeeper_function); - - String disk_storage_dir = config.getString("access_control_path", ""); - if (!disk_storage_dir.empty()) - addDiskStorage(disk_storage_dir); } + + String disk_storage_dir = config.getString("access_control_path", ""); + if (!disk_storage_dir.empty()) + addDiskStorage(disk_storage_dir); + + if (has_user_directories) + addStoragesFromUserDirectoriesConfig(config, "user_directories", config_dir, dbms_dir, include_from_path, get_zookeeper_function); } diff --git a/tests/integration/test_user_directories/configs/local_directories.xml b/tests/integration/test_user_directories/configs/local_directories.xml index e2cbcd135df..7b9601da982 100644 --- a/tests/integration/test_user_directories/configs/local_directories.xml +++ b/tests/integration/test_user_directories/configs/local_directories.xml @@ -12,4 +12,6 @@ /var/lib/clickhouse/access3-ro/ + +
diff --git a/tests/integration/test_user_directories/configs/memory.xml b/tests/integration/test_user_directories/configs/memory.xml index 6e906d2b1d6..78da38ed0bc 100644 --- a/tests/integration/test_user_directories/configs/memory.xml +++ b/tests/integration/test_user_directories/configs/memory.xml @@ -5,4 +5,7 @@ + + + diff --git a/tests/integration/test_user_directories/configs/mixed_style.xml b/tests/integration/test_user_directories/configs/mixed_style.xml new file mode 100644 index 00000000000..d6ddecf6f5d --- /dev/null +++ b/tests/integration/test_user_directories/configs/mixed_style.xml @@ -0,0 +1,8 @@ + + + + + + /etc/clickhouse-server/users6.xml + /var/lib/clickhouse/access6/ + diff --git a/tests/integration/test_user_directories/configs/old_style.xml b/tests/integration/test_user_directories/configs/old_style.xml index a0ff36edaba..cc753006b22 100644 --- a/tests/integration/test_user_directories/configs/old_style.xml +++ b/tests/integration/test_user_directories/configs/old_style.xml @@ -1,5 +1,6 @@ /etc/clickhouse-server/users2.xml /var/lib/clickhouse/access2/ + diff --git a/tests/integration/test_user_directories/configs/relative_path.xml b/tests/integration/test_user_directories/configs/relative_path.xml index 8906478959e..c4ef3c5fd79 100644 --- a/tests/integration/test_user_directories/configs/relative_path.xml +++ b/tests/integration/test_user_directories/configs/relative_path.xml @@ -4,4 +4,7 @@ users4.xml + + + diff --git a/tests/integration/test_user_directories/test.py b/tests/integration/test_user_directories/test.py index 8b7f34cf999..218330cb1a5 100644 --- a/tests/integration/test_user_directories/test.py +++ b/tests/integration/test_user_directories/test.py @@ -16,6 +16,7 @@ def started_cluster(): node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users3.xml") node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users4.xml") node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users5.xml") + node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users6.xml") yield cluster @@ -49,3 +50,10 @@ def test_memory(): node.restart_clickhouse() assert node.query("SELECT * FROM system.user_directories") == TSV([["users.xml", "users.xml", "/etc/clickhouse-server/users5.xml", 1, 1], ["memory", "memory", "", 0, 2]]) + +def test_mixed_style(): + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/mixed_style.xml"), '/etc/clickhouse-server/config.d/z.xml') + node.restart_clickhouse() + assert node.query("SELECT * FROM system.user_directories") == TSV([["users.xml", "users.xml", "/etc/clickhouse-server/users6.xml", 1, 1], + ["local directory", "local directory", "/var/lib/clickhouse/access6/", 0, 2], + ["memory", "memory", "", 0, 3]]) From 4fecfdbe2f85c6abe1f3f68843f415885618411c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 26 Aug 2020 08:54:29 +0000 Subject: [PATCH 078/535] Better & cleaner --- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 3 - src/Storages/RabbitMQ/RabbitMQHandler.cpp | 3 + .../ReadBufferFromRabbitMQConsumer.cpp | 4 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 83 +++----- src/Storages/RabbitMQ/StorageRabbitMQ.h | 5 +- .../WriteBufferToRabbitMQProducer.cpp | 135 +++---------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 6 +- .../integration/test_storage_rabbitmq/test.py | 190 ++++++------------ 8 files changed, 127 insertions(+), 302 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 517b6bfaf68..4fd5836b1a9 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -61,10 +61,7 @@ void RabbitMQBlockOutputStream::writeSuffix() child->writeSuffix(); if (buffer) - { buffer->updateMaxWait(); - buffer->commit(); - } } } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index 835ded1718c..d5b0a7d5c02 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -36,8 +36,11 @@ void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); + LOG_DEBUG(log, "Background loop started"); while (loop_state.load() == Loop::RUN) uv_run(loop, UV_RUN_NOWAIT); + + LOG_DEBUG(log, "Background loop ended"); } void RabbitMQHandler::iterateLoop() diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 197b9f7e057..86a39a95c1b 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -120,7 +120,7 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) /* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one * specific queue when its name is specified in queue_base setting. */ - const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(queue_id); + const String queue_name = !hash_exchange ? queue_base : std::to_string(channel_id_base) + "_" + std::to_string(queue_id) + "_" + queue_base; setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); while (!binding_created) @@ -173,7 +173,7 @@ void ReadBufferFromRabbitMQConsumer::ackMessages() AckTracker record = last_inserted_record; /// Do not send ack to server if message's channel is not the same as current running channel. - if (record.channel_id == channel_id && record.delivery_tag && record.delivery_tag > prev_tag) + if (record.channel_id == channel_id && record.delivery_tag && record.delivery_tag > prev_tag && event_handler->connectionRunning()) { consumer_channel->ack(record.delivery_tag, AMQP::multiple); /// Will ack all up to last tag starting from last acked. prev_tag = record.delivery_tag; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f82773ed367..4db2d75cd38 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -73,7 +73,6 @@ StorageRabbitMQ::StorageRabbitMQ( const String & exchange_type_, size_t num_consumers_, size_t num_queues_, - const bool use_transactional_channel_, const String & queue_base_, const String & deadletter_exchange_, const bool persistent_) @@ -87,7 +86,6 @@ StorageRabbitMQ::StorageRabbitMQ( , schema_name(global_context.getMacros()->expand(schema_name_)) , num_consumers(num_consumers_) , num_queues(num_queues_) - , use_transactional_channel(use_transactional_channel_) , queue_base(queue_base_) , deadletter_exchange(deadletter_exchange_) , persistent(persistent_) @@ -122,8 +120,14 @@ StorageRabbitMQ::StorageRabbitMQ( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + /// One looping task for all consumers as they share the same connection == the same handler == the same event loop + event_handler->updateLoopState(Loop::STOP); + looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); + looping_task->deactivate(); + streaming_task = global_context.getSchedulePool().createTask("RabbitMQStreamingTask", [this]{ threadFunc(); }); streaming_task->deactivate(); + heartbeat_task = global_context.getSchedulePool().createTask("RabbitMQHeartbeatTask", [this]{ heartbeatFunc(); }); heartbeat_task->deactivate(); @@ -156,15 +160,15 @@ StorageRabbitMQ::StorageRabbitMQ( /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base - * for the names of later declared queue (as everything is based on names). + * for the names of later declared queues. */ - queue_base = "queue_" + table_name; + queue_base = table_name; } else { /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and - * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need - * to share sharding exchange and bridge exchange. + * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share + * sharding exchange and bridge exchange. */ sharding_exchange = exchange_name + "_" + queue_base; } @@ -175,11 +179,6 @@ StorageRabbitMQ::StorageRabbitMQ( * (Cannot use table_name here because it must be a different string if table was restored) */ unique_strbase = getRandomName(); - - - /// One looping task for all consumers as they share the same connection == the same handler == the same event loop - looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); - looping_task->deactivate(); } @@ -216,7 +215,7 @@ void StorageRabbitMQ::initExchange() + std::string(message), ErrorCodes::LOGICAL_ERROR); }); - /// Bridge exchange is needed to easily disconnect consumer queues and also simplifies queue bindings a lot. + /// Bridge exchange is needed to easily disconnect consumer queues and also simplifies queue bindings. setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) .onError([&](const char * message) { @@ -230,7 +229,7 @@ void StorageRabbitMQ::initExchange() } /* Change hash property because by default it will be routing key, which has to be an integer, but with support for any exchange - * type - routing keys will not be such. + * type - routing keys might be of any type. */ AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; @@ -328,6 +327,10 @@ void StorageRabbitMQ::unbindExchange() */ std::call_once(flag, [&]() { + event_handler->updateLoopState(Loop::STOP); + looping_task->deactivate(); + heartbeat_task->deactivate(); + setup_channel->removeExchange(bridge_exchange) .onSuccess([&]() { @@ -342,10 +345,6 @@ void StorageRabbitMQ::unbindExchange() { event_handler->iterateLoop(); } - - event_handler->updateLoopState(Loop::STOP); - looping_task->deactivate(); - heartbeat_task->deactivate(); }); } @@ -442,11 +441,8 @@ Pipe StorageRabbitMQ::read( pipes.emplace_back(std::make_shared(converting_stream)); } - if (!loop_started) - { - loop_started = true; + if (!std::exchange(loop_started, true)) looping_task->activateAndSchedule(); - } LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); return Pipe::unitePipes(std::move(pipes)); @@ -479,6 +475,7 @@ void StorageRabbitMQ::startup() } } + event_handler->updateLoopState(Loop::RUN); streaming_task->activateAndSchedule(); heartbeat_task->activateAndSchedule(); } @@ -488,6 +485,7 @@ void StorageRabbitMQ::shutdown() { stream_cancelled = true; event_handler->updateLoopState(Loop::STOP); + wait_confirm.store(false); looping_task->deactivate(); streaming_task->deactivate(); @@ -560,7 +558,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - ++producer_id, unique_strbase, use_transactional_channel, persistent, log, + ++producer_id, unique_strbase, persistent, wait_confirm, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } @@ -663,12 +661,6 @@ bool StorageRabbitMQ::streamToViews() rabbit_stream->setLimits(limits); } - if (!loop_started) - { - loop_started = true; - looping_task->activateAndSchedule(); - } - // Join multiple streams if necessary BlockInputStreamPtr in; if (streams.size() > 1) @@ -676,6 +668,9 @@ bool StorageRabbitMQ::streamToViews() else in = streams[0]; + if (!std::exchange(loop_started, true)) + looping_task->activateAndSchedule(); + std::atomic stub = {false}; copyData(*in, *block_io.out, &stub); @@ -847,26 +842,12 @@ void registerStorageRabbitMQ(StorageFactory & factory) } } - bool use_transactional_channel = static_cast(rabbitmq_settings.rabbitmq_transactional_channel); + String queue_base = rabbitmq_settings.rabbitmq_queue_base.value; if (args_count >= 10) { + engine_args[9] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[9], args.local_context); + const auto * ast = engine_args[9]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - use_transactional_channel = static_cast(safeGet(ast->value)); - } - else - { - throw Exception("Transactional channel parameter is a bool", ErrorCodes::BAD_ARGUMENTS); - } - } - - String queue_base = rabbitmq_settings.rabbitmq_queue_base.value; - if (args_count >= 11) - { - engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); - - const auto * ast = engine_args[10]->as(); if (ast && ast->value.getType() == Field::Types::String) { queue_base = safeGet(ast->value); @@ -874,11 +855,11 @@ void registerStorageRabbitMQ(StorageFactory & factory) } String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; - if (args_count >= 12) + if (args_count >= 11) { - engine_args[11] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[11], args.local_context); + engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); - const auto * ast = engine_args[11]->as(); + const auto * ast = engine_args[10]->as(); if (ast && ast->value.getType() == Field::Types::String) { deadletter_exchange = safeGet(ast->value); @@ -886,9 +867,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) } bool persistent = static_cast(rabbitmq_settings.rabbitmq_persistent_mode); - if (args_count >= 13) + if (args_count >= 12) { - const auto * ast = engine_args[12]->as(); + const auto * ast = engine_args[11]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { persistent = static_cast(safeGet(ast->value)); @@ -902,7 +883,7 @@ void registerStorageRabbitMQ(StorageFactory & factory) return StorageRabbitMQ::create( args.table_id, args.context, args.columns, host_port, routing_keys, exchange, format, row_delimiter, schema, exchange_type, num_consumers, - num_queues, use_transactional_channel, queue_base, deadletter_exchange, persistent); + num_queues, queue_base, deadletter_exchange, persistent); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 60bc1aa7157..4c83257209c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -79,7 +79,6 @@ protected: const String & exchange_type_, size_t num_consumers_, size_t num_queues_, - const bool use_transactional_channel_, const String & queue_base_, const String & deadletter_exchange, const bool persistent_); @@ -99,7 +98,6 @@ private: size_t num_created_consumers = 0; bool hash_exchange; size_t num_queues; - const bool use_transactional_channel; String queue_base; const String deadletter_exchange; const bool persistent; @@ -120,7 +118,8 @@ private: String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; size_t producer_id = 0, consumer_id = 0; - std::atomic loop_started = false, exchange_removed = false; + bool loop_started = false; + std::atomic exchange_removed = false, wait_confirm = true; ChannelPtr setup_channel; std::mutex connection_mutex, restore_connection; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 8cd769e792f..6b8670fe9e7 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -15,7 +15,7 @@ namespace DB static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -static const auto BATCH = 10000; +static const auto BATCH = 1000; static const auto RETURNED_LIMIT = 50000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( @@ -27,8 +27,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const AMQP::ExchangeType exchange_type_, const size_t channel_id_base_, const String channel_base_, - const bool use_txn_, const bool persistent_, + std::atomic & wait_confirm_, Poco::Logger * log_, std::optional delimiter, size_t rows_per_message, @@ -41,8 +41,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , exchange_type(exchange_type_) , channel_id_base(std::to_string(channel_id_base_)) , channel_base(channel_base_) - , use_txn(use_txn_) , persistent(persistent_) + , wait_confirm(wait_confirm_) , payloads(BATCH) , returned(RETURNED_LIMIT) , log(log_) @@ -58,11 +58,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( if (setupConnection(false)) setupChannel(); - if (!use_txn) - { - writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); - writing_task->deactivate(); - } + writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); + writing_task->deactivate(); if (exchange_type == AMQP::ExchangeType::headers) { @@ -114,17 +111,8 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - if (!use_txn) - { - /// "publisher confirms" will be used, this is default. - ++payload_counter; - payloads.push(std::make_pair(payload_counter, payload)); - } - else - { - /// means channel->startTransaction() was called, not default, enabled only with table setting. - publish(payload); - } + ++payload_counter; + payloads.push(std::make_pair(payload_counter, payload)); } } @@ -189,28 +177,21 @@ void WriteBufferToRabbitMQProducer::setupChannel() channel_id = channel_id_base + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); - if (use_txn) + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, + * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times + * slower than the second, so default is second and the first is turned on in table setting. + * + * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html + */ + producer_channel->confirmSelect() + .onAck([&](uint64_t acked_delivery_tag, bool multiple) { - producer_channel->startTransaction(); - } - else + removeConfirmed(acked_delivery_tag, multiple, false); + }) + .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) { - /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, - * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times - * slower than the second, so default is second and the first is turned on in table setting. - * - * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html - */ - producer_channel->confirmSelect() - .onAck([&](uint64_t acked_delivery_tag, bool multiple) - { - removeConfirmed(acked_delivery_tag, multiple, false); - }) - .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) - { - removeConfirmed(nacked_delivery_tag, multiple, true); - }); - } + removeConfirmed(nacked_delivery_tag, multiple, true); + }); }); } @@ -272,7 +253,7 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable()) publish(returned, true); @@ -337,74 +318,6 @@ void WriteBufferToRabbitMQProducer::writingFunc() } -/* This publish is for the case when transaction is delcared on the channel with channel->startTransaction(). Here only publish - * once payload is available and then commitTransaction() is called, where a needed event loop will run. - */ -void WriteBufferToRabbitMQProducer::publish(const String & payload) -{ - AMQP::Envelope envelope(payload.data(), payload.size()); - - if (persistent) - envelope.setDeliveryMode(2); - - if (exchange_type == AMQP::ExchangeType::consistent_hash) - { - producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); - } - else if (exchange_type == AMQP::ExchangeType::headers) - { - producer_channel->publish(exchange_name, "", envelope); - } - else - { - producer_channel->publish(exchange_name, routing_keys[0], envelope); - } -} - - -void WriteBufferToRabbitMQProducer::commit() -{ - /* Actually have not yet found any information about how is it supposed work once any error occurs with a channel, because any channel - * error closes this channel and any operation on a closed channel will fail (but transaction is unique to channel). - * RabbitMQ transactions seem not trust-worthy at all - see https://www.rabbitmq.com/semantics.html. Seems like its best to always - * use "publisher confirms" rather than transactions (and by default it is so). Probably even need to delete this option. - */ - if (!use_txn || !producer_channel->usable()) - return; - - std::atomic answer_received = false, wait_rollback = false; - - producer_channel->commitTransaction() - .onSuccess([&]() - { - answer_received = true; - LOG_TRACE(log, "All messages were successfully published"); - }) - .onError([&](const char * message1) - { - answer_received = true; - wait_rollback = true; - LOG_TRACE(log, "Publishing not successful: {}", message1); - - producer_channel->rollbackTransaction() - .onSuccess([&]() - { - wait_rollback = false; - }) - .onError([&](const char * message2) - { - wait_rollback = false; - LOG_ERROR(log, "Failed to rollback transaction: {}", message2); - }); - }); - - while (!answer_received || wait_rollback) - { - iterateEventLoop(); - } -} - - void WriteBufferToRabbitMQProducer::nextImpl() { chunks.push_back(std::string()); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 85c90cd0ce9..1ab90cb0b1d 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -26,8 +26,8 @@ public: const AMQP::ExchangeType exchange_type_, const size_t channel_id_, const String channel_base_, - const bool use_txn_, const bool persistent_, + std::atomic & wait_confirm_, Poco::Logger * log_, std::optional delimiter, size_t rows_per_message, @@ -38,7 +38,6 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } - void commit(); void updateMaxWait() { wait_num.store(payload_counter); } private: @@ -49,7 +48,6 @@ private: void setupChannel(); void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); void publish(ConcurrentBoundedQueue> & message, bool republishing); - void publish(const String & payload); std::pair parsed_address; const std::pair login_password; @@ -58,8 +56,8 @@ private: AMQP::ExchangeType exchange_type; const String channel_id_base; const String channel_base; - const bool use_txn; const bool persistent; + std::atomic & wait_confirm; AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index b8ccbf9ce56..503396188b5 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -16,8 +16,6 @@ from helpers.network import PartitionManager import json import subprocess -import avro.schema -from confluent.schemaregistry.client import CachedSchemaRegistryClient from confluent.schemaregistry.serializers.MessageSerializer import MessageSerializer from google.protobuf.internal.encoder import _VarintBytes @@ -645,18 +643,15 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): ''') for mv_id in range(NUM_MV): - table_name = 'view{}'.format(mv_id) - print("Setting up {}".format(table_name)) - instance.query(''' - DROP TABLE IF EXISTS test.{0}; - DROP TABLE IF EXISTS test.{0}_mv; - CREATE TABLE test.{0} (key UInt64, value UInt64) + DROP TABLE IF EXISTS test.combo_{0}; + DROP TABLE IF EXISTS test.combo_{0}_mv; + CREATE TABLE test.combo_{0} (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; - CREATE MATERIALIZED VIEW test.{0}_mv TO test.{0} AS + CREATE MATERIALIZED VIEW test.combo_{0}_mv TO test.combo_{0} AS SELECT * FROM test.rabbitmq; - '''.format(table_name)) + '''.format(mv_id)) time.sleep(2) @@ -692,8 +687,8 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): while True: result = 0 - for view in range(NUM_MV): - result += int(instance.query('SELECT count() FROM test.view{0}'.format(view))) + for mv_id in range(NUM_MV): + result += int(instance.query('SELECT count() FROM test.combo_{0}'.format(mv_id))) if int(result) == messages_num * threads_num * NUM_MV: break time.sleep(1) @@ -702,10 +697,10 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): thread.join() for mv_id in range(NUM_MV): - table_name = 'view{}'.format(mv_id) instance.query(''' - DROP TABLE IF EXISTS test.{0}; - '''.format(table_name)) + DROP TABLE test.combo_{0}; + DROP TABLE test.combo_{0}_mv; + '''.format(mv_id)) assert int(result) == messages_num * threads_num * NUM_MV, 'ClickHouse lost some messages: {}'.format(result) @@ -879,10 +874,10 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): break instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_consume; - DROP TABLE IF EXISTS test.rabbitmq_many; - DROP TABLE IF EXISTS test.consumer_many; - DROP TABLE IF EXISTS test.view_many; + DROP TABLE test.rabbitmq_consume; + DROP TABLE test.rabbitmq_many; + DROP TABLE test.consumer_many; + DROP TABLE test.view_many; ''') for thread in threads: @@ -953,10 +948,10 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): break instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_overload; - DROP TABLE IF EXISTS test.consumer_overload; - DROP TABLE IF EXISTS test.view_overload; - DROP TABLE IF EXISTS test.view_consume; + DROP TABLE test.consumer_overload; + DROP TABLE test.view_overload; + DROP TABLE test.rabbitmq_consume; + DROP TABLE test.rabbitmq_overload; ''') for thread in threads: @@ -1028,8 +1023,8 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): for consumer_id in range(num_tables): instance.query(''' - DROP TABLE IF EXISTS test.direct_exchange_{0}; - DROP TABLE IF EXISTS test.direct_exchange_{0}_mv; + DROP TABLE test.direct_exchange_{0}; + DROP TABLE test.direct_exchange_{0}_mv; '''.format(consumer_id)) instance.query(''' @@ -1098,12 +1093,12 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): for consumer_id in range(num_tables): instance.query(''' - DROP TABLE IF EXISTS test.fanout_exchange_{0}; - DROP TABLE IF EXISTS test.fanout_exchange_{0}_mv; + DROP TABLE test.fanout_exchange_{0}; + DROP TABLE test.fanout_exchange_{0}_mv; '''.format(consumer_id)) instance.query(''' - DROP TABLE IF EXISTS test.destination; + DROP TABLE test.destination; ''') assert int(result) == messages_num * num_tables, 'ClickHouse lost some messages: {}'.format(result) @@ -1195,12 +1190,12 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): for consumer_id in range(num_tables * 2): instance.query(''' - DROP TABLE IF EXISTS test.topic_exchange_{0}; - DROP TABLE IF EXISTS test.topic_exchange_{0}_mv; + DROP TABLE test.topic_exchange_{0}; + DROP TABLE test.topic_exchange_{0}_mv; '''.format(consumer_id)) instance.query(''' - DROP TABLE IF EXISTS test.destination; + DROP TABLE test.destination; ''') assert int(result) == messages_num * num_tables + messages_num * num_tables, 'ClickHouse lost some messages: {}'.format(result) @@ -1278,12 +1273,12 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): for consumer_id in range(num_tables): table_name = 'rabbitmq_consumer{}'.format(consumer_id) instance.query(''' - DROP TABLE IF EXISTS test.{0}; - DROP TABLE IF EXISTS test.{0}_mv; + DROP TABLE test.{0}; + DROP TABLE test.{0}_mv; '''.format(table_name)) instance.query(''' - DROP TABLE IF EXISTS test.destination; + DROP TABLE test.destination; ''') for thread in threads: @@ -1361,9 +1356,9 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): thread.join() instance.query(''' - DROP TABLE IF EXISTS test.bindings; - DROP TABLE IF EXISTS test.bindings_mv; - DROP TABLE IF EXISTS test.destination; + DROP TABLE test.bindings; + DROP TABLE test.bindings_mv; + DROP TABLE test.destination; ''') assert int(result) == messages_num * threads_num * 5, 'ClickHouse lost some messages: {}'.format(result) @@ -1450,12 +1445,12 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): for consumer_id in range(num_tables_to_receive + num_tables_to_ignore): instance.query(''' - DROP TABLE IF EXISTS test.headers_exchange_{0}_mv; - DROP TABLE IF EXISTS test.headers_exchange_{0}; + DROP TABLE test.headers_exchange_{0}; + DROP TABLE test.headers_exchange_{0}_mv; '''.format(consumer_id)) instance.query(''' - DROP TABLE IF EXISTS test.destination; + DROP TABLE test.destination; ''') assert int(result) == messages_num * num_tables_to_receive, 'ClickHouse lost some messages: {}'.format(result) @@ -1516,7 +1511,8 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): ''' instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + DROP TABLE test.rabbitmq_virtuals; + DROP TABLE test.view; ''') assert TSV(result) == TSV(expected) @@ -1578,9 +1574,9 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ''' instance.query(''' - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.rabbitmq_virtuals_mv ''') assert TSV(result) == TSV(expected) @@ -1663,9 +1659,9 @@ def test_rabbitmq_queue_resume(rabbitmq_cluster): break instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_queue_resume; - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; + DROP TABLE test.rabbitmq_queue_resume; + DROP TABLE test.consumer; + DROP TABLE test.view; ''') assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) @@ -1733,9 +1729,9 @@ def test_rabbitmq_no_loss_on_table_drop(rabbitmq_cluster): break instance.query(''' - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.rabbitmq_consumer_acks; ''') assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) @@ -1813,12 +1809,12 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): for consumer_id in range(num_tables): instance.query(''' - DROP TABLE IF EXISTS test.many_consumers_{0}; - DROP TABLE IF EXISTS test.many_consumers_{0}_mv; + DROP TABLE test.many_consumers_{0}; + DROP TABLE test.many_consumers_{0}_mv; '''.format(consumer_id)) instance.query(''' - DROP TABLE IF EXISTS test.destination; + DROP TABLE test.destination; ''') assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) @@ -1827,68 +1823,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_consumer_restore_failed_connection_without_losses_1(rabbitmq_cluster): - instance.query(''' - CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'consumer_reconnect', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - ''') - - i = 0 - messages_num = 100000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i, 'value': i})) - i += 1 - for message in messages: - channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - - instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.consumer_reconnect; - ''') - - while int(instance.query('SELECT count() FROM test.view')) == 0: - time.sleep(0.1) - - kill_rabbitmq(); - time.sleep(4); - revive_rabbitmq(); - - #collected = int(instance.query('SELECT count() FROM test.view')) - - while True: - result = instance.query('SELECT count(DISTINCT key) FROM test.view') - time.sleep(1) - if int(result) == messages_num: - break - - instance.query(''' - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer_reconnect; - ''') - - assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) - - -@pytest.mark.timeout(420) -def test_rabbitmq_producer_restore_failed_connection_without_losses(rabbitmq_cluster): +def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.consume; DROP TABLE IF EXISTS test.view; @@ -1949,17 +1884,17 @@ def test_rabbitmq_producer_restore_failed_connection_without_losses(rabbitmq_clu break instance.query(''' - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consume; - DROP TABLE IF EXISTS test.producer_reconnect; + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.consume; + DROP TABLE test.producer_reconnect; ''') assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) -def test_rabbitmq_consumer_restore_failed_connection_without_losses_2(rabbitmq_cluster): +def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): instance.query(''' CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -2005,12 +1940,12 @@ def test_rabbitmq_consumer_restore_failed_connection_without_losses_2(rabbitmq_c time.sleep(8); revive_rabbitmq(); - while int(instance.query('SELECT count() FROM test.view')) == 0: - time.sleep(0.1) + #while int(instance.query('SELECT count() FROM test.view')) == 0: + # time.sleep(0.1) - kill_rabbitmq(); - time.sleep(2); - revive_rabbitmq(); + #kill_rabbitmq(); + #time.sleep(2); + #revive_rabbitmq(); while True: result = instance.query('SELECT count(DISTINCT key) FROM test.view') @@ -2019,9 +1954,8 @@ def test_rabbitmq_consumer_restore_failed_connection_without_losses_2(rabbitmq_c break instance.query(''' - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer_reconnect; + DROP TABLE test.consumer; + DROP TABLE test.consumer_reconnect; ''') assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) From e2574da1f5b39ad610dec0fb565860172095150e Mon Sep 17 00:00:00 2001 From: Gao Qiang <30835199+dreamerfable@users.noreply.github.com> Date: Thu, 27 Aug 2020 21:21:43 +0800 Subject: [PATCH 079/535] Update mergetree.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Update all content in this article,fix wrong formats, fix wrong translation,unified description of main concept, remove deprecated content,add the new features. --- .../mergetree-family/mergetree.md | 427 ++++++++++++++---- 1 file changed, 338 insertions(+), 89 deletions(-) diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index e92621c12df..e733994b73d 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -2,44 +2,47 @@ Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及该系列(`*MergeTree`)中的其他引擎。 -`MergeTree` 引擎系列的基本理念如下。当你有巨量数据要插入到表中,你要高效地一批批写入数据片段,并希望这些数据片段在后台按照一定规则合并。相比在插入时不断修改(重写)数据进存储,这种策略会高效很多。 +`MergeTree` 系列的引擎被设计用于插入极大量的数据到一张表当中。数据可以以数据片段的形式一个接着一个的快速写入,数据片段在后台按照一定的规则进行合并。相比在插入时不断修改(重写)已存储的数据,这种策略会高效很多。 主要特点: - 存储的数据按主键排序。 - 这让你可以创建一个用于快速检索数据的小稀疏索引。 + 这使得你能够创建一个小型的稀疏索引来加快数据检索。 -- 允许使用分区,如果指定了 [分区键](custom-partitioning-key.md) 的话。 +- 支持数据分区,如果指定了 [分区键](custom-partitioning-key.md) 的话。 在相同数据集和相同结果集的情况下 ClickHouse 中某些带分区的操作会比普通操作更快。查询中指定了分区键时 ClickHouse 会自动截取分区数据。这也有效增加了查询性能。 - 支持数据副本。 - `ReplicatedMergeTree` 系列的表便是用于此。更多信息,请参阅 [数据副本](replication.md) 一节。 + `ReplicatedMergeTree` 系列的表提供了数据副本功能。更多信息,请参阅 [数据副本](replication.md) 一节。 - 支持数据采样。 需要的话,你可以给表设置一个采样方法。 -!!! 注意 "注意" +!!! note "注意" [合并](../special/merge.md#merge) 引擎并不属于 `*MergeTree` 系列。 ## 建表 {#table_engine-mergetree-creating-a-table} - CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] - ( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... - INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, - INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2 - ) ENGINE = MergeTree() - [PARTITION BY expr] - [ORDER BY expr] - [PRIMARY KEY expr] - [SAMPLE BY expr] - [SETTINGS name=value, ...] +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], + ... + INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, + INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2 +) ENGINE = MergeTree() +ORDER BY expr +[PARTITION BY expr] +[PRIMARY KEY expr] +[SAMPLE BY expr] +[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...] +[SETTINGS name=value, ...] +``` 对于以上参数的描述,可参考 [CREATE 语句 的描述](../../../engines/table-engines/mergetree-family/mergetree.md) 。 @@ -62,7 +65,7 @@ Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及 要按月分区,可以使用表达式 `toYYYYMM(date_column)` ,这里的 `date_column` 是一个 [Date](../../../engines/table-engines/mergetree-family/mergetree.md) 类型的列。分区名的格式会是 `"YYYYMM"` 。 -- `PRIMARY KEY` - 主键,如果要设成 [跟排序键不相同](#xuan-ze-gen-pai-xu-jian-bu-yi-yang-zhu-jian),可选。 +- `PRIMARY KEY` - 主键,如果要 [选择与排序键不同的主键](#choosing-a-primary-key-that-differs-from-the-sorting-key),可选。 默认情况下主键跟排序键(由 `ORDER BY` 子句指定)相同。 因此,大部分情况下不需要再专门指定一个 `PRIMARY KEY` 子句。 @@ -72,17 +75,19 @@ Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及 如果要用抽样表达式,主键中必须包含这个表达式。例如: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))` 。 -- TTL 指定行存储的持续时间并定义 PART 在硬盘和卷上的移动逻辑的规则列表,可选。 +- TTL 指定行存储的持续时间并定义数据片段在硬盘和卷上的移动逻辑的规则列表,可选。 表达式中必须存在至少一个 `Date` 或 `DateTime` 类型的列,比如: `TTL date + INTERVAl 1 DAY` - 规则的类型 `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'`指定了当满足条件(到达当前时间)时所要执行的动作:移除过期的行,还是将 PART (如果PART中的所有行都满足表达式的话)移动到指定的磁盘(`TO DISK 'xxx'`) 或 卷(`TO VOLUME 'xxx'`)。默认的规则是移除(`DELETE`)。可以在列表中指定多个规则,但最多只能有一个`DELETE`的规则。 + 规则的类型 `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'`指定了当满足条件(到达指定时间)时所要执行的动作:移除过期的行,还是将数据片段(如果数据片段中的所有行都满足表达式的话)移动到指定的磁盘(`TO DISK 'xxx'`) 或 卷(`TO VOLUME 'xxx'`)。默认的规则是移除(`DELETE`)。可以在列表中指定多个规则,但最多只能有一个`DELETE`的规则。 + + 更多细节,请查看 [表和列的 TTL](#table_engine-mergetree-ttl) -- `SETTINGS` — 影响 `MergeTree` 性能的额外参数: +- `SETTINGS` — 控制 `MergeTree` 行为的额外参数: - - `index_granularity` — 索引粒度。索引中相邻的『标记』间的数据行数。默认值,8192 。参考[Data Storage](#mergetree-data-storage)。 + - `index_granularity` — 索引粒度。索引中相邻的『标记』间的数据行数。默认值,8192 。参考[数据存储](#mergetree-data-storage)。 - `index_granularity_bytes` — 索引粒度,以字节为单位,默认值: 10Mb。如果想要仅按数据行数限制索引粒度, 请设置为0(不建议)。 - `enable_mixed_granularity_parts` — 是否启用通过 `index_granularity_bytes` 控制索引粒度的大小。在19.11版本之前, 只有 `index_granularity` 配置能够用于限制索引粒度的大小。当从具有很大的行(几十上百兆字节)的表中查询数据时候,`index_granularity_bytes` 配置能够提升ClickHouse的性能。如果你的表里有很大的行,可以开启这项配置来提升`SELECT` 查询的性能。 - `use_minimalistic_part_header_in_zookeeper` — 是否在 ZooKeeper 中启用最小的数据片段头 。如果设置了 `use_minimalistic_part_header_in_zookeeper=1` ,ZooKeeper 会存储更少的数据。更多信息参考『服务配置参数』这章中的 [设置描述](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) 。 @@ -90,18 +95,21 @@ Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及 - `merge_with_ttl_timeout` — TTL合并频率的最小间隔时间,单位:秒。默认值: 86400 (1 天)。 - `write_final_mark` — 是否启用在数据片段尾部写入最终索引标记。默认值: 1(不建议更改)。 - - `storage_policy` — 存储策略。 参见 [使用多个区块装置进行数据存储](#table_engine-mergetree-multiple-volumes). - - `min_bytes_for_wide_part`,`min_rows_for_wide_part` 在数据分段中可以使用`Wide`格式进行存储的最小字节数/行数。你可以不设置、只设置一个,或全都设置。参考:[Data Storage](#mergetree-data-storage) + - `merge_max_block_size` — 在块中进行合并操作时的最大行数限制。默认值:8192 + - `storage_policy` — 存储策略。 参见 [使用具有多个块的设备进行数据存储](#table_engine-mergetree-multiple-volumes). + - `min_bytes_for_wide_part`,`min_rows_for_wide_part` 在数据片段中可以使用`Wide`格式进行存储的最小字节数/行数。你可以不设置、只设置一个,或全都设置。参考:[数据存储](#mergetree-data-storage) **示例配置** - ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 +``` sql +ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 +``` -示例中,我们设为按月分区。 +在这个例子中,我们设置了按月进行分区。 -同时我们设置了一个按用户ID哈希的抽样表达式。这让你可以有该表中每个 `CounterID` 和 `EventDate` 下面的数据的伪随机分布。如果你在查询时指定了 [SAMPLE](../../../engines/table-engines/mergetree-family/mergetree.md#select-sample-clause) 子句。 ClickHouse会返回对于用户子集的一个均匀的伪随机数据采样。 +同时我们设置了一个按用户 ID 哈希的抽样表达式。这使得你可以对该表中每个 `CounterID` 和 `EventDate` 的数据伪随机分布。如果你在查询时指定了 [SAMPLE](../../../engines/table-engines/mergetree-family/mergetree.md#select-sample-clause) 子句。 ClickHouse会返回对于用户子集的一个均匀的伪随机数据采样。 -`index_granularity` 可省略,默认值为 8192 。 +`index_granularity` 可省略因为 8192 是默认设置 。
@@ -133,15 +141,20 @@ Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及 ## 数据存储 {#mergetree-data-storage} -表由按主键排序的数据 *片段* 组成。 +表由按主键排序的数据片段(DATA PART)组成。 -当数据被插入到表中时,会分成数据片段并按主键的字典序排序。例如,主键是 `(CounterID, Date)` 时,片段中数据按 `CounterID` 排序,具有相同 `CounterID` 的部分按 `Date` 排序。 +当数据被插入到表中时,会创建多个数据片段并按主键的字典序排序。例如,主键是 `(CounterID, Date)` 时,片段中数据首先按 `CounterID` 排序,具有相同 `CounterID` 的部分按 `Date` 排序。 -不同分区的数据会被分成不同的片段,ClickHouse 在后台合并数据片段以便更高效存储。不会合并来自不同分区的数据片段。这个合并机制并不保证相同主键的所有行都会合并到同一个数据片段中。 +不同分区的数据会被分成不同的片段,ClickHouse 在后台合并数据片段以便更高效存储。不同分区的数据片段不会进行合并。合并机制并不保证具有相同主键的行全都合并到同一个数据片段中。 -ClickHouse 会为每个数据片段创建一个索引文件,索引文件包含每个索引行(『标记』)的主键值。索引行号定义为 `n * index_granularity` 。最大的 `n` 等于总行数除以 `index_granularity` 的值的整数部分。对于每列,跟主键相同的索引行处也会写入『标记』。这些『标记』让你可以直接找到数据所在的列。 +数据片段可以以 `Wide` 或 `Compact` 格式存储。在 `Wide` 格式下,每一列都会在文件系统中存储为单独的文件,在 `Compact` 格式下所有列都存储在一个文件中。`Compact` 格式可以提高插入量少插入频率频繁时的性能。 -你可以只用一单一大表并不断地一块块往里面加入数据 – `MergeTree` 引擎的就是为了这样的场景。 +数据存储格式由 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part` 表引擎参数控制。如果数据片段中的字节数或行数少于相应的设置值,数据片段会以 `Compact` 格式存储,否则会以 `Wide` 格式存储。 + +每个数据片段被逻辑的分割成颗粒(granules)。颗粒是 ClickHouse 中进行数据查询时的最小不可分割数据集。ClickHouse 不会对行或值进行拆分,所以每个颗粒总是包含整数个行。每个颗粒的第一行通过该行的主键值进行标记, +ClickHouse 会为每个数据片段创建一个索引文件来存储这些标记。对于每列,无论它是否包含在主键当中,ClickHouse 都会存储类似标记。这些标记让你可以在列文件中直接找到数据。 + +颗粒的大小通过表引擎参数 `index_granularity` 和 `index_granularity_bytes` 控制。取决于行的大小,颗粒的行数的在 `[1, index_granularity]` 范围中。如果单行的大小超过了 `index_granularity_bytes` 设置的值,那么一个颗粒的大小会超过 `index_granularity_bytes`。在这种情况下,颗粒的大小等于该行的大小。 ## 主键和索引在查询中的表现 {#primary-keys-and-indexes-in-queries} @@ -162,56 +175,53 @@ ClickHouse 会为每个数据片段创建一个索引文件,索引文件包含 上面例子可以看出使用索引通常会比全表描述要高效。 -稀疏索引会引起额外的数据读取。当读取主键单个区间范围的数据时,每个数据块中最多会多读 `index_granularity * 2` 行额外的数据。大部分情况下,当 `index_granularity = 8192` 时,ClickHouse的性能并不会降级。 +稀疏索引会引起额外的数据读取。当读取主键单个区间范围的数据时,每个数据块中最多会多读 `index_granularity * 2` 行额外的数据。 -稀疏索引让你能操作有巨量行的表。因为这些索引是常驻内存(RAM)的。 +稀疏索引使得你可以处理极大量的行,因为大多数情况下,这些索引常驻与内存(RAM)中。 -ClickHouse 不要求主键惟一。所以,你可以插入多条具有相同主键的行。 +ClickHouse 不要求主键惟一,所以你可以插入多条具有相同主键的行。 ### 主键的选择 {#zhu-jian-de-xuan-ze} -主键中列的数量并没有明确的限制。依据数据结构,你应该让主键包含多些或少些列。这样可以: +主键中列的数量并没有明确的限制。依据数据结构,你可以在主键包含多些或少些列。这样可以: - 改善索引的性能。 - 如果当前主键是 `(a, b)` ,然后加入另一个 `c` 列,满足下面条件时,则可以改善性能: - - 有带有 `c` 列条件的查询。 - - 很长的数据范围( `index_granularity` 的数倍)里 `(a, b)` 都是相同的值,并且这种的情况很普遍。换言之,就是加入另一列后,可以让你的查询略过很长的数据范围。 + 如果当前主键是 `(a, b)` ,在下列情况下添加另一个 `c` 列会提升性能: + + - 查询会使用 `c` 列作为条件 + - 很长的数据范围( `index_granularity` 的数倍)里 `(a, b)` 都是相同的值,并且这样的情况很普遍。换言之,就是加入另一列后,可以让你的查询略过很长的数据范围。 - 改善数据压缩。 - ClickHouse 以主键排序片段数据,所以,数据的一致性越高,压缩越好。 + ClickHouse 以主键排序片段数据,所以,数据的一致性越高,压缩越好。 -- [折叠树](collapsingmergetree.md#table_engine-collapsingmergetree) 和 [SummingMergeTree](summingmergetree.md) 引擎里,数据合并时,会有额外的处理逻辑。 +- 在[CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) 和 [SummingMergeTree](summingmergetree.md) 引擎里进行数据合并时会提供额外的处理逻辑。 - 在这种情况下,指定一个跟主键不同的 *排序键* 也是有意义的。 + 在这种情况下,指定与主键不同的 *排序键* 也是有意义的。 长的主键会对插入性能和内存消耗有负面影响,但主键中额外的列并不影响 `SELECT` 查询的性能。 -### 选择跟排序键不一样主键 {#xuan-ze-gen-pai-xu-jian-bu-yi-yang-zhu-jian} +可以使用 `ORDER BY tuple()` 语法创建没有主键的表。在这种情况下 ClickHouse 根据数据插入的顺序存储。如果在使用 `INSERT ... SELECT` 时希望保持数据的排序,请设置 [max\_insert\_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads)。 -指定一个跟排序键(用于排序数据片段中行的表达式) -不一样的主键(用于计算写到索引文件的每个标记值的表达式)是可以的。 -这种情况下,主键表达式元组必须是排序键表达式元组的一个前缀。 +想要根据初始顺序进行数据查询,使用 [单线程查询](../../../operations/settings/settings.md#settings-max_threads) -当使用 [SummingMergeTree](summingmergetree.md) 和 -[AggregatingMergeTree](aggregatingmergetree.md) 引擎时,这个特性非常有用。 -通常,使用这类引擎时,表里列分两种:*维度* 和 *度量* 。 -典型的查询是在 `GROUP BY` 并过虑维度的情况下统计度量列的值。 -像 SummingMergeTree 和 AggregatingMergeTree ,用相同的排序键值统计行时, -通常会加上所有的维度。结果就是,这键的表达式会是一长串的列组成, -并且这组列还会因为新加维度必须频繁更新。 +### 选择与排序键不同主键 {#choosing-a-primary-key-that-differs-from-the-sorting-key} -这种情况下,主键中仅预留少量列保证高效范围扫描, -剩下的维度列放到排序键元组里。这样是合理的。 +指定一个跟排序键不一样的主键是可以的,此时排序键用于在数据片段中进行排序,主键用于在索引文件中进行标记的写入。这种情况下,主键表达式元组必须是排序键表达式元组的前缀。 -[排序键的修改](../../../engines/table-engines/mergetree-family/mergetree.md) 是轻量级的操作,因为一个新列同时被加入到表里和排序键后时,已存在的数据片段并不需要修改。由于旧的排序键是新排序键的前缀,并且刚刚添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。 +当使用 [SummingMergeTree](summingmergetree.md) 和 [AggregatingMergeTree](aggregatingmergetree.md) 引擎时,这个特性非常有用。通常在使用这类引擎时,表里的列分两种:*维度* 和 *度量* 。典型的查询会通过任意的 `GROUP BY` 对度量列进行聚合并通过维度列进行过滤。由于 SummingMergeTree 和 AggregatingMergeTree 会对排序键相同的行进行聚合,所以把所有的维度放进排序键是很自然的做法。但这将导致排序键中包含大量的列,并且排序键会伴随着新添加的维度不断的更新。 -### 索引和分区在查询中的应用 {#suo-yin-he-fen-qu-zai-cha-xun-zhong-de-ying-yong} +在这种情况下合理的做法是,只保留少量的列在主键当中用于提升扫描效率,将维度列添加到排序键中。 -对于 `SELECT` 查询,ClickHouse 分析是否可以使用索引。如果 `WHERE/PREWHERE` 子句具有下面这些表达式(作为谓词链接一子项或整个)则可以使用索引:基于主键或分区键的列或表达式的部分的等式或比较运算表达式;基于主键或分区键的列或表达式的固定前缀的 `IN` 或 `LIKE` 表达式;基于主键或分区键的列的某些函数;基于主键或分区键的表达式的逻辑表达式。 +对排序键进行 [ALTER](../../../sql-reference/statements/alter/index.md) 是轻量级的操作,因为当一个新列同时被加入到表里和排序键里时,已存在的数据片段并不需要修改。由于旧的排序键是新排序键的前缀,并且新添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。 -因此,在索引键的一个或多个区间上快速地跑查询都是可能的。下面例子中,指定标签;指定标签和日期范围;指定标签和日期;指定多个标签和日期范围等运行查询,都会非常快。 +### 索引和分区在查询中的应用 {#use-of-indexes-and-partitions-in-queries} + +对于 `SELECT` 查询,ClickHouse 分析是否可以使用索引。如果 `WHERE/PREWHERE` 子句具有下面这些表达式(作为谓词链接一子项或整个)则可以使用索引:包含一个表示与主键/分区键中的部分字段或全部字段相等/不等的比较表达式;基于主键/分区键的字段上的 `IN` 或 固定前缀的`LIKE` 表达式;基于主键/分区键的字段上的某些函数;基于主键/分区键的表达式的逻辑表达式。 + + +因此,在索引键的一个或多个区间上快速地执行查询都是可能的。下面例子中,指定标签;指定标签和日期范围;指定标签和日期;指定多个标签和日期范围等执行查询,都会非常快。 当引擎配置如下时: @@ -237,11 +247,18 @@ SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' 要检查 ClickHouse 执行一个查询时能否使用索引,可设置 [force\_index\_by\_date](../../../operations/settings/settings.md#settings-force_index_by_date) 和 [force\_primary\_key](../../../operations/settings/settings.md) 。 -按月分区的分区键是只能读取包含适当范围日期的数据块。这种情况下,数据块会包含很多天(最多整月)的数据。在块中,数据按主键排序,主键第一列可能不包含日期。因此,仅使用日期而没有带主键前缀条件的查询将会导致读取超过这个日期范围。 +按月分区的分区键是只能读取包含适当范围日期的数据块。这种情况下,数据块会包含很多天(最多整月)的数据。在块中,数据按主键排序,主键第一列可能不包含日期。因此,仅使用日期而没有带主键前几个字段作为条件的查询将会导致需要读取超过这个指定日期以外的数据。 -### 跳数索引(分段汇总索引,实验性的) {#tiao-shu-suo-yin-fen-duan-hui-zong-suo-yin-shi-yan-xing-de} +### 部分单调主键的使用 -需要设置 `allow_experimental_data_skipping_indices` 为 1 才能使用此索引。(执行 `SET allow_experimental_data_skipping_indices = 1`)。 +考虑这样的场景,比如一个月中的几天。它们在一个月的范围内形成一个[单调序列](https://zh.wikipedia.org/wiki/单调函数) ,但如果扩展到更大的时间范围它们就不再单调了。这就是一个部分单调序列。如果用户使用部分单调的主键创建表,ClickHouse同样会创建一个稀疏索引。当用户从这类表中查询数据时,ClickHouse 会对查询条件进行分析。如果用户希望获取两个索引标记之间的数据并且这两个标记在一个月以内,ClickHouse 可以在这种特殊情况下使用到索引,因为它可以计算出查询参数与索引标记之间的距离。 + +如果查询参数范围内的主键不是单调序列,那么 ClickHouse 无法使用索引。在这种情况下,ClickHouse 会进行全表扫描。 + +ClickHouse 在任何主键代表一个部分单调序列的情况下都会使用这个逻辑。 + + +### 跳数索引 {#tiao-shu-suo-yin-fen-duan-hui-zong-suo-yin-shi-yan-xing-de} 此索引在 `CREATE` 语句的列部分里定义。 @@ -249,12 +266,14 @@ SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' INDEX index_name expr TYPE type(...) GRANULARITY granularity_value ``` -`*MergeTree` 系列的表都能指定跳数索引。 +`*MergeTree` 系列的表可以指定跳数索引。 这些索引是由数据块按粒度分割后的每部分在指定表达式上汇总信息 `granularity_value` 组成(粒度大小用表引擎里 `index_granularity` 的指定)。 这些汇总信息有助于用 `where` 语句跳过大片不满足的数据,从而减少 `SELECT` 查询从磁盘读取的数据量, -示例 +这些索引会在数据块上聚合指定表达式的信息,这些信息以 granularity_value 指定的粒度组成 (粒度的大小通过在表引擎中定义 index_granularity 定义)。这些汇总信息有助于跳过大片不满足 `where` 条件的数据,从而减少 `SELECT` 查询从磁盘读取的数据量。 + +**示例** ``` sql CREATE TABLE table_name @@ -282,19 +301,27 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 存储指定表达式的极值(如果表达式是 `tuple` ,则存储 `tuple` 中每个元素的极值),这些信息用于跳过数据块,类似主键。 - `set(max_rows)` - 存储指定表达式的惟一值(不超过 `max_rows` 个,`max_rows=0` 则表示『无限制』)。这些信息可用于检查 `WHERE` 表达式是否满足某个数据块。 + 存储指定表达式的不重复值(不超过 `max_rows` 个,`max_rows=0` 则表示『无限制』)。这些信息可用于检查 数据块是否满足 `WHERE` 条件。 - `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` - 存储包含数据块中所有 n 元短语的 [布隆过滤器](https://en.wikipedia.org/wiki/Bloom_filter) 。只可用在字符串上。 + 存储一个包含数据块中所有 n元短语(ngram) 的 [布隆过滤器](https://en.wikipedia.org/wiki/Bloom_filter) 。只可用在字符串上。 可用于优化 `equals` , `like` 和 `in` 表达式的性能。 `n` – 短语长度。 - `size_of_bloom_filter_in_bytes` – 布隆过滤器大小,单位字节。(因为压缩得好,可以指定比较大的值,如256或512)。 - `number_of_hash_functions` – 布隆过滤器中使用的 hash 函数的个数。 - `random_seed` – hash 函数的随机种子。 + `size_of_bloom_filter_in_bytes` – 布隆过滤器大小,单位字节。(因为压缩得好,可以指定比较大的值,如 256 或 512)。 + `number_of_hash_functions` – 布隆过滤器中使用的哈希函数的个数。 + `random_seed` – 哈希函数的随机种子。 - `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` - 跟 `ngrambf_v1` 类似,不同于 ngrams 存储字符串指定长度的所有片段。它只存储被非字母数据字符分割的片段。 + 跟 `ngrambf_v1` 类似,不同于 ngrams 存储字符串指定长度的所有片段。它只存储被非字母数字字符分割的片段。 +- `bloom_filter(bloom_filter([false_positive])` – 为指定的列存储布隆过滤器 + + 可选的参数 false_positive 用来指定从布隆过滤器收到错误响应的几率。取值范围是 (0,1),默认值:0.025 + + 支持的数据类型:`Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`。 + + 以下函数会用到这个索引: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md) + ``` sql @@ -303,17 +330,62 @@ INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100 INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4 ``` -## 并发数据访问 {#bing-fa-shu-ju-fang-wen} +#### 函数支持 {#functions-support} + +WHERE 子句中的条件包含对列的函数调用,如果列是索引的一部分,ClickHouse 会在执行函数时尝试使用索引。不同的函数对索引的支持是不同的。 + +`set` 索引会对所有函数生效,其他索引对函数的生效情况见下表 + +| 函数 (操作符) / 索引 | primary key | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter | +|------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| +| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | +| [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | +| [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [less (\<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | + +常量参数小于 ngram 大小的函数不能使用 `ngrambf_v1` 进行查询优化。 + +!!! note "注意" +布隆过滤器可能会包含不符合条件的匹配,所以 `ngrambf_v1`, `tokenbf_v1` 和 `bloom_filter` 索引不能用于负向的函数,例如: + +- 可以用来优化的场景 + - `s LIKE '%test%'` + - `NOT s NOT LIKE '%test%'` + - `s = 1` + - `NOT s != 1` + - `startsWith(s, 'test')` +- 不能用来优化的场景 + - `NOT s LIKE '%test%'` + - `s NOT LIKE '%test%'` + - `NOT s = 1` + - `s != 1` + - `NOT startsWith(s, 'test')` + +## 并发数据访问 {#concurrent-data-access} 应对表的并发访问,我们使用多版本机制。换言之,当同时读和更新表时,数据从当前查询到的一组片段中读取。没有冗长的的锁。插入不会阻碍读取。 对表的读操作是自动并行的。 -## 列和表的TTL {#table_engine-mergetree-ttl} +## 列和表的 TTL {#table_engine-mergetree-ttl} -TTL可以设置值的生命周期,它既可以为整张表设置,也可以为每个列字段单独设置。如果`TTL`同时作用于表和字段,ClickHouse会使用先到期的那个。 +TTL 可以设置值的生命周期,它既可以为整张表设置,也可以为每个列字段单独设置。表级别的 TTL 还会指定数据在磁盘和卷上自动转移的逻辑。 -被设置TTL的表,必须拥有[日期](../../../engines/table-engines/mergetree-family/mergetree.md) 或 [日期时间](../../../engines/table-engines/mergetree-family/mergetree.md) 类型的字段。要定义数据的生命周期,需要在这个日期字段上使用操作符,例如: +TTL 表达式的计算结果必须是 [日期](../../../engines/table-engines/mergetree-family/mergetree.md) 或 [日期时间](../../../engines/table-engines/mergetree-family/mergetree.md) 类型的字段。 + +示例: ``` sql TTL time_column @@ -327,15 +399,15 @@ TTL date_time + INTERVAL 1 MONTH TTL date_time + INTERVAL 15 HOUR ``` -### 列字段 TTL {#mergetree-column-ttl} +### 列 TTL {#mergetree-column-ttl} -当列字段中的值过期时, ClickHouse会将它们替换成数据类型的默认值。如果分区内,某一列的所有值均已过期,则ClickHouse会从文件系统中删除这个分区目录下的列文件。 +当列中的值过期时, ClickHouse会将它们替换成该列数据类型的默认值。如果数据片段中列的所有值均已过期,则ClickHouse 会从文件系统中的数据片段中此列。 `TTL`子句不能被用于主键字段。 -示例说明: +示例: -创建一张包含 `TTL` 的表 +创建表时指定 `TTL` ``` sql CREATE TABLE example_table @@ -368,11 +440,21 @@ ALTER TABLE example_table ### 表 TTL {#mergetree-table-ttl} -当表内的数据过期时, ClickHouse会删除所有对应的行。 +表可以设置一个用于移除过期行的表达式,以及多个用于在磁盘或卷上自动转移数据片段的表达式。当表中的行过期时,ClickHouse 会删除所有对应的行。对于数据片段的转移特性,必须所有的行都满足转移条件。 -举例说明: +``` sql +TTL expr [DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'], ... +``` -创建一张包含 `TTL` 的表 +TTL 规则的类型紧跟在每个 TTL 表达式后面,它会影响满足表达式时(到达指定时间时)应当执行的操作: + +- `DELETE` - 删除过期的行(默认操作); +- `TO DISK 'aaa'` - 将数据片段移动到磁盘 `aaa`; +- `TO VOLUME 'bbb'` - 将数据片段移动到卷 `bbb`. + +示例: + +创建时指定 TTL ``` sql CREATE TABLE example_table @@ -383,7 +465,9 @@ CREATE TABLE example_table ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY d -TTL d + INTERVAL 1 MONTH; +TTL d + INTERVAL 1 MONTH [DELETE], + d + INTERVAL 1 WEEK TO VOLUME 'aaa', + d + INTERVAL 2 WEEK TO DISK 'bbb'; ``` 修改表的 `TTL` @@ -395,14 +479,179 @@ ALTER TABLE example_table **删除数据** -当ClickHouse合并数据分区时, 会删除TTL过期的数据。 +ClickHouse 在数据片段合并时会删除掉过期的数据。 -当ClickHouse发现数据过期时, 它将会执行一个计划外的合并。要控制这类合并的频率, 你可以设置 `merge_with_ttl_timeout`。如果该值被设置的太低, 它将导致执行许多的计划外合并,这可能会消耗大量资源。 +当ClickHouse发现数据过期时, 它将会执行一个计划外的合并。要控制这类合并的频率, 你可以设置 `merge_with_ttl_timeout`。如果该值被设置的太低, 它将引发大量计划外的合并,这可能会消耗大量资源。 -如果在合并的时候执行`SELECT` 查询, 则可能会得到过期的数据。为了避免这种情况,可以在`SELECT`之前使用 [OPTIMIZE](../../../engines/table-engines/mergetree-family/mergetree.md#misc_operations-optimize) 查询。 +如果在合并的过程中执行 `SELECT` 查询, 则可能会得到过期的数据。为了避免这种情况,可以在 `SELECT` 之前使用 [OPTIMIZE](../../../engines/table-engines/mergetree-family/mergetree.md#misc_operations-optimize) 查询。 -## 使用多个块设备进行数据存储 {#table_engine-mergetree-multiple-volumes} +## 使用具有多个块的设备进行数据存储 {#table_engine-mergetree-multiple-volumes} + +### 介绍 {#introduction} + +MergeTree 系列表引擎可以将数据存储在多块设备上。这对某些可以潜在被划分为“冷”“热”的表来说是很有用的。近期数据被定期的查询但只需要很小的空间。相反,详尽的历史数据很少被用到。如果有多块磁盘可用,那么“热”的数据可以放置在快速的磁盘上(比如 NVMe 固态硬盘或内存),“冷”的数据可以放在相对较慢的磁盘上(比如机械硬盘)。 + +数据片段是 `MergeTree` 引擎表的最小可移动单元。属于同一个数据片段的数据被存储在同一块磁盘上。数据片段会在后台自动的在磁盘间移动,也可以通过 [ALTER](../../../sql-reference/statements/alter/partition.md#alter_move-partition) 查询来移动。 + +### 术语 {#terms} + +- 磁盘 — 挂载到文件系统的块设备 +- 默认磁盘 — 在服务器设置中通过 [path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) 参数指定的数据存储 +- 卷 — 磁盘的等效有序集合 (类似于 [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)) +- 存储策略 — 卷的集合及他们之间的数据移动规则 ### 配置 {#table_engine-mergetree-multiple-volumes_configure} -[来源文章](https://clickhouse.tech/docs/en/operations/table_engines/mergetree/) +磁盘、卷和存储策略应当在主文件 `config.xml` 或 `config.d` 目录中的独立文件中的 `` 标签内定义。 + +配置结构: + +``` xml + + + + /mnt/fast_ssd/clickhouse/ + + + /mnt/hdd1/clickhouse/ + 10485760 + + + /mnt/hdd2/clickhouse/ + 10485760 + + + ... + + + ... + +``` + +标签: + +- `` — 磁盘名,名称必须与其他磁盘不同. +- `path` — 服务器将用来存储数据 (`data` 和 `shadow` 目录) 的路径, 应当以 ‘/’ 结尾. +- `keep_free_space_bytes` — 需要保留的剩余磁盘空间. + +磁盘定义的顺序无关紧要。 + +存储策略配置: + +``` xml + + ... + + + + + disk_name_from_disks_configuration + 1073741824 + + + + + + + 0.2 + + + + + + + + ... + +``` + +标签: + +- `policy_name_N` — 策略名称,不能重复。 +- `volume_name_N` — 卷名称,不能重复。 +- `disk` — 卷中的磁盘。 +- `max_data_part_size_bytes` — 任意卷上的磁盘可以存储的数据片段的最大大小。 +- `move_factor` — 当可用空间少于这个因子时,数据将自动的向下一个卷(如果有的话)移动 (默认值为 0.1)。 + +配置示例: + +``` xml + + ... + + + + + disk1 + disk2 + + + + + + + + fast_ssd + 1073741824 + + + disk1 + + + 0.2 + + + ... + +``` + +在给出的例子中, `hdd_in_order` 策略实现了 [循环制](https://zh.wikipedia.org/wiki/循环制) 方法。因此这个策略只定义了一个卷(`single`),数据片段会以循环的顺序全部存储到它的磁盘上。当有多个类似的磁盘挂载到系统上,但没有配置 RAID 时,这种策略非常有用。请注意一个每个独立的磁盘驱动都并不可靠,你可能需要用 3 或更大的复制因此来补偿它。 + +如果在系统中有不同类型的磁盘可用,可以使用 `moving_from_ssd_to_hdd`。`hot` 卷由 SSD 磁盘(`fast_ssd`)组成,这个卷上可以存储的数据片段的最大大小为 1GB。所有大于 1GB 的数据片段都会被直接存储到 `cold` 卷上,`cold` 卷包含一个名为 `disk1` 的 HDD 磁盘。 +同样,一旦 `fast_ssd` 被填充超过 80%,数据会通过后台进程向 `disk1` 进行转移。 + +存储策略中卷的枚举顺序是很重要的。因为当一个卷被充满时,数据会向下一个卷转移。磁盘的枚举顺序同样重要,因为数据是依次存储在磁盘上的。 + +在创建表时,可以将一个配置好的策略应用到表: + +``` sql +CREATE TABLE table_with_non_default_policy ( + EventDate Date, + OrderID UInt64, + BannerID UInt64, + SearchPhrase String +) ENGINE = MergeTree +ORDER BY (OrderID, BannerID) +PARTITION BY toYYYYMM(EventDate) +SETTINGS storage_policy = 'moving_from_ssd_to_hdd' +``` + +`default` 存储策略意味着只使用一个卷,这个卷只包含一个在 `` 中定义的磁盘。表创建后,它的存储策略就不能改变了。 + +可以通过 [background\_move\_pool\_size](../../../operations/settings/settings.md#background_move_pool_size) 设置调整执行后台任务的线程数。 + +### 详细说明 {#details} + +对于 `MergeTree` 表,数据通过以下不同的方式写入到磁盘当中: + +- 作为插入(`INSERT`查询)的结果 +- 在后台合并和[数据变异](../../../sql-reference/statements/alter/index.md#alter-mutations)期间 +- 当从另一个副本下载时 +- 作为 [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter/partition.md#alter_freeze-partition) 冻结分区的结果 + +除了数据变异和冻结分区以外的情况下,数据按照以下逻辑存储到卷或磁盘上: + +1. 首个卷(按定义顺序)拥有足够的磁盘空间存储数据片段(`unreserved_space > current_part_size`)并且允许存储给定数据片段的大小(`max_data_part_size_bytes > current_part_size`) +2. 在这个数据卷内,紧挨着先前存储数据的那块磁盘之后的磁盘,拥有比数据片段大的剩余空间。(`unreserved_space - keep_free_space_bytes > current_part_size`) + +更进一步,数据变异和分区冻结使用的是 [硬链接](https://en.wikipedia.org/wiki/Hard_link)。不同磁盘之间的硬链接是不支持的,所以在这种情况下数据片段都会被存储到初始化的那一块磁盘上。 + +在后台,数据片段基于剩余空间(`move_factor`参数)根据卷在配置文件中定义的顺序进行转移。数据永远不会从最后一个移出也不会从第一个移入。可以通过系统表 [system.part\_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (字段 `type = MOVE_PART`) 和 [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (字段 `path` 和 `disk`) 来监控后台的移动情况。同时,具体细节可以通过服务器日志查看。 + +用户可以通过 [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter/partition.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。 + +数据移动不会妨碍到数据复制。也就是说,同一张表的不同副本可以指定不同的存储策略。 + +在后台合并和数据变异之后,就的数据片段会在一定时间后被移除 (`old_parts_lifetime`)。在这期间,他们不能被移动到其他的卷或磁盘。也就是说,直到数据片段被完全移除,它们仍然会被磁盘占用空间计算在内。 + +[原始文章](https://clickhouse.tech/docs/en/operations/table_engines/mergetree/) From 4e0c61972109f7c9ffd6962b37e3652e7201bfd8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 28 Aug 2020 08:52:02 +0000 Subject: [PATCH 080/535] Global refactoring --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 56 ++-- .../RabbitMQ/RabbitMQBlockInputStream.h | 14 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 9 +- src/Storages/RabbitMQ/RabbitMQHandler.h | 6 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 - .../ReadBufferFromRabbitMQConsumer.cpp | 81 +++-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 17 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 295 ++++++++++-------- src/Storages/RabbitMQ/StorageRabbitMQ.h | 16 +- .../WriteBufferToRabbitMQProducer.cpp | 39 +-- .../format_schemas/rabbitmq.proto | 8 +- .../test_storage_rabbitmq/rabbitmq_pb2.py | 24 +- .../integration/test_storage_rabbitmq/test.py | 84 +---- 14 files changed, 328 insertions(+), 324 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 16ba14094ac..fee65b65f08 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -14,24 +14,24 @@ namespace DB RabbitMQBlockInputStream::RabbitMQBlockInputStream( StorageRabbitMQ & storage_, const StorageMetadataPtr & metadata_snapshot_, - const Context & context_, - const Names & columns) + const std::shared_ptr & context_, + const Names & columns, + bool ack_in_suffix_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) + , ack_in_suffix(ack_in_suffix_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header(metadata_snapshot->getSampleBlockForColumns( {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered", "_message_id"}, storage.getVirtuals(), storage.getStorageID())) { - if (!storage.getSchemaName().empty()) - context.setSetting("format_schema", storage.getSchemaName()); } RabbitMQBlockInputStream::~RabbitMQBlockInputStream() { - if (!claimed) + if (!buffer) return; storage.pushReadBuffer(buffer); @@ -46,16 +46,29 @@ Block RabbitMQBlockInputStream::getHeader() const void RabbitMQBlockInputStream::readPrefixImpl() { - auto timeout = std::chrono::milliseconds(context.getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); - + auto timeout = std::chrono::milliseconds(context->getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); buffer = storage.popReadBuffer(timeout); - claimed = !!buffer; +} - if (!buffer || finished) + +bool RabbitMQBlockInputStream::needManualChannelUpdate() +{ + if (!buffer) + return false; + + return !buffer->channelUsable() && buffer->channelAllowed() && storage.connectionRunning(); +} + + +void RabbitMQBlockInputStream::updateChannel() +{ + if (!buffer) return; - if (!buffer->channelUsable() && (storage.connectionRunning() || storage.restoreConnection())) - buffer->restoreChannel(storage.getChannel()); + buffer->updateAckTracker(); + + storage.updateChannel(buffer->getChannel()); + buffer->setupChannel(); } @@ -70,7 +83,7 @@ Block RabbitMQBlockInputStream::readImpl() MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, 1); + storage.getFormatName(), *buffer, non_virtual_header, *context, 1); InputPort port(input_format->getPort().getHeader(), input_format.get()); connect(input_format->getPort(), port); @@ -151,7 +164,7 @@ Block RabbitMQBlockInputStream::readImpl() buffer->allowNext(); - if (!new_rows || !checkTimeLimit()) + if (buffer->queueEmpty() || !checkTimeLimit()) break; } @@ -162,9 +175,7 @@ Block RabbitMQBlockInputStream::readImpl() auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); for (const auto & column : virtual_block.getColumnsWithTypeAndName()) - { result_block.insert(column); - } return result_block; } @@ -172,10 +183,19 @@ Block RabbitMQBlockInputStream::readImpl() void RabbitMQBlockInputStream::readSuffixImpl() { - if (!buffer) - return; + if (ack_in_suffix) + sendAck(); +} - buffer->ackMessages(); +bool RabbitMQBlockInputStream::sendAck() +{ + if (!buffer || !buffer->channelUsable()) + return false; + + if (!buffer->ackMessages()) + return false; + + return true; } } diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h index f4405ce44df..08cfe090c6e 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h @@ -16,8 +16,9 @@ public: RabbitMQBlockInputStream( StorageRabbitMQ & storage_, const StorageMetadataPtr & metadata_snapshot_, - const Context & context_, - const Names & columns); + const std::shared_ptr & context_, + const Names & columns, + bool ack_in_suffix = true); ~RabbitMQBlockInputStream() override; @@ -28,13 +29,18 @@ public: Block readImpl() override; void readSuffixImpl() override; + void updateChannel(); + bool needManualChannelUpdate(); + bool sendAck(); + private: StorageRabbitMQ & storage; StorageMetadataPtr metadata_snapshot; - Context context; + const std::shared_ptr context; Names column_names; + bool ack_in_suffix; + bool finished = false; - bool claimed = false; const Block non_virtual_header; const Block virtual_header; diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 4fd5836b1a9..8e05b10fa47 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -33,7 +33,7 @@ Block RabbitMQBlockOutputStream::getHeader() const void RabbitMQBlockOutputStream::writePrefix() { - if (storage.checkBridge()) + if (!storage.exchangeRemoved()) storage.unbindExchange(); buffer = storage.createWriteBuffer(); diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index d5b0a7d5c02..d08b4806db8 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -13,17 +13,16 @@ RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : loop(loop_), log(log_), connection_running(false), + loop_running(false), loop_state(Loop::STOP) { } ///Method that is called when the connection ends up in an error state. -void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * message) +void RabbitMQHandler::onError(AMQP::TcpConnection * /* connection */, const char * message) { LOG_ERROR(log, "Library error report: {}", message); connection_running.store(false); - if (connection) - connection->close(); } void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) @@ -36,11 +35,15 @@ void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); + LOG_DEBUG(log, "Background loop started"); + loop_running.store(true); + while (loop_state.load() == Loop::RUN) uv_run(loop, UV_RUN_NOWAIT); LOG_DEBUG(log, "Background loop ended"); + loop_running.store(false); } void RabbitMQHandler::iterateLoop() diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 3c0c5a2af37..51cfbdc1144 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -22,12 +22,16 @@ class RabbitMQHandler : public AMQP::LibUvHandler public: RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_); + void onError(AMQP::TcpConnection * connection, const char * message) override; void onReady(AMQP::TcpConnection * connection) override; void startLoop(); void iterateLoop(); + bool connectionRunning() { return connection_running.load(); } + bool loopRunning() { return loop_running.load(); } + void updateLoopState(UInt8 state) { loop_state.store(state); } UInt8 getLoopState() { return loop_state.load(); } @@ -35,7 +39,7 @@ private: uv_loop_t * loop; Poco::Logger * log; - std::atomic connection_running; + std::atomic connection_running, loop_running; std::atomic loop_state; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 2416a15f65a..bc71a929e8f 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -18,7 +18,6 @@ namespace DB M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \ M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ - M(Bool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ M(String, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ M(String, rabbitmq_deadletter_exchange, "", "Exchange name to be passed as a dead-letter-exchange name.", 0) \ M(Bool, rabbitmq_persistent_mode, false, "If set, delivery mode will be set to 2 (makes messages 'persistent', durable).", 0) \ diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 86a39a95c1b..80a630117d8 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -50,26 +50,12 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , row_delimiter(row_delimiter_) , stopped(stopped_) , received(QUEUE_SIZE * num_queues) + , last_inserted_record(AckTracker()) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) bindQueue(queue_id); - consumer_channel->onReady([&]() - { - channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; - LOG_TRACE(log, "Channel {} is created", channel_id); - - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Channel {} error: {}", channel_id, message); - channel_error.store(true); - }); - - updateAckTracker(AckTracker()); - subscribe(); - - channel_error.store(false); - }); + setupChannel(); } @@ -93,7 +79,7 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for - * fanout exchange it can be arbitrary. + * fanout exchange it can be arbitrary */ setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id_base)) .onSuccess([&] @@ -118,7 +104,7 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) } /* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one - * specific queue when its name is specified in queue_base setting. + * specific queue when its name is specified in queue_base setting */ const String queue_name = !hash_exchange ? queue_base : std::to_string(channel_id_base) + "_" + std::to_string(queue_id) + "_" + queue_base; setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); @@ -138,6 +124,9 @@ void ReadBufferFromRabbitMQConsumer::subscribe() .onSuccess([&](const std::string & /* consumer_tag */) { LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); + + if (++subscribed == queues.size()) + wait_subscription.store(false); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { @@ -155,39 +144,39 @@ void ReadBufferFromRabbitMQConsumer::subscribe() }) .onError([&](const char * message) { + /* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which + * arises from queue settings mismatch or queue level error, which should not happen as noone else is supposed to touch them + */ LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); + wait_subscription.store(false); }); } } -void ReadBufferFromRabbitMQConsumer::ackMessages() +bool ReadBufferFromRabbitMQConsumer::ackMessages() { - /* Delivery tags are scoped per channel, so if channel fails, then all previous delivery tags become invalid. Also this check ensures - * that there is no data race with onReady callback in restoreChannel() (they can be called at the same time from different threads). - * And there is no need to synchronize this method with updateAckTracker() as they are not supposed to be called at the same time. - */ - if (channel_error.load()) - return; - AckTracker record = last_inserted_record; - /// Do not send ack to server if message's channel is not the same as current running channel. - if (record.channel_id == channel_id && record.delivery_tag && record.delivery_tag > prev_tag && event_handler->connectionRunning()) + /* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per + * channel, so if channel fails, all previous delivery tags become invalid + */ + if (record.channel_id == channel_id && record.delivery_tag && record.delivery_tag > prev_tag) { - consumer_channel->ack(record.delivery_tag, AMQP::multiple); /// Will ack all up to last tag starting from last acked. - prev_tag = record.delivery_tag; + /// Commit all received messages with delivery tags from last commited to last inserted + if (!consumer_channel->ack(record.delivery_tag, AMQP::multiple)) + return false; - LOG_TRACE(log, "Consumer acknowledged messages with deliveryTags up to {} on the channel {}", record.delivery_tag, channel_id); + prev_tag = record.delivery_tag; + LOG_TRACE(log, "Consumer acknowledged messages with deliveryTags up to {} on channel {}", record.delivery_tag, channel_id); } + + return true; } void ReadBufferFromRabbitMQConsumer::updateAckTracker(AckTracker record) { - /* This method can be called from readImpl and from channel->onError() callback, but channel_error check ensures that it is not done - * at the same time, so no synchronization needed. - */ if (record.delivery_tag && channel_error.load()) return; @@ -198,29 +187,31 @@ void ReadBufferFromRabbitMQConsumer::updateAckTracker(AckTracker record) } -void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) +void ReadBufferFromRabbitMQConsumer::setupChannel() { - consumer_channel = std::move(new_channel); + wait_subscription.store(true); + consumer_channel->onReady([&]() { /* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer, * i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that - * channel_id is unique for each table. + * channel_id is unique for each table */ channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_TRACE(log, "Channel {} is created", channel_id); - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Channel {} error: {}", channel_id, message); - channel_error.store(true); - }); - - updateAckTracker(AckTracker()); + subscribed = 0; subscribe(); - channel_error.store(false); }); + + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Channel {} error: {}", channel_id, message); + + channel_error.store(true); + wait_subscription.store(false); + }); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index e00e8172509..5524a5b52cc 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -56,12 +56,18 @@ public: AckTracker track; }; - void allowNext() { allowed = true; } // Allow to read next message. bool channelUsable() { return !channel_error.load(); } - void restoreChannel(ChannelPtr new_channel); + /// Do not allow to update channel untill current channel is properly set up and subscribed + bool channelAllowed() { return !wait_subscription.load(); } - void ackMessages(); - void updateAckTracker(AckTracker record); + ChannelPtr & getChannel() { return consumer_channel; } + void setupChannel(); + + bool ackMessages(); + void updateAckTracker(AckTracker record = AckTracker()); + + bool queueEmpty() { return received.empty(); } + void allowNext() { allowed = true; } // Allow to read next message. auto getChannelID() const { return current.track.channel_id; } auto getDeliveryTag() const { return current.track.delivery_tag; } @@ -93,10 +99,11 @@ private: const std::atomic & stopped; String channel_id; - std::atomic channel_error = true; + std::atomic channel_error = true, wait_subscription = false; std::vector queues; ConcurrentBoundedQueue received; MessageData current; + size_t subscribed = 0; AckTracker last_inserted_record; UInt64 prev_tag = 0, channel_id_counter = 0; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 4db2d75cd38..b78c21ae96d 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -78,7 +78,6 @@ StorageRabbitMQ::StorageRabbitMQ( const bool persistent_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) - , rabbitmq_context(Context(global_context)) , routing_keys(global_context.getMacros()->expand(routing_keys_)) , exchange_name(exchange_name_) , format_name(global_context.getMacros()->expand(format_name_)) @@ -99,23 +98,15 @@ StorageRabbitMQ::StorageRabbitMQ( loop = std::make_unique(); uv_loop_init(loop.get()); event_handler = std::make_shared(loop.get(), log); - connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - size_t cnt_retries = 0; - while (!connection->ready() && ++cnt_retries != RETRIES_MAX) - { - event_handler->iterateLoop(); - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); - } - - if (!connection->ready()) + if (!restoreConnection(false)) { if (!connection->closed()) connection->close(true); + throw Exception("Cannot connect to RabbitMQ", ErrorCodes::CANNOT_CONNECT_RABBITMQ); } - rabbitmq_context.makeQueryContext(); StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); @@ -153,14 +144,14 @@ StorageRabbitMQ::StorageRabbitMQ( if (queue_base.empty()) { /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to - * be table_name and not just a random string, because local exchanges should be declared the same for same tables. + * be table_name and not just a random string, because local exchanges should be declared the same for same tables */ sharding_exchange = exchange_name + "_" + table_name; /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base - * for the names of later declared queues. + * for the names of later declared queues */ queue_base = table_name; } @@ -168,7 +159,7 @@ StorageRabbitMQ::StorageRabbitMQ( { /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need to share - * sharding exchange and bridge exchange. + * sharding exchange and bridge exchange */ sharding_exchange = exchange_name + "_" + queue_base; } @@ -186,7 +177,6 @@ void StorageRabbitMQ::heartbeatFunc() { if (!stream_cancelled && event_handler->connectionRunning()) { - LOG_TRACE(log, "Sending RabbitMQ heartbeat"); connection->heartbeat(); heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); } @@ -196,17 +186,14 @@ void StorageRabbitMQ::heartbeatFunc() void StorageRabbitMQ::loopingFunc() { if (event_handler->connectionRunning()) - { - LOG_DEBUG(log, "Starting event looping iterations"); event_handler->startLoop(); - } } void StorageRabbitMQ::initExchange() { /* Binding scheme is the following: client's exchange -> key bindings by routing key list -> bridge exchange (fanout) -> - * -> sharding exchange (only if needed) -> queues. + * -> sharding exchange (only if needed) -> queues */ setup_channel->declareExchange(exchange_name, exchange_type, AMQP::durable) .onError([&](const char * message) @@ -215,7 +202,7 @@ void StorageRabbitMQ::initExchange() + std::string(message), ErrorCodes::LOGICAL_ERROR); }); - /// Bridge exchange is needed to easily disconnect consumer queues and also simplifies queue bindings. + /// Bridge exchange is needed to easily disconnect consumer queues and also simplifies queue bindings setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) .onError([&](const char * message) { @@ -229,7 +216,7 @@ void StorageRabbitMQ::initExchange() } /* Change hash property because by default it will be routing key, which has to be an integer, but with support for any exchange - * type - routing keys might be of any type. + * type - routing keys might be of any type */ AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; @@ -313,23 +300,66 @@ void StorageRabbitMQ::bindExchange() } +bool StorageRabbitMQ::restoreConnection(bool reconnecting) +{ + size_t cnt_retries = 0; + + if (reconnecting) + { + heartbeat_task->deactivate(); + connection->close(); /// Connection might be unusable, but not closed + + /* Connection is not closed immediately (firstly, all pending operations are completed, and then + * an AMQP closing-handshake is performed). But cannot open a new connection untill previous one is properly closed + */ + while (!connection->closed() && ++cnt_retries != RETRIES_MAX) + event_handler->iterateLoop(); + + /// This will force immediate closure if not yet closed + if (!connection->closed()) + connection->close(true); + + LOG_TRACE(log, "Trying to restore consumer connection"); + } + + connection = std::make_shared(event_handler.get(), + AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + + cnt_retries = 0; + while (!connection->ready() && ++cnt_retries != RETRIES_MAX) + { + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } + + return event_handler->connectionRunning(); +} + + +void StorageRabbitMQ::updateChannel(ChannelPtr & channel) +{ + channel = std::make_shared(connection.get()); +} + + void StorageRabbitMQ::unbindExchange() { - /* This is needed because with RabbitMQ can't (without special adjustments) use the same table for reading and writing (alternating them), - * because publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in + /* This is needed because with RabbitMQ (without special adjustments) can't, for example, properly make mv if there was insert query + * on the same table before, and in another direction it will make redundant copies, but most likely nobody will do that. + * As publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local - * queues, then messages will go both ways and in one of them they will remain not consumed. Therefore, if insert query is called, need - * to desconnect local consumers, but then MV cannot be afterwards created on the same table. It can be reverted to allow alternating - * these queries, but it will be ugly and seems pointless because probably nobody uses tables alternating INSERT and MV queries on the - * same table. + * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange + * bindings to remove redunadant message copies, but after that mv cannot work unless thoso bindings recreated. Recreating them is not + * difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. */ std::call_once(flag, [&]() { + heartbeat_task->deactivate(); + streaming_task->deactivate(); event_handler->updateLoopState(Loop::STOP); looping_task->deactivate(); - heartbeat_task->deactivate(); setup_channel->removeExchange(bridge_exchange) .onSuccess([&]() @@ -349,73 +379,6 @@ void StorageRabbitMQ::unbindExchange() } -bool StorageRabbitMQ::restoreConnection() -{ - if (restore_connection.try_lock()) - { - /// This lock is to synchronize with getChannel(). - std::lock_guard lk(connection_mutex); - - if (!event_handler->connectionRunning()) - { - /// Stopping loop now and not right after connection error, because need to run it to let it properly close connection. - if (event_handler->getLoopState() == Loop::RUN) - { - event_handler->updateLoopState(Loop::STOP); - looping_task->deactivate(); - heartbeat_task->deactivate(); - } - - /* connection->close() is called in onError() method (which is called by the AMQP library when a fatal error occurs on the - * connection) inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then - * an AMQP closing-handshake is performed). But cannot open a new connection untill previous one is properly closed. - */ - size_t cnt_retries = 0; - while (!connection->closed() && ++cnt_retries != RETRIES_MAX) - event_handler->iterateLoop(); - - /// This will force immediate closure if not yet closed. - if (!connection->closed()) - connection->close(true); - - LOG_TRACE(log, "Trying to restore consumer connection"); - connection = std::make_shared(event_handler.get(), - AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - - cnt_retries = 0; - while (!connection->ready() && ++cnt_retries != RETRIES_MAX) - { - event_handler->iterateLoop(); - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); - } - - if (event_handler->connectionRunning()) - { - looping_task->activateAndSchedule(); - heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); - } - } - - restore_connection.unlock(); - } - else - { - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); - } - - return event_handler->connectionRunning(); -} - - -ChannelPtr StorageRabbitMQ::getChannel() -{ - std::lock_guard lk(connection_mutex); - ChannelPtr new_channel = std::make_shared(connection.get()); - - return new_channel; -} - - Pipe StorageRabbitMQ::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -428,20 +391,55 @@ Pipe StorageRabbitMQ::read( if (num_created_consumers == 0) return {}; + auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + + auto new_context = std::make_shared(context); + if (!schema_name.empty()) + new_context->setSetting("format_schema", schema_name); + + bool update_channels = false; + if (!event_handler->connectionRunning()) + { + if (event_handler->loopRunning()) + { + event_handler->updateLoopState(Loop::STOP); + looping_task->deactivate(); + } + + if ((update_channels = restoreConnection(true))) + heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); + } + Pipes pipes; pipes.reserve(num_created_consumers); - auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); for (size_t i = 0; i < num_created_consumers; ++i) { - auto rabbit_stream = std::make_shared( - *this, metadata_snapshot, context, column_names); + auto rabbit_stream = std::make_shared(*this, metadata_snapshot, new_context, column_names); + + /* It is a possible but rare case when channel gets into error state and does not also close connection, so need manual update. + * But I believe that in current context and with local rabbitmq settings this will never happen and any channel error will also + * close connection, but checking anyway (in second condition of if statement). This must be done here (and also in streamToViews()) + * and not in readPrefix as it requires to stop heartbeats and looping tasks to avoid race conditions inside the library + */ + if (update_channels || rabbit_stream->needManualChannelUpdate()) + { + if (event_handler->loopRunning()) + { + event_handler->updateLoopState(Loop::STOP); + looping_task->deactivate(); + heartbeat_task->deactivate(); + } + + rabbit_stream->updateChannel(); + } + auto converting_stream = std::make_shared( rabbit_stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Name); pipes.emplace_back(std::make_shared(converting_stream)); } - if (!std::exchange(loop_started, true)) + if (!event_handler->loopRunning() && event_handler->connectionRunning()) looping_task->activateAndSchedule(); LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); @@ -484,23 +482,21 @@ void StorageRabbitMQ::startup() void StorageRabbitMQ::shutdown() { stream_cancelled = true; - event_handler->updateLoopState(Loop::STOP); wait_confirm.store(false); - looping_task->deactivate(); streaming_task->deactivate(); heartbeat_task->deactivate(); + event_handler->updateLoopState(Loop::STOP); + looping_task->deactivate(); + connection->close(); size_t cnt_retries = 0; - while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) - { + while (!connection->closed() && ++cnt_retries != RETRIES_MAX) event_handler->iterateLoop(); - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP >> 3)); - } - /// Should actually force closure, if not yet closed, but it generates distracting error logs. + /// Should actually force closure, if not yet closed, but it generates distracting error logs //if (!connection->closed()) // connection->close(true); @@ -558,7 +554,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - ++producer_id, unique_strbase, persistent, wait_confirm, log, + producer_id.fetch_add(1), unique_strbase, persistent, wait_confirm, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } @@ -636,29 +632,38 @@ bool StorageRabbitMQ::streamToViews() auto insert = std::make_shared(); insert->table_id = table_id; - InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true); + auto rabbitmq_context = std::make_shared(global_context); + rabbitmq_context->makeQueryContext(); + if (!schema_name.empty()) + rabbitmq_context->setSetting("format_schema", schema_name); + + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, *rabbitmq_context, false, true, true); auto block_io = interpreter.execute(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto column_names = block_io.out->getHeader().getNames(); + auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + + if (!event_handler->loopRunning() && event_handler->connectionRunning()) + looping_task->activateAndSchedule(); + // Create a stream for each consumer and join them in a union stream BlockInputStreams streams; streams.reserve(num_created_consumers); - auto metadata_snapshot = getInMemoryMetadataPtr(); - auto column_names = block_io.out->getHeader().getNames(); - auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); for (size_t i = 0; i < num_created_consumers; ++i) { - auto rabbit_stream = std::make_shared(*this, metadata_snapshot, rabbitmq_context, column_names); - auto converting_stream = std::make_shared(rabbit_stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Name); - - streams.emplace_back(converting_stream); + auto stream = std::make_shared(*this, metadata_snapshot, rabbitmq_context, column_names, false); + streams.emplace_back(stream); // Limit read batch to maximum block size to allow DDL IBlockInputStream::LocalLimits limits; - const Settings & settings = global_context.getSettingsRef(); - limits.speed_limits.max_execution_time = settings.stream_flush_interval_ms; + + limits.speed_limits.max_execution_time = global_context.getSettingsRef().stream_flush_interval_ms; limits.timeout_overflow_mode = OverflowMode::BREAK; - rabbit_stream->setLimits(limits); + + stream->setLimits(limits); } // Join multiple streams if necessary @@ -668,12 +673,56 @@ bool StorageRabbitMQ::streamToViews() else in = streams[0]; - if (!std::exchange(loop_started, true)) - looping_task->activateAndSchedule(); - std::atomic stub = {false}; copyData(*in, *block_io.out, &stub); + /* Need to stop loop even if connection is ok, because sending ack() with loop running in another thread will lead to a lot of data + * races inside the library, but only in case any error occurs or connection is lost while ack is being sent + */ + if (event_handler->loopRunning()) + { + event_handler->updateLoopState(Loop::STOP); + looping_task->deactivate(); + } + + if (!event_handler->connectionRunning()) + { + if (restoreConnection(true)) + { + for (auto & stream : streams) + stream->as()->updateChannel(); + + } + else + { + /// Reschedule if unable to connect to rabbitmq + return false; + } + } + else + { + heartbeat_task->deactivate(); + + /// Commit + for (auto & stream : streams) + { + if (!stream->as()->sendAck()) + { + /* Almost any error with channel will lead to connection closure, but if so happens that channel errored and connection + * is not closed - also need to restore channels + */ + if (!stream->as()->needManualChannelUpdate()) + stream->as()->updateChannel(); + else + break; + } + } + } + + event_handler->updateLoopState(Loop::RUN); + looping_task->activateAndSchedule(); + heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); /// It is also deactivated in restoreConnection(), so reschedule anyway + // Check whether the limits were applied during query execution bool limits_applied = false; const BlockStreamProfileInfo & info = in->getProfileInfo(); @@ -808,10 +857,6 @@ void registerStorageRabbitMQ(StorageFactory & factory) { exchange_type = safeGet(ast->value); } - - if (exchange_type != "fanout" && exchange_type != "direct" && exchange_type != "topic" - && exchange_type != "headers" && exchange_type != "consistent_hash") - throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); } UInt64 num_consumers = rabbitmq_settings.rabbitmq_num_consumers; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 4c83257209c..01592f11e20 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -55,15 +55,14 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; - const auto & getSchemaName() const { return schema_name; } const String getExchange() const { return exchange_name; } - bool checkBridge() const { return !exchange_removed.load(); } void unbindExchange(); + bool exchangeRemoved() { return exchange_removed.load(); } bool connectionRunning() { return event_handler->connectionRunning(); } - bool restoreConnection(); - ChannelPtr getChannel(); + bool restoreConnection(bool reconnecting); + void updateChannel(ChannelPtr & channel); protected: StorageRabbitMQ( @@ -85,7 +84,6 @@ protected: private: Context global_context; - Context rabbitmq_context; Names routing_keys; const String exchange_name; @@ -117,11 +115,10 @@ private: String unique_strbase; String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; - size_t producer_id = 0, consumer_id = 0; - bool loop_started = false; - std::atomic exchange_removed = false, wait_confirm = true; + size_t consumer_id = 0; + std::atomic producer_id = 1; + std::atomic wait_confirm = true, exchange_removed = false; ChannelPtr setup_channel; - std::mutex connection_mutex, restore_connection; BackgroundSchedulePool::TaskHolder streaming_task; BackgroundSchedulePool::TaskHolder heartbeat_task; @@ -134,6 +131,7 @@ private: void threadFunc(); void heartbeatFunc(); void loopingFunc(); + void initExchange(); void bindExchange(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 6b8670fe9e7..945ebd5ac9a 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -120,19 +120,20 @@ void WriteBufferToRabbitMQProducer::countRow() bool WriteBufferToRabbitMQProducer::setupConnection(bool reconnecting) { size_t cnt_retries = 0; + if (reconnecting) { - /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) - * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP - * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed. - */ - while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + connection->close(); + + while (!connection->closed() && ++cnt_retries != RETRIES_MAX) event_handler->iterateLoop(); + if (!connection->closed()) connection->close(true); + + LOG_TRACE(log, "Trying to set up connection"); } - LOG_TRACE(log, "Trying to set up connection"); connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); @@ -143,7 +144,7 @@ bool WriteBufferToRabbitMQProducer::setupConnection(bool reconnecting) std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); } - return connection->ready(); + return event_handler->connectionRunning(); } @@ -159,7 +160,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->close(); /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once - * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid. + * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid */ for (const auto & record : delivery_record) returned.tryPush(record.second); @@ -235,31 +236,31 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue payload; /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged - * messages cannot exceed returned.size(), because they all might end up there. + * messages cannot exceed returned.size(), because they all might end up there */ while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) { messages.pop(payload); AMQP::Envelope envelope(payload.second.data(), payload.second.size()); - /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty. + /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty AMQP::Table message_settings = key_arguments; /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the * server, then it means that publisher will never know whether those messages were delivered or not, and therefore those records * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To - * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata. + * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata */ message_settings["republished"] = std::to_string(republishing); envelope.setHeaders(message_settings); /* Adding here a messageID property to message metadata. Since RabbitMQ does not guarantee exactly-once delivery, then on the * consumer side "republished" field of message metadata can be checked and, if it set to 1, consumer might also check "messageID" - * property. This way detection of duplicates is guaranteed. + * property. This way detection of duplicates is guaranteed */ envelope.setMessageID(std::to_string(payload.first)); - /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse if (persistent) envelope.setDeliveryMode(2); @@ -276,11 +277,11 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueuepublish(exchange_name, routing_keys[0], envelope); } - /// This is needed for "publisher confirms", which guarantees at-least-once delivery. + /// This is needed for "publisher confirms", which guarantees at-least-once delivery ++delivery_tag; delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - /// Need to break at some point to let event loop run, because no publishing actually happens before looping. + /// Need to break at some point to let event loop run, because no publishing actually happens before looping if (delivery_tag % BATCH == 0) break; } @@ -291,11 +292,11 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable()) publish(returned, true); @@ -306,11 +307,11 @@ void WriteBufferToRabbitMQProducer::writingFunc() /* wait_num != 0 if there will be no new payloads pushed to payloads.queue in countRow(), delivery_record is empty if there are * no more pending acknowldgements from the server (if receieved ack(), records are deleted, if received nack(), records are pushed - * to returned.queue and deleted, because server will attach new delivery tags to them). + * to returned.queue and deleted, because server will attach new delivery tags to them) */ if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) wait_all = false; - else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection(true))) + else if ((!producer_channel->usable() && event_handler->connectionRunning()) || (!event_handler->connectionRunning() && setupConnection(true))) setupChannel(); } diff --git a/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto b/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto index 96b24be4938..44a29facd13 100644 --- a/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto +++ b/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto @@ -1,6 +1,6 @@ syntax = "proto3"; -message KeyValuePair { - uint64 key = 1; - string value = 2; -} \ No newline at end of file + message KeyValueProto { + uint64 key = 1; + string value = 2; + } diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index fb0f1413eac..bd03d3c21d6 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -19,29 +19,29 @@ DESCRIPTOR = _descriptor.FileDescriptor( syntax='proto3', serialized_options=None, create_key=_descriptor._internal_create_key, - serialized_pb=b'\n-clickhouse_path/format_schemas/rabbitmq.proto\"*\n\x0cKeyValuePair\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' + serialized_pb=b'\n-clickhouse_path/format_schemas/rabbitmq.proto\"+\n\rKeyValueProto\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' ) -_KEYVALUEPAIR = _descriptor.Descriptor( - name='KeyValuePair', - full_name='KeyValuePair', +_KEYVALUEPROTO = _descriptor.Descriptor( + name='KeyValueProto', + full_name='KeyValueProto', filename=None, file=DESCRIPTOR, containing_type=None, create_key=_descriptor._internal_create_key, fields=[ _descriptor.FieldDescriptor( - name='key', full_name='KeyValuePair.key', index=0, + name='key', full_name='KeyValueProto.key', index=0, number=1, type=4, cpp_type=4, label=1, has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key), _descriptor.FieldDescriptor( - name='value', full_name='KeyValuePair.value', index=1, + name='value', full_name='KeyValueProto.value', index=1, number=2, type=9, cpp_type=9, label=1, has_default_value=False, default_value=b"".decode('utf-8'), message_type=None, enum_type=None, containing_type=None, @@ -60,18 +60,18 @@ _KEYVALUEPAIR = _descriptor.Descriptor( oneofs=[ ], serialized_start=49, - serialized_end=91, + serialized_end=92, ) -DESCRIPTOR.message_types_by_name['KeyValuePair'] = _KEYVALUEPAIR +DESCRIPTOR.message_types_by_name['KeyValueProto'] = _KEYVALUEPROTO _sym_db.RegisterFileDescriptor(DESCRIPTOR) -KeyValuePair = _reflection.GeneratedProtocolMessageType('KeyValuePair', (_message.Message,), { - 'DESCRIPTOR' : _KEYVALUEPAIR, +KeyValueProto = _reflection.GeneratedProtocolMessageType('KeyValueProto', (_message.Message,), { + 'DESCRIPTOR' : _KEYVALUEPROTO, '__module__' : 'clickhouse_path.format_schemas.rabbitmq_pb2' - # @@protoc_insertion_point(class_scope:KeyValuePair) + # @@protoc_insertion_point(class_scope:KeyValueProto) }) -_sym_db.RegisterMessage(KeyValuePair) +_sym_db.RegisterMessage(KeyValueProto) # @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 503396188b5..0a328301baa 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -328,7 +328,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'pb', rabbitmq_format = 'Protobuf', - rabbitmq_schema = 'rabbitmq.proto:KeyValuePair'; + rabbitmq_schema = 'rabbitmq.proto:KeyValueProto'; ''') credentials = pika.PlainCredentials('root', 'clickhouse') @@ -338,7 +338,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): data = '' for i in range(0, 20): - msg = rabbitmq_pb2.KeyValuePair() + msg = rabbitmq_pb2.KeyValueProto() msg.key = i msg.value = str(i) serialized_msg = msg.SerializeToString() @@ -346,7 +346,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): channel.basic_publish(exchange='pb', routing_key='', body=data) data = '' for i in range(20, 21): - msg = rabbitmq_pb2.KeyValuePair() + msg = rabbitmq_pb2.KeyValueProto() msg.key = i msg.value = str(i) serialized_msg = msg.SerializeToString() @@ -354,7 +354,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): channel.basic_publish(exchange='pb', routing_key='', body=data) data = '' for i in range(21, 50): - msg = rabbitmq_pb2.KeyValuePair() + msg = rabbitmq_pb2.KeyValueProto() msg.key = i msg.value = str(i) serialized_msg = msg.SerializeToString() @@ -1583,7 +1583,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_queue_resume(rabbitmq_cluster): +def test_rabbitmq_no_loss_on_table_drop(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1655,7 +1655,7 @@ def test_rabbitmq_queue_resume(rabbitmq_cluster): while True: result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) - if int(result1) >= messages_num * threads_num: + if int(result1) == messages_num * threads_num: break instance.query(''' @@ -1664,77 +1664,7 @@ def test_rabbitmq_queue_resume(rabbitmq_cluster): DROP TABLE test.view; ''') - assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - -@pytest.mark.timeout(420) -def test_rabbitmq_no_loss_on_table_drop(rabbitmq_cluster): - instance.query(''' - CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'consumer_acks', - rabbitmq_queue_base = 'consumer_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - ''') - - i = 0 - messages_num = 100000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i, 'value': i})) - i += 1 - for message in messages: - channel.basic_publish(exchange='consumer_acks', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - - instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq_consumer_acks; - ''') - - while int(instance.query('SELECT count() FROM test.view')) == 0: - time.sleep(1) - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; - ''') - - #collected = int(instance.query('SELECT count() FROM test.view')) - - instance.query(''' - CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_queue_base = 'consumer_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - ''') - - while True: - result = instance.query('SELECT count(DISTINCT key) FROM test.view') - time.sleep(1) - if int(result) == messages_num: - break - - instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; - DROP TABLE test.rabbitmq_consumer_acks; - ''') - - assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) From 00c8dce39c3d9644c6bd7e8e3e1939ef06e0b432 Mon Sep 17 00:00:00 2001 From: Sergei Shtykov Date: Fri, 28 Aug 2020 14:01:33 +0300 Subject: [PATCH 081/535] CLICKHOUSEDOCS-744: Fixed VersionedCollapsingMergeTree description. --- .../mergetree-family/versionedcollapsingmergetree.md | 2 +- .../mergetree-family/versionedcollapsingmergetree.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md index a010a395c64..b23139b402b 100644 --- a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md @@ -121,7 +121,7 @@ To find out why we need two rows for each change, see [Algorithm](#table_engines **Notes on Usage** -1. The program that writes the data should remember the state of an object in order to cancel it. The “cancel” string should be a copy of the “state” string with the opposite `Sign`. This increases the initial size of storage but allows to write the data quickly. +1. The program that writes the data should remember the state of an object to be able to cancel it. “Cancel” string should contain copies of the primary key fields and the version of the “state” string and the opposite `Sign`. It increases the initial size of storage but allows to write the data quickly. 2. Long growing arrays in columns reduce the efficiency of the engine due to the load for writing. The more straightforward the data, the better the efficiency. 3. `SELECT` results depend strongly on the consistency of the history of object changes. Be accurate when preparing data for inserting. You can get unpredictable results with inconsistent data, such as negative values for non-negative metrics like session depth. diff --git a/docs/ru/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md b/docs/ru/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md index 5dc9589bef5..bf280eb52bc 100644 --- a/docs/ru/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md @@ -116,7 +116,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **Примечания по использованию** -1. Программа, которая записывает данные, должна помнить состояние объекта, чтобы иметь возможность отменить его. Строка отмены состояния должна быть копией предыдущей строки состояния с противоположным значением `Sign`. Это увеличивает начальный размер хранилища, но позволяет быстро записывать данные. +1. Программа, которая записывает данные, должна помнить состояние объекта, чтобы иметь возможность отменить его. Строка отмены состояния должна содержать копии полей первичного ключа и копию версии строки состояния и противоположное значение `Sign`. Это увеличивает начальный размер хранилища, но позволяет быстро записывать данные. 2. Длинные растущие массивы в столбцах снижают эффективность работы движка за счёт нагрузки на запись. Чем проще данные, тем выше эффективность. 3. `SELECT` результаты сильно зависят от согласованности истории изменений объекта. Будьте точны при подготовке данных для вставки. Вы можете получить непредсказуемые результаты с несогласованными данными, такими как отрицательные значения для неотрицательных метрик, таких как глубина сеанса. From bd9c01e4c0579af6b2d066cc2d5dfbe96efc24c1 Mon Sep 17 00:00:00 2001 From: Gao Qiang <30835199+dreamerfable@users.noreply.github.com> Date: Fri, 28 Aug 2020 22:54:30 +0800 Subject: [PATCH 082/535] Update mergetree.md --- .../table-engines/mergetree-family/mergetree.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index e733994b73d..0b886547229 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -214,7 +214,7 @@ ClickHouse 不要求主键惟一,所以你可以插入多条具有相同主键 在这种情况下合理的做法是,只保留少量的列在主键当中用于提升扫描效率,将维度列添加到排序键中。 -对排序键进行 [ALTER](../../../sql-reference/statements/alter/index.md) 是轻量级的操作,因为当一个新列同时被加入到表里和排序键里时,已存在的数据片段并不需要修改。由于旧的排序键是新排序键的前缀,并且新添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。 +对排序键进行 [ALTER](../../../sql-reference/statements/alter.md) 是轻量级的操作,因为当一个新列同时被加入到表里和排序键里时,已存在的数据片段并不需要修改。由于旧的排序键是新排序键的前缀,并且新添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。 ### 索引和分区在查询中的应用 {#use-of-indexes-and-partitions-in-queries} @@ -491,7 +491,7 @@ ClickHouse 在数据片段合并时会删除掉过期的数据。 MergeTree 系列表引擎可以将数据存储在多块设备上。这对某些可以潜在被划分为“冷”“热”的表来说是很有用的。近期数据被定期的查询但只需要很小的空间。相反,详尽的历史数据很少被用到。如果有多块磁盘可用,那么“热”的数据可以放置在快速的磁盘上(比如 NVMe 固态硬盘或内存),“冷”的数据可以放在相对较慢的磁盘上(比如机械硬盘)。 -数据片段是 `MergeTree` 引擎表的最小可移动单元。属于同一个数据片段的数据被存储在同一块磁盘上。数据片段会在后台自动的在磁盘间移动,也可以通过 [ALTER](../../../sql-reference/statements/alter/partition.md#alter_move-partition) 查询来移动。 +数据片段是 `MergeTree` 引擎表的最小可移动单元。属于同一个数据片段的数据被存储在同一块磁盘上。数据片段会在后台自动的在磁盘间移动,也可以通过 [ALTER](../../../sql-reference/statements/alter.md#alter_move-partition) 查询来移动。 ### 术语 {#terms} @@ -635,9 +635,9 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 对于 `MergeTree` 表,数据通过以下不同的方式写入到磁盘当中: - 作为插入(`INSERT`查询)的结果 -- 在后台合并和[数据变异](../../../sql-reference/statements/alter/index.md#alter-mutations)期间 +- 在后台合并和[数据变异](../../../sql-reference/statements/alter.md#alter-mutations)期间 - 当从另一个副本下载时 -- 作为 [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter/partition.md#alter_freeze-partition) 冻结分区的结果 +- 作为 [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter.md#alter_freeze-partition) 冻结分区的结果 除了数据变异和冻结分区以外的情况下,数据按照以下逻辑存储到卷或磁盘上: @@ -648,7 +648,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 在后台,数据片段基于剩余空间(`move_factor`参数)根据卷在配置文件中定义的顺序进行转移。数据永远不会从最后一个移出也不会从第一个移入。可以通过系统表 [system.part\_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (字段 `type = MOVE_PART`) 和 [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (字段 `path` 和 `disk`) 来监控后台的移动情况。同时,具体细节可以通过服务器日志查看。 -用户可以通过 [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter/partition.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。 +用户可以通过 [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。 数据移动不会妨碍到数据复制。也就是说,同一张表的不同副本可以指定不同的存储策略。 From e22ee38a353fd19785ff6106f33a4ce382c4b01c Mon Sep 17 00:00:00 2001 From: Dao Minh Thuc Date: Sun, 30 Aug 2020 22:48:43 +0700 Subject: [PATCH 083/535] Fix build for AppleClang --- contrib/capnproto-cmake/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt index 8bdac0beec0..e5d62c59327 100644 --- a/contrib/capnproto-cmake/CMakeLists.txt +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -29,6 +29,10 @@ set (KJ_SRCS ${CAPNPROTO_SOURCE_DIR}/kj/parse/char.c++ ) +if (CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-char8_t") +endif () + add_library(kj ${KJ_SRCS}) target_include_directories(kj SYSTEM PUBLIC ${CAPNPROTO_SOURCE_DIR}) From 647cf5718ed9b76c72413b699930cb448f1627c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Aug 2020 09:12:36 +0000 Subject: [PATCH 084/535] Better settings --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 6 +- .../RabbitMQ/RabbitMQBlockInputStream.h | 4 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 8 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 348 ++++++------------ src/Storages/RabbitMQ/StorageRabbitMQ.h | 31 +- .../integration/test_storage_rabbitmq/test.py | 45 +-- 6 files changed, 137 insertions(+), 305 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index fee65b65f08..4742ea2a33a 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -14,7 +14,7 @@ namespace DB RabbitMQBlockInputStream::RabbitMQBlockInputStream( StorageRabbitMQ & storage_, const StorageMetadataPtr & metadata_snapshot_, - const std::shared_ptr & context_, + Context & context_, const Names & columns, bool ack_in_suffix_) : storage(storage_) @@ -46,7 +46,7 @@ Block RabbitMQBlockInputStream::getHeader() const void RabbitMQBlockInputStream::readPrefixImpl() { - auto timeout = std::chrono::milliseconds(context->getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); + auto timeout = std::chrono::milliseconds(context.getSettingsRef().rabbitmq_max_wait_ms.totalMilliseconds()); buffer = storage.popReadBuffer(timeout); } @@ -83,7 +83,7 @@ Block RabbitMQBlockInputStream::readImpl() MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, *context, 1); + storage.getFormatName(), *buffer, non_virtual_header, context, 1); InputPort port(input_format->getPort().getHeader(), input_format.get()); connect(input_format->getPort(), port); diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h index 08cfe090c6e..4f52d64189e 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h @@ -16,7 +16,7 @@ public: RabbitMQBlockInputStream( StorageRabbitMQ & storage_, const StorageMetadataPtr & metadata_snapshot_, - const std::shared_ptr & context_, + Context & context_, const Names & columns, bool ack_in_suffix = true); @@ -36,7 +36,7 @@ public: private: StorageRabbitMQ & storage; StorageMetadataPtr metadata_snapshot; - const std::shared_ptr context; + Context context; Names column_names; bool ack_in_suffix; diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index bc71a929e8f..110093ef2f3 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -10,17 +10,19 @@ namespace DB #define LIST_OF_RABBITMQ_SETTINGS(M) \ M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \ - M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ M(String, rabbitmq_format, "", "The message format.", 0) \ + M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \ + M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ M(Char, rabbitmq_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ M(String, rabbitmq_schema, "", "Schema identifier (used by schema-based formats) for RabbitMQ engine", 0) \ - M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \ M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ M(String, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ M(String, rabbitmq_deadletter_exchange, "", "Exchange name to be passed as a dead-letter-exchange name.", 0) \ - M(Bool, rabbitmq_persistent_mode, false, "If set, delivery mode will be set to 2 (makes messages 'persistent', durable).", 0) \ + M(Bool, rabbitmq_persistent, false, "If set, delivery mode will be set to 2 (makes messages 'persistent', durable).", 0) \ + M(UInt64, rabbitmq_max_block_size, 0, "Number of row collected before flushing data from RabbitMQ.", 0) \ + M(Milliseconds, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \ DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index b78c21ae96d..40c972de508 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -47,6 +46,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int CANNOT_CONNECT_RABBITMQ; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace ExchangeType @@ -60,40 +60,35 @@ namespace ExchangeType static const String HEADERS = "headers"; } + StorageRabbitMQ::StorageRabbitMQ( const StorageID & table_id_, Context & context_, const ColumnsDescription & columns_, - const String & host_port_, - const Names & routing_keys_, - const String & exchange_name_, - const String & format_name_, - char row_delimiter_, - const String & schema_name_, - const String & exchange_type_, - size_t num_consumers_, - size_t num_queues_, - const String & queue_base_, - const String & deadletter_exchange_, - const bool persistent_) + std::unique_ptr rabbitmq_settings_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) - , routing_keys(global_context.getMacros()->expand(routing_keys_)) - , exchange_name(exchange_name_) - , format_name(global_context.getMacros()->expand(format_name_)) - , row_delimiter(row_delimiter_) - , schema_name(global_context.getMacros()->expand(schema_name_)) - , num_consumers(num_consumers_) - , num_queues(num_queues_) - , queue_base(queue_base_) - , deadletter_exchange(deadletter_exchange_) - , persistent(persistent_) + , rabbitmq_context(Context(global_context)) + , rabbitmq_settings(std::move(rabbitmq_settings_)) + , exchange_name(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_name.value)) + , format_name(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_format.value)) + , exchange_type(defineExchangeType(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_type.value))) + , routing_keys(parseRoutingKeys(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_routing_key_list.value))) + , row_delimiter(rabbitmq_settings->rabbitmq_row_delimiter.value) + , schema_name(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_schema.value)) + , num_consumers(rabbitmq_settings->rabbitmq_num_consumers.value) + , num_queues(rabbitmq_settings->rabbitmq_num_queues.value) + , queue_base(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_queue_base.value)) + , deadletter_exchange(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_deadletter_exchange.value)) + , persistent(rabbitmq_settings->rabbitmq_persistent.value) + , hash_exchange(num_consumers > 1 || num_queues > 1) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) - , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) + , parsed_address(parseAddress(global_context.getMacros()->expand(rabbitmq_settings->rabbitmq_host_port.value), 5672)) , login_password(std::make_pair( global_context.getConfigRef().getString("rabbitmq.username"), global_context.getConfigRef().getString("rabbitmq.password"))) - , semaphore(0, num_consumers_) + , semaphore(0, num_consumers) + , unique_strbase(getRandomName()) { loop = std::make_unique(); uv_loop_init(loop.get()); @@ -111,6 +106,10 @@ StorageRabbitMQ::StorageRabbitMQ( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + rabbitmq_context.makeQueryContext(); + if (!schema_name.empty()) + rabbitmq_context.setSetting("format_schema", schema_name); + /// One looping task for all consumers as they share the same connection == the same handler == the same event loop event_handler->updateLoopState(Loop::STOP); looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); @@ -122,38 +121,19 @@ StorageRabbitMQ::StorageRabbitMQ( heartbeat_task = global_context.getSchedulePool().createTask("RabbitMQHeartbeatTask", [this]{ heartbeatFunc(); }); heartbeat_task->deactivate(); - hash_exchange = num_consumers > 1 || num_queues > 1; - - if (exchange_type_ != ExchangeType::DEFAULT) - { - if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; - else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct; - else if (exchange_type_ == ExchangeType::TOPIC) exchange_type = AMQP::ExchangeType::topic; - else if (exchange_type_ == ExchangeType::HASH) exchange_type = AMQP::ExchangeType::consistent_hash; - else if (exchange_type_ == ExchangeType::HEADERS) exchange_type = AMQP::ExchangeType::headers; - else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); - } - else - { - exchange_type = AMQP::ExchangeType::fanout; - } - - auto table_id = getStorageID(); - String table_name = table_id.table_name; - if (queue_base.empty()) { /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to - * be table_name and not just a random string, because local exchanges should be declared the same for same tables + * be table-based and not just a random string, because local exchanges should be declared the same for same tables */ - sharding_exchange = exchange_name + "_" + table_name; + sharding_exchange = getTableBasedName(exchange_name, table_id_); /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base * for the names of later declared queues */ - queue_base = table_name; + queue_base = getTableBasedName("", table_id_); } else { @@ -165,11 +145,51 @@ StorageRabbitMQ::StorageRabbitMQ( } bridge_exchange = sharding_exchange + "_bridge"; +} - /* Generate a random string, which will be used for channelID's, which must be unique to tables and to channels within each table. - * (Cannot use table_name here because it must be a different string if table was restored) - */ - unique_strbase = getRandomName(); + +Names StorageRabbitMQ::parseRoutingKeys(String routing_key_list) +{ + Names result; + boost::split(result, routing_key_list, [](char c){ return c == ','; }); + for (String & key : result) + boost::trim(key); + + return result; +} + + +AMQP::ExchangeType StorageRabbitMQ::defineExchangeType(String exchange_type_) +{ + AMQP::ExchangeType type; + if (exchange_type_ != ExchangeType::DEFAULT) + { + if (exchange_type_ == ExchangeType::FANOUT) type = AMQP::ExchangeType::fanout; + else if (exchange_type_ == ExchangeType::DIRECT) type = AMQP::ExchangeType::direct; + else if (exchange_type_ == ExchangeType::TOPIC) type = AMQP::ExchangeType::topic; + else if (exchange_type_ == ExchangeType::HASH) type = AMQP::ExchangeType::consistent_hash; + else if (exchange_type_ == ExchangeType::HEADERS) type = AMQP::ExchangeType::headers; + else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); + } + else + { + type = AMQP::ExchangeType::fanout; + } + + return type; +} + + +String StorageRabbitMQ::getTableBasedName(String name, const StorageID & table_id) +{ + std::stringstream ss; + + if (name.empty()) + ss << table_id.database_name << "_" << table_id.table_name; + else + ss << name << "_" << table_id.database_name << "_" << table_id.table_name; + + return ss.str(); } @@ -393,9 +413,9 @@ Pipe StorageRabbitMQ::read( auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); - auto new_context = std::make_shared(context); + auto modified_context = context; if (!schema_name.empty()) - new_context->setSetting("format_schema", schema_name); + modified_context.setSetting("format_schema", schema_name); bool update_channels = false; if (!event_handler->connectionRunning()) @@ -415,7 +435,7 @@ Pipe StorageRabbitMQ::read( for (size_t i = 0; i < num_created_consumers; ++i) { - auto rabbit_stream = std::make_shared(*this, metadata_snapshot, new_context, column_names); + auto rabbit_stream = std::make_shared(*this, metadata_snapshot, modified_context, column_names); /* It is a possible but rare case when channel gets into error state and does not also close connection, so need manual update. * But I believe that in current context and with local rabbitmq settings this will never happen and any channel error will also @@ -632,13 +652,8 @@ bool StorageRabbitMQ::streamToViews() auto insert = std::make_shared(); insert->table_id = table_id; - auto rabbitmq_context = std::make_shared(global_context); - rabbitmq_context->makeQueryContext(); - if (!schema_name.empty()) - rabbitmq_context->setSetting("format_schema", schema_name); - // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, *rabbitmq_context, false, true, true); + InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true); auto block_io = interpreter.execute(); auto metadata_snapshot = getInMemoryMetadataPtr(); @@ -740,199 +755,52 @@ void registerStorageRabbitMQ(StorageFactory & factory) size_t args_count = engine_args.size(); bool has_settings = args.storage_def->settings; - RabbitMQSettings rabbitmq_settings; + auto rabbitmq_settings = std::make_unique(); if (has_settings) { - rabbitmq_settings.loadFromQuery(*args.storage_def); + rabbitmq_settings->loadFromQuery(*args.storage_def); } - String host_port = rabbitmq_settings.rabbitmq_host_port; - if (args_count >= 1) - { - const auto * ast = engine_args[0]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - host_port = safeGet(ast->value); + // Check arguments and settings + #define CHECK_RABBITMQ_STORAGE_ARGUMENT(ARG_NUM, ARG_NAME) \ + /* One of the three required arguments is not specified */ \ + if (args_count < (ARG_NUM) && (ARG_NUM) <= 3 && !rabbitmq_settings->ARG_NAME.changed) \ + { \ + throw Exception("Required parameter '" #ARG_NAME "' for storage RabbitMQ not specified", \ + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); \ + } \ + if (args_count >= (ARG_NUM)) \ + { \ + if (rabbitmq_settings->ARG_NAME.changed) /* The same argument is given in two places */ \ + { \ + throw Exception("The argument №" #ARG_NUM " of storage RabbitMQ " \ + "and the parameter '" #ARG_NAME "' is duplicated", ErrorCodes::BAD_ARGUMENTS); \ + } \ } - else - { - throw Exception(String("RabbitMQ host:port must be a string"), ErrorCodes::BAD_ARGUMENTS); - } - } - String routing_key_list = rabbitmq_settings.rabbitmq_routing_key_list.value; - if (args_count >= 2) - { - engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.local_context); - routing_key_list = engine_args[1]->as().value.safeGet(); - } + CHECK_RABBITMQ_STORAGE_ARGUMENT(1, rabbitmq_host_port) + CHECK_RABBITMQ_STORAGE_ARGUMENT(2, rabbitmq_exchange_name) + CHECK_RABBITMQ_STORAGE_ARGUMENT(3, rabbitmq_format) - Names routing_keys; - boost::split(routing_keys, routing_key_list, [](char c){ return c == ','; }); - for (String & key : routing_keys) - { - boost::trim(key); - } + CHECK_RABBITMQ_STORAGE_ARGUMENT(4, rabbitmq_exchange_type) + CHECK_RABBITMQ_STORAGE_ARGUMENT(5, rabbitmq_routing_key_list) + CHECK_RABBITMQ_STORAGE_ARGUMENT(6, rabbitmq_row_delimiter) + CHECK_RABBITMQ_STORAGE_ARGUMENT(7, rabbitmq_schema) + CHECK_RABBITMQ_STORAGE_ARGUMENT(8, rabbitmq_num_consumers) + CHECK_RABBITMQ_STORAGE_ARGUMENT(9, rabbitmq_num_queues) + CHECK_RABBITMQ_STORAGE_ARGUMENT(10, rabbitmq_queue_base) + CHECK_RABBITMQ_STORAGE_ARGUMENT(11, rabbitmq_deadletter_exchange) + CHECK_RABBITMQ_STORAGE_ARGUMENT(12, rabbitmq_persistent) - String exchange = rabbitmq_settings.rabbitmq_exchange_name.value; - if (args_count >= 3) - { - engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); + CHECK_RABBITMQ_STORAGE_ARGUMENT(13, rabbitmq_max_block_size) + CHECK_RABBITMQ_STORAGE_ARGUMENT(14, rabbitmq_flush_interval_ms) - const auto * ast = engine_args[2]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - exchange = safeGet(ast->value); - } - } + #undef CHECK_RABBITMQ_STORAGE_ARGUMENT - String format = rabbitmq_settings.rabbitmq_format.value; - if (args_count >= 4) - { - engine_args[3] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[3], args.local_context); - - const auto * ast = engine_args[3]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - format = safeGet(ast->value); - } - else - { - throw Exception("Format must be a string", ErrorCodes::BAD_ARGUMENTS); - } - } - - char row_delimiter = rabbitmq_settings.rabbitmq_row_delimiter; - if (args_count >= 5) - { - engine_args[4] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], args.local_context); - - const auto * ast = engine_args[4]->as(); - String arg; - if (ast && ast->value.getType() == Field::Types::String) - { - arg = safeGet(ast->value); - } - else - { - throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS); - } - if (arg.size() > 1) - { - throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS); - } - else if (arg.empty()) - { - row_delimiter = '\0'; - } - else - { - row_delimiter = arg[0]; - } - } - - String schema = rabbitmq_settings.rabbitmq_schema.value; - if (args_count >= 6) - { - engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[5], args.local_context); - - const auto * ast = engine_args[5]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - schema = safeGet(ast->value); - } - else - { - throw Exception("Format schema must be a string", ErrorCodes::BAD_ARGUMENTS); - } - } - - String exchange_type = rabbitmq_settings.rabbitmq_exchange_type.value; - if (args_count >= 7) - { - engine_args[6] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[6], args.local_context); - - const auto * ast = engine_args[6]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - exchange_type = safeGet(ast->value); - } - } - - UInt64 num_consumers = rabbitmq_settings.rabbitmq_num_consumers; - if (args_count >= 8) - { - const auto * ast = engine_args[7]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - num_consumers = safeGet(ast->value); - } - else - { - throw Exception("Number of consumers must be a positive integer", ErrorCodes::BAD_ARGUMENTS); - } - } - - UInt64 num_queues = rabbitmq_settings.rabbitmq_num_queues; - if (args_count >= 9) - { - const auto * ast = engine_args[8]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - num_consumers = safeGet(ast->value); - } - else - { - throw Exception("Number of queues must be a positive integer", ErrorCodes::BAD_ARGUMENTS); - } - } - - String queue_base = rabbitmq_settings.rabbitmq_queue_base.value; - if (args_count >= 10) - { - engine_args[9] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[9], args.local_context); - - const auto * ast = engine_args[9]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - queue_base = safeGet(ast->value); - } - } - - String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; - if (args_count >= 11) - { - engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); - - const auto * ast = engine_args[10]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - deadletter_exchange = safeGet(ast->value); - } - } - - bool persistent = static_cast(rabbitmq_settings.rabbitmq_persistent_mode); - if (args_count >= 12) - { - const auto * ast = engine_args[11]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - persistent = static_cast(safeGet(ast->value)); - } - else - { - throw Exception("Transactional channel parameter is a bool", ErrorCodes::BAD_ARGUMENTS); - } - } - - return StorageRabbitMQ::create( - args.table_id, args.context, args.columns, - host_port, routing_keys, exchange, format, row_delimiter, schema, exchange_type, num_consumers, - num_queues, queue_base, deadletter_exchange, persistent); + return StorageRabbitMQ::create(args.table_id, args.context, args.columns, std::move(rabbitmq_settings)); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); - } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 01592f11e20..1f483c9b17e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -64,42 +65,32 @@ public: bool restoreConnection(bool reconnecting); void updateChannel(ChannelPtr & channel); + protected: StorageRabbitMQ( const StorageID & table_id_, Context & context_, const ColumnsDescription & columns_, - const String & host_port_, - const Names & routing_keys_, - const String & exchange_name_, - const String & format_name_, - char row_delimiter_, - const String & schema_name_, - const String & exchange_type_, - size_t num_consumers_, - size_t num_queues_, - const String & queue_base_, - const String & deadletter_exchange, - const bool persistent_); + std::unique_ptr rabbitmq_settings_); private: Context global_context; + Context rabbitmq_context; + std::unique_ptr rabbitmq_settings; - Names routing_keys; const String exchange_name; - AMQP::ExchangeType exchange_type; - const String format_name; + AMQP::ExchangeType exchange_type; + Names routing_keys; char row_delimiter; const String schema_name; size_t num_consumers; - size_t num_created_consumers = 0; - bool hash_exchange; size_t num_queues; String queue_base; const String deadletter_exchange; const bool persistent; + bool hash_exchange; Poco::Logger * log; std::pair parsed_address; std::pair login_password; @@ -108,6 +99,7 @@ private: std::shared_ptr event_handler; std::shared_ptr connection; /// Connection for all consumers + size_t num_created_consumers = 0; Poco::Semaphore semaphore; std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers @@ -132,6 +124,11 @@ private: void heartbeatFunc(); void loopingFunc(); + Names parseRoutingKeys(String routing_key_list); + AMQP::ExchangeType defineExchangeType(String exchange_type_); + size_t getMaxBlockSize(); + String getTableBasedName(String name, const StorageID & table_id); + void initExchange(); void bindExchange(); diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 0a328301baa..68f7bb506e6 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -131,12 +131,12 @@ def rabbitmq_setup_teardown(): # Tests @pytest.mark.timeout(180) -def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): +def test_rabbitmq_select(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'new', + rabbitmq_exchange_name = 'select', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') @@ -146,48 +146,12 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): connection = pika.BlockingConnection(parameters) channel = connection.channel() - messages = [] - for i in range(25): - messages.append(json.dumps({'key': i, 'value': i})) - - for message in messages: - channel.basic_publish(exchange='new', routing_key='', body=message) - - messages = [] - for i in range(25, 50): - messages.append(json.dumps({'key': i, 'value': i})) - for message in messages: - channel.basic_publish(exchange='new', routing_key='', body=message) - - connection.close() - - result = '' - while True: - result += instance.query('SELECT * FROM test.rabbitmq', ignore_error=True) - if rabbitmq_check_result(result): - break - - rabbitmq_check_result(result, True) - - -@pytest.mark.timeout(180) -def test_rabbitmq_select_from_old_syntax_table(rabbitmq_cluster): - instance.query(''' - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ('rabbitmq1:5672', 'old', 'old', 'JSONEachRow', '\\n'); - ''') - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='old', routing_key='old', body=message) + channel.basic_publish(exchange='select', routing_key='', body=message) connection.close() @@ -206,6 +170,7 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'empty', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -1774,7 +1739,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'producer_reconnect', - rabbitmq_persistent_mode = '1', + rabbitmq_persistent = '1', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') From e57d1c827fffaf02af6cfbd4f11aec89f52c94a1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Aug 2020 10:00:28 +0000 Subject: [PATCH 085/535] Better shutdown --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 45 ++++++++++++++++------- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 +- 2 files changed, 33 insertions(+), 15 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 40c972de508..4b013d11574 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -210,6 +210,28 @@ void StorageRabbitMQ::loopingFunc() } +/* Need to deactivate this way because otherwise might get a deadlock when first deactivate streaming task in shutdown and then + * inside streaming task try to deactivate any other task + */ +void StorageRabbitMQ::deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop) +{ + if (task_mutex.try_lock()) + { + if (stop_loop) + event_handler->updateLoopState(Loop::STOP); + + task->deactivate(); + task_mutex.unlock(); + } + else if (wait) + { + /// Wait only if deactivating from shutdown + std::lock_guard lock(task_mutex); + task->deactivate(); + } +} + + void StorageRabbitMQ::initExchange() { /* Binding scheme is the following: client's exchange -> key bindings by routing key list -> bridge exchange (fanout) -> @@ -326,7 +348,7 @@ bool StorageRabbitMQ::restoreConnection(bool reconnecting) if (reconnecting) { - heartbeat_task->deactivate(); + deactivateTask(heartbeat_task, 0, 0); connection->close(); /// Connection might be unusable, but not closed /* Connection is not closed immediately (firstly, all pending operations are completed, and then @@ -346,7 +368,7 @@ bool StorageRabbitMQ::restoreConnection(bool reconnecting) AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); cnt_retries = 0; - while (!connection->ready() && ++cnt_retries != RETRIES_MAX) + while (!connection->ready() && !stream_cancelled && ++cnt_retries != RETRIES_MAX) { event_handler->iterateLoop(); std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); @@ -504,11 +526,9 @@ void StorageRabbitMQ::shutdown() stream_cancelled = true; wait_confirm.store(false); - streaming_task->deactivate(); - heartbeat_task->deactivate(); - - event_handler->updateLoopState(Loop::STOP); - looping_task->deactivate(); + deactivateTask(streaming_task, 1, 1); + deactivateTask(heartbeat_task, 1, 0); + deactivateTask(looping_task, 1, 1); connection->close(); @@ -695,14 +715,11 @@ bool StorageRabbitMQ::streamToViews() * races inside the library, but only in case any error occurs or connection is lost while ack is being sent */ if (event_handler->loopRunning()) - { - event_handler->updateLoopState(Loop::STOP); - looping_task->deactivate(); - } + deactivateTask(looping_task, 0, 1); if (!event_handler->connectionRunning()) { - if (restoreConnection(true)) + if (!stream_cancelled && restoreConnection(true)) { for (auto & stream : streams) stream->as()->updateChannel(); @@ -710,13 +727,13 @@ bool StorageRabbitMQ::streamToViews() } else { - /// Reschedule if unable to connect to rabbitmq + /// Reschedule if unable to connect to rabbitmq or quit if cancelled return false; } } else { - heartbeat_task->deactivate(); + deactivateTask(heartbeat_task, 0, 0); /// Commit for (auto & stream : streams) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 1f483c9b17e..522dfff9a23 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -101,7 +101,7 @@ private: size_t num_created_consumers = 0; Poco::Semaphore semaphore; - std::mutex mutex; + std::mutex mutex, task_mutex; std::vector buffers; /// available buffers for RabbitMQ consumers String unique_strbase; @@ -128,6 +128,7 @@ private: AMQP::ExchangeType defineExchangeType(String exchange_type_); size_t getMaxBlockSize(); String getTableBasedName(String name, const StorageID & table_id); + void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); void initExchange(); void bindExchange(); From 4834bed35b251fee8f53d72fa7c2650fd473a195 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 14:35:53 +0300 Subject: [PATCH 086/535] Add recompression TTL parser --- src/Parsers/ASTTTLElement.cpp | 5 +++++ src/Parsers/ASTTTLElement.h | 2 ++ src/Parsers/ExpressionElementParsers.cpp | 19 +++++++++++++++++++ src/Storages/TTLDescription.cpp | 15 ++++++++++++++- src/Storages/TTLDescription.h | 5 +++++ src/Storages/TTLMode.h | 3 ++- 6 files changed, 47 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 1635d376d30..f37631769b8 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -57,6 +57,11 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st } } } + else if (mode == TTLMode::RECOMPRESS) + { + settings.ostr << " RECOMPRESS "; + recompression_codec->formatImpl(settings, state, frame); + } else if (mode == TTLMode::DELETE) { /// It would be better to output "DELETE" here but that will break compatibility with earlier versions. diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 7ee1f4795ff..aadd019b59c 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -20,6 +20,8 @@ public: ASTs group_by_key; std::vector> group_by_aggregations; + ASTPtr recompression_codec; + ASTTTLElement(TTLMode mode_, DataDestinationType destination_type_, const String & destination_name_) : mode(mode_) , destination_type(destination_type_) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e24bb9c4129..67c3737f6f0 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1506,6 +1506,8 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_where("WHERE"); ParserKeyword s_group_by("GROUP BY"); ParserKeyword s_set("SET"); + ParserKeyword s_recompress("RECOMPRESS"); + ParserKeyword s_codec("CODEC"); ParserToken s_comma(TokenType::Comma); ParserToken s_eq(TokenType::Equals); @@ -1513,6 +1515,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserStringLiteral parser_string_literal; ParserExpression parser_exp; ParserExpressionList parser_expression_list(false); + ParserCodec parser_codec; ASTPtr ttl_expr; if (!parser_exp.parse(pos, ttl_expr, expected)) @@ -1536,6 +1539,10 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { mode = TTLMode::GROUP_BY; } + else if (s_recompress.ignore(pos)) + { + mode = TTLMode::RECOMPRESS; + } else { s_delete.ignore(pos); @@ -1544,6 +1551,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr where_expr; ASTPtr ast_group_by_key; + ASTPtr recompression_codec; std::vector> group_by_aggregations; if (mode == TTLMode::MOVE) @@ -1587,6 +1595,14 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parser_exp.parse(pos, where_expr, expected)) return false; } + else if (mode == TTLMode::RECOMPRESS) + { + if (!s_codec.ignore(pos)) + return false; + + if (!parser_codec.parse(pos, recompression_codec, expected)) + return false; + } auto ttl_element = std::make_shared(mode, destination_type, destination_name); ttl_element->setTTL(std::move(ttl_expr)); @@ -1599,6 +1615,9 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ttl_element->group_by_aggregations = std::move(group_by_aggregations); } + if (mode == TTLMode::RECOMPRESS) + ttl_element->recompression_codec = recompression_codec; + node = ttl_element; return true; } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 4c9da095278..656baf39971 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -263,6 +264,12 @@ TTLDescription TTLDescription::getTTLFromAST( result.aggregate_descriptions.push_back(descr); } } + else if (ttl_element->mode == TTLMode::RECOMPRESS) + { + result.recompression_codec = + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( + ttl_element->recompression_codec, {}, !context.getSettingsRef().allow_suspicious_codecs); + } } checkTTLExpression(result.expression, result.result_column); @@ -311,15 +318,21 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( for (const auto & ttl_element_ptr : definition_ast->children) { auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); - if (ttl.destination_type == DataDestinationType::DELETE) + if (ttl.mode == TTLMode::DELETE) { if (seen_delete_ttl) throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); result.rows_ttl = ttl; seen_delete_ttl = true; } + else if (ttl.mode == TTLMode::RECOMPRESS) + { + result.recompression_ttl.emplace_back(std::move(ttl)); + } else + { result.move_ttl.emplace_back(std::move(ttl)); + } } return result; } diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index f7769fd42e9..4b0d4370a70 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -75,6 +75,9 @@ struct TTLDescription /// Name of destination disk or volume String destination_name; + /// Codec name which will be used to recompress data + ASTPtr recompression_codec; + /// Parse TTL structure from definition. Able to parse both column and table /// TTLs. static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context, const KeyDescription & primary_key); @@ -102,6 +105,8 @@ struct TTLTableDescription /// Moving data TTL (to other disks or volumes) TTLDescriptions move_ttl; + TTLDescriptions recompression_ttl; + TTLTableDescription() = default; TTLTableDescription(const TTLTableDescription & other); TTLTableDescription & operator=(const TTLTableDescription & other); diff --git a/src/Storages/TTLMode.h b/src/Storages/TTLMode.h index 0681f10fc17..7f5fe0315c6 100644 --- a/src/Storages/TTLMode.h +++ b/src/Storages/TTLMode.h @@ -8,7 +8,8 @@ enum class TTLMode { DELETE, MOVE, - GROUP_BY + GROUP_BY, + RECOMPRESS, }; } From 42c210fcba41d2e0ba657b38048278667ebf5963 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 15:12:51 +0300 Subject: [PATCH 087/535] Recompress TTLs in memory metadata --- src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h | 5 ++++- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 5 +++++ src/Storages/StorageInMemoryMetadata.cpp | 10 ++++++++++ src/Storages/StorageInMemoryMetadata.h | 4 ++++ 4 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 209d7181b66..d2e131d5650 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -38,13 +38,16 @@ struct MergeTreeDataPartTTLInfos MergeTreeDataPartTTLInfo table_ttl; /// `part_min_ttl` and `part_max_ttl` are TTLs which are used for selecting parts - /// to merge in order to remove expired rows. + /// to merge in order to remove expired rows. time_t part_min_ttl = 0; time_t part_max_ttl = 0; /// Order is important as it would be serialized and hashed for checksums std::map moves_ttl; + /// Order is important as it would be serialized and hashed for checksums + std::map recompression_ttl; + void read(ReadBuffer & in); void write(WriteBuffer & out) const; void update(const MergeTreeDataPartTTLInfos & other_infos); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 445a02b06f0..23569a13b85 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -234,6 +234,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); + const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + for (const auto & ttl_entry : recompression_ttl_entries) + updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); + + NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); VolumePtr volume = data.getStoragePolicy()->getVolume(0); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index b7f4565a55a..f611c1ec95d 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -157,6 +157,16 @@ bool StorageInMemoryMetadata::hasAnyMoveTTL() const return !table_ttl.move_ttl.empty(); } +TTLDescriptions StorageInMemoryMetadata::getRecompressionTTLs() const +{ + return table_ttl.recompression_ttl; +} + +bool StorageInMemoryMetadata::hasAnyRecompressionTTL() const +{ + return !table_ttl.recompression_ttl.empty(); +} + ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet & updated_columns) const { if (updated_columns.empty()) diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 4c78d72a9d1..3656edf71f4 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -114,6 +114,10 @@ struct StorageInMemoryMetadata TTLDescriptions getMoveTTLs() const; bool hasAnyMoveTTL() const; + // Just wrapper for table TTLs, return info about recompression ttl + TTLDescriptions getRecompressionTTLs() const; + bool hasAnyRecompressionTTL() const; + /// 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; From adc18f4d3f8915a1ad505ebc67cace8d98d81c04 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 16:29:31 +0300 Subject: [PATCH 088/535] Write with recompression TTL --- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 34 +++++++++++++++++++ .../MergeTree/MergeTreeDataWriter.cpp | 9 +++-- 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 37d036fc6fc..94a2b4269ef 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -67,6 +67,18 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) moves_ttl.emplace(expression, ttl_info); } } + if (json.has("recompression")) + { + const JSON & moves = json["recompression"]; + for (auto move : moves) // NOLINT + { + MergeTreeDataPartTTLInfo ttl_info; + ttl_info.min = move["min"].getUInt(); + ttl_info.max = move["max"].getUInt(); + String expression = move["expression"].getString(); + recompression_ttl.emplace(expression, ttl_info); + } + } } @@ -122,6 +134,28 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const } writeString("]", out); } + if (!recompression_ttl.empty()) + { + if (!moves_ttl.empty() || !columns_ttl.empty() || table_ttl.min) + writeString(",", out); + + writeString(R"("recompression":[)", out); + for (auto it = recompression_ttl.begin(); it != recompression_ttl.end(); ++it) + { + if (it != recompression_ttl.begin()) + writeString(",", out); + + writeString(R"({"expression":)", out); + writeString(doubleQuoteString(it->first), out); + writeString(R"(,"min":)", out); + writeIntText(it->second.min, out); + writeString(R"(,"max":)", out); + writeIntText(it->second.max, out); + writeString("}", out); + } + writeString("]", out); + + } writeString("}", out); } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 23569a13b85..92bf5345d5a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -234,11 +234,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); - const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); - for (const auto & ttl_entry : recompression_ttl_entries) - updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); - - NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); VolumePtr volume = data.getStoragePolicy()->getVolume(0); @@ -303,6 +298,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); + const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + for (const auto & ttl_entry : recompression_ttl_entries) + updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); + new_data_part->ttl_infos.update(move_ttl_infos); /// This effectively chooses minimal compression method: From b20a0bc254e769e66093e7c2a2a574b252b5a698 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 16:42:42 +0300 Subject: [PATCH 089/535] Add recompression flag in ReplicatedEntry --- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 10 ++++++++++ src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + 2 files changed, 11 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index af6d980ad98..a4fc600d1b3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -36,6 +36,9 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << s << '\n'; out << "into\n" << new_part_name; out << "\ndeduplicate: " << deduplicate; + /// For backward compatibility write only if enabled + if (recompress) + out << "\nrecompress: " << recompress; break; case DROP_RANGE: @@ -149,7 +152,14 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) } in >> new_part_name; if (format_version >= 4) + { in >> "\ndeduplicate: " >> deduplicate; + in >> "\n"; + if (in.eof()) + trailing_newline_found = true; + else if (checkString("recompress\n", in)) + in >> recompress; + } } else if (type_str == "drop" || type_str == "detach") { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index ae5fad0b83c..62599c2c3a7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -79,6 +79,7 @@ struct ReplicatedMergeTreeLogEntryData Strings source_parts; bool deduplicate = false; /// Do deduplicate on merge + bool recompress = false; /// Recompress parts on merge String column_name; String index_name; From 46f833b7df64f77d361f78d629d3075f83945ebb Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 22:50:42 +0300 Subject: [PATCH 090/535] Some changes --- src/Storages/MergeTree/MergeTreeData.cpp | 30 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 3 ++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 1 + .../MergeTree/MergeTreeDataMergerMutator.h | 1 + .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 7 +++++ 5 files changed, 42 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b6a495161f5..b721cf4afbf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3055,6 +3055,36 @@ MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & tt return max_max_ttl ? *best_entry_it : std::optional(); } + +CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const +{ + + time_t max_max_ttl = 0; + TTLDescriptions::const_iterator best_entry_it; + auto metadata_snapshot = getInMemoryMetadataPtr(); + + const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + for (auto ttl_entry_it = recompression_ttl_entries.begin(); ttl_entry_it != recompression_ttl_entries.end(); ++ttl_entry_it) + { + auto ttl_info_it = ttl_infos.recompression_ttl.find(ttl_entry_it->result_column); + /// Prefer TTL rule which went into action last. + if (ttl_info_it != ttl_infos.recompression_ttl.end() + && ttl_info_it->second.max <= current_time + && max_max_ttl <= ttl_info_it->second.max) + { + best_entry_it = ttl_entry_it; + max_max_ttl = ttl_info_it->second.max; + } + } + + if (max_max_ttl) + return CompressionCodecFactory::instance().get(best_entry_it->recompression_codec, {}); + + return global_context.chooseCompressionCodec( + part_size_compressed, + static_cast(part_size_compressed) / getTotalActiveSizeInBytes()); +} + MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const { DataParts res; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e088a1c098b..ab115927e1e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -669,6 +669,9 @@ public: std::optional selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; + + CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const; + /// Limiting parallel sends per one table, used in DataPartsExchange std::atomic_uint current_table_sends {0}; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 673ad02bfb6..8cece66dafb 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -284,6 +284,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( current_time, data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); + parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge); } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index d5798fe3582..e13711f8064 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -249,6 +249,7 @@ private: /// Stores the next TTL merge due time for each partition (used only by TTLMergeSelector) TTLMergeSelector::PartitionIdToTTLs next_ttl_merge_times_by_partition; + /// Performing TTL merges independently for each partition guarantees that /// there is only a limited number of TTL merges and no partition stores data, that is too stale }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 94a2b4269ef..4b0a8bdfa9e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -16,6 +16,12 @@ void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_i updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } + for (const auto & [name, ttl_info] : other_infos.recompression_ttl) + { + recompression_ttl[name].update(ttl_info); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); + } + for (const auto & [expression, ttl_info] : other_infos.moves_ttl) { moves_ttl[expression].update(ttl_info); @@ -77,6 +83,7 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) ttl_info.max = move["max"].getUInt(); String expression = move["expression"].getString(); recompression_ttl.emplace(expression, ttl_info); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } } } From 8fa61f785faa3b21f913b6780fbb5bb667eec1ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Sep 2020 01:52:12 +0300 Subject: [PATCH 091/535] Better check for tuple size in complex key external dictionaries --- src/Dictionaries/ExternalQueryBuilder.cpp | 8 ++++++-- .../SSDComplexKeyCacheDictionary.cpp | 6 ++++++ src/Dictionaries/SSDComplexKeyCacheDictionary.h | 10 ++++------ src/Functions/FunctionsExternalDictionaries.h | 16 ++++++++++++++++ 4 files changed, 32 insertions(+), 8 deletions(-) diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index b682aaeb557..e8d71b1fd85 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; } @@ -239,12 +240,15 @@ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector } -std::string -ExternalQueryBuilder::composeLoadKeysQuery(const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix) +std::string ExternalQueryBuilder::composeLoadKeysQuery( + const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix) { if (!dict_struct.key) throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD}; + if (key_columns.size() != dict_struct.key->size()) + throw Exception{"The size of key_columns does not equal to the size of dictionary key", ErrorCodes::LOGICAL_ERROR}; + WriteBufferFromOwnString out; writeString("SELECT ", out); diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp index 826a61f7312..b1e4686e938 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp @@ -1120,6 +1120,8 @@ void SSDComplexKeyCacheStorage::update( AbsentIdHandler && on_key_not_found, const DictionaryLifetime lifetime) { + assert(key_columns.size() == key_types.size()); + auto append_block = [&key_types, this]( const Columns & new_keys, const SSDComplexKeyCachePartition::Attributes & new_attributes, @@ -1447,6 +1449,10 @@ void SSDComplexKeyCacheDictionary::getItemsNumberImpl( const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out, DefaultGetter && get_default) const { + assert(dict_struct.key); + assert(key_columns.size() == key_types.size()); + assert(key_columns.size() == dict_struct.key->size()); + const auto now = std::chrono::system_clock::now(); TemporalComplexKeysPool not_found_pool; diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.h b/src/Dictionaries/SSDComplexKeyCacheDictionary.h index 89e88982eee..af9a0c0a7ee 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.h +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.h @@ -427,9 +427,8 @@ private: using SSDComplexKeyCachePartitionPtr = std::shared_ptr; -/* - Class for managing SSDCachePartition and getting data from source. -*/ +/** Class for managing SSDCachePartition and getting data from source. + */ class SSDComplexKeyCacheStorage { public: @@ -515,9 +514,8 @@ private: }; -/* - Dictionary interface -*/ +/** Dictionary interface + */ class SSDComplexKeyCacheDictionary final : public IDictionaryBase { public: diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 609c247ce42..5472f0eebf8 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -971,6 +971,14 @@ private: const auto & key_columns = assert_cast(*key_col).getColumnsCopy(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); + assert(key_columns.size() == key_types.size()); + const auto & structure = dict->getStructure(); + assert(structure.key); + size_t key_size = structure.key->size(); + if (key_columns.size() != key_size) + throw Exception{ErrorCodes::TYPE_MISMATCH, + "Wrong size of tuple at the third argument of function {} must be {}", getName(), key_size}; + typename ColVec::MutablePtr out; if constexpr (IsDataTypeDecimal) out = ColVec::create(key_columns.front()->size(), decimal_scale); @@ -1294,6 +1302,14 @@ private: const auto & key_columns = typeid_cast(*key_col).getColumnsCopy(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); + assert(key_columns.size() == key_types.size()); + const auto & structure = dict->getStructure(); + assert(structure.key); + size_t key_size = structure.key->size(); + if (key_columns.size() != key_size) + throw Exception{ErrorCodes::TYPE_MISMATCH, + "Wrong size of tuple at the third argument of function {} must be {}", getName(), key_size}; + /// @todo detect when all key columns are constant const auto rows = key_col->size(); typename ColVec::MutablePtr out; From 142a5bcede36257a37905bbf50047eca09b20f88 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Sep 2020 02:10:04 +0300 Subject: [PATCH 092/535] Added validation of key types to SSD Cache dictionary --- src/Dictionaries/SSDComplexKeyCacheDictionary.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp index b1e4686e938..972d10da24d 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp @@ -1453,6 +1453,8 @@ void SSDComplexKeyCacheDictionary::getItemsNumberImpl( assert(key_columns.size() == key_types.size()); assert(key_columns.size() == dict_struct.key->size()); + dict_struct.validateKeyTypes(key_types); + const auto now = std::chrono::system_clock::now(); TemporalComplexKeysPool not_found_pool; @@ -1533,6 +1535,8 @@ void SSDComplexKeyCacheDictionary::getItemsStringImpl( ColumnString * out, DefaultGetter && get_default) const { + dict_struct.validateKeyTypes(key_types); + const auto now = std::chrono::system_clock::now(); TemporalComplexKeysPool not_found_pool; From 25140b9bd5b6421b84ef8586827cc49b9d015e7b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 04:39:36 +0300 Subject: [PATCH 093/535] fsync MergeTree format file --- src/Storages/MergeTree/MergeTreeData.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bbefba70c58..bc668659b6a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -220,6 +220,8 @@ MergeTreeData::MergeTreeData( format_version = min_format_version; auto buf = version_file.second->writeFile(version_file.first); writeIntText(format_version.toUnderType(), *buf); + if (global_context.getSettingsRef().fsync_metadata) + buf->sync(); } else { From 927eb32e882d070ff5ff5446d5b9e0071e2c6f9d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 04:46:40 +0300 Subject: [PATCH 094/535] add test for durability (draft) --- utils/durability-test/create.sql | 1 + utils/durability-test/durability-test.sh | 154 +++++++++++++++++++++++ utils/durability-test/insert.sql | 1 + utils/durability-test/install.sh | 3 + utils/durability-test/sshd_config | 8 ++ utils/durability-test/startup.exp | 23 ++++ 6 files changed, 190 insertions(+) create mode 100644 utils/durability-test/create.sql create mode 100644 utils/durability-test/durability-test.sh create mode 100644 utils/durability-test/insert.sql create mode 100644 utils/durability-test/install.sh create mode 100644 utils/durability-test/sshd_config create mode 100755 utils/durability-test/startup.exp diff --git a/utils/durability-test/create.sql b/utils/durability-test/create.sql new file mode 100644 index 00000000000..1ec394100e2 --- /dev/null +++ b/utils/durability-test/create.sql @@ -0,0 +1 @@ +CREATE TABLE test (a Int, s String) ENGINE = MergeTree ORDER BY a; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh new file mode 100644 index 00000000000..1f47c900f49 --- /dev/null +++ b/utils/durability-test/durability-test.sh @@ -0,0 +1,154 @@ +#!/bin/bash + +URL=http://cloud-images.ubuntu.com/bionic/current +IMAGE=bionic-server-cloudimg-amd64.img +SSH_PORT=11022 +CLICKHOUSE_PORT=9090 +PASSWORD=root + +TABLE_NAME=$1 +CREATE_QUERY=$2 +INSERT_QUERY=$3 + +if [[ -z $TABLE_NAME || -z $CREATE_QUERY || -z $INSERT_QUERY ]]; then + echo "Required 3 arguments: table name, file with create query, file with insert query" + exit 1 +fi + +function run() +{ + sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" +} + +function copy() +{ + sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 +} + +function wait_vm_for_start() +{ + echo "Waiting until VM started..." + started=0 + for i in {0..100}; do + run "exit" + if [ $? -eq 0 ]; then + started=1 + break + fi + sleep 1s + done + + if ((started == 0)); then + echo "Can't start or connect to VM." + exit 1 + fi + + echo "Started VM" +} + +function wait_clickhouse_for_start() +{ + echo "Waiting until ClickHouse started..." + started=0 + for i in {0..15}; do + run "clickhouse client --query 'select 1'" + if [ $? -eq 0 ]; then + started=1 + break + fi + sleep 1s + done + + if ((started == 0)); then + echo "Can't start ClickHouse." + fi + + echo "Started ClickHouse" +} + +echo "Downloading image" +curl -O $URL/$IMAGE + +qemu-img resize $IMAGE +10G +virt-customize -a $IMAGE --root-password password:$PASSWORD +virt-copy-in -a $IMAGE sshd_config /etc/ssh + +echo "Starting VM" + +chmod +x ./startup.exp +./startup.exp > qemu.log 2>&1 & + +wait_vm_for_start + +echo "Preparing VM" + +# Resize partition +run "growpart /dev/sda 1 && resize2fs /dev/sda1" + +if [[ -z $CLICKHOUSE_BINARY ]]; then + CLICKHOUSE_BINARY=/usr/bin/clickhouse +fi + +if [[ -z $CLICKHOUSE_CONFIG_DIR ]]; then + CLICKHOUSE_CONFIG_DIR=/etc/clickhouse-server +fi + +echo "Using ClickHouse binary: " $CLICKHOUSE_BINARY +echo "Using ClickHouse config from: " $CLICKHOUSE_CONFIG_DIR + +copy $CLICKHOUSE_BINARY /usr/bin +copy $CLICKHOUSE_CONFIG_DIR /etc +run "mv /etc/$CLICKHOUSE_CONFIG_DIR /etc/clickhouse-server" + +echo "Prepared VM" +echo "Starting ClickHouse" + +run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & + +wait_clickhouse_for_start + +echo "Started ClickHouse" + +query=`cat $CREATE_QUERY` +echo "Executing query:" $query +run "clickhouse client --query '$query'" + +query=`cat $INSERT_QUERY` +echo "Will run in a loop query: " $query +run "clickhouse benchmark <<< '$query'" & +echo "Running queries" + +pid=`pidof qemu-system-x86_64` +sec=$(( (RANDOM % 3) + 25 )) + +ms=$(( RANDOM % 1000 )) + +echo "Will kill VM in $sec.$ms sec" + +sleep $sec.$ms +kill -9 $pid + +echo "Restarting" + +./startup.exp > qemu.log 2>&1 & +wait_vm_for_start + +run "rm -r *data/system" +run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & +wait_clickhouse_for_start + +result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'"` +if [[ -n $result ]]; then + echo "FAIL. Can't attach table:" + echo $result + exit 1 +fi + +result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remove broken part'"` +if [[ -n $result ]]; then + echo "FAIL. Have broken parts:" + echo $result + exit 1 +fi + +echo OK diff --git a/utils/durability-test/insert.sql b/utils/durability-test/insert.sql new file mode 100644 index 00000000000..8982ad47228 --- /dev/null +++ b/utils/durability-test/insert.sql @@ -0,0 +1 @@ +INSERT INTO test SELECT number, toString(number) FROM numbers(10) diff --git a/utils/durability-test/install.sh b/utils/durability-test/install.sh new file mode 100644 index 00000000000..526cde6743f --- /dev/null +++ b/utils/durability-test/install.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +apt update && apt install qemu-kvm qemu virt-manager virt-viewer libguestfs-tools sshpass expect diff --git a/utils/durability-test/sshd_config b/utils/durability-test/sshd_config new file mode 100644 index 00000000000..6ed06d3d8ad --- /dev/null +++ b/utils/durability-test/sshd_config @@ -0,0 +1,8 @@ +PermitRootLogin yes +PasswordAuthentication yes +ChallengeResponseAuthentication no +UsePAM yes +X11Forwarding yes +PrintMotd no +AcceptEnv LANG LC_* +Subsystem sftp /usr/lib/openssh/sftp-server diff --git a/utils/durability-test/startup.exp b/utils/durability-test/startup.exp new file mode 100755 index 00000000000..540cfc0e4b8 --- /dev/null +++ b/utils/durability-test/startup.exp @@ -0,0 +1,23 @@ +#!/usr/bin/expect -f + +# Wait enough (forever) until a long-time boot +set timeout -1 + +spawn qemu-system-x86_64 \ + -hda bionic-server-cloudimg-amd64.img \ + -cpu qemu64,+ssse3,+sse4.1,+sse4.2,+popcnt -smp 8 \ + -net nic -net user,hostfwd=tcp::11022-:22 \ + -m 4096 -nographic + +expect "login: " +send "root\n" + +expect "Password: " +send "root\n" + +# Without it ssh is not working on guest machine for some reason +expect "# " +send "dhclient && ssh-keygen -A && systemctl restart sshd.service\n" + +# Wait forever +expect "########" From c3dd968931e31db7bc59483b85e67acc961dbafd Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 31 Aug 2020 21:42:27 -0700 Subject: [PATCH 095/535] fix ALTER LIVE VIEW lock issue This PR fixes a lock issue that happens while executing `ALTER LIVE VIEW` query with the `REFRESH` command that results in a exception. The problem is that lock is currently being acquired in `InterpreterALterQuery.cpp` in the `InterpreterAlterQuery::execute()` method and lock is again being reacquired in `StorageLiveView.cpp` in the ` StorageLiveView::refresh` method. This removes that extra lock. Before fix: ```sql --create table CREATE TABLE test0 ( c0 UInt64 ) ENGINE = MergeTree() PARTITION BY c0 ORDER BY c0; -- enable experimental_live_view :) SET allow_experimental_live_view=1 -- create live view; :) CREATE LIVE VIEW live1 AS SELECT * FROM table0; -- alter live view results in exception :) ALTER LIVE VIEW live1 REFRESH; ... ... Received exception from server (version 20.8.1): Code: 49. DB::Exception: Received from localhost:9000. DB::Exception: RWLockImpl::getLock(): RWLock is already locked in exclusive mode. ``` After fix: ```sql :) ALTER LIVE VIEW live1 REFRESH; ALTER LIVE VIEW live1 REFRESH Ok. 0 rows in set. Elapsed: 0.016 sec. ``` --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 5 +++-- src/Storages/LiveView/StorageLiveView.h | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index e0313803e9a..8cf581eb463 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -101,7 +101,7 @@ BlockIO InterpreterAlterQuery::execute() switch (command.type) { case LiveViewCommand::REFRESH: - live_view->refresh(context); + live_view->refresh(); break; } } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 54ac5bcc791..4da02365232 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -518,9 +518,10 @@ void StorageLiveView::drop() condition.notify_all(); } -void StorageLiveView::refresh(const Context & context) +void StorageLiveView::refresh() { - auto table_lock = lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + // Lock is already acquired exclusively from InterperterAlterQuery.cpp InterpreterAlterQuery::execute() method. + // So, reacquiring lock is not needed and will result in an exception. { std::lock_guard lock(mutex); if (getNewBlocks()) diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 43afd169a92..0c099d01a29 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -122,7 +122,7 @@ public: void startup() override; void shutdown() override; - void refresh(const Context & context); + void refresh(); Pipe read( const Names & column_names, From 2c0353587eb754fc5dc7c3efb7c223b05ec34a95 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 31 Aug 2020 22:04:40 -0700 Subject: [PATCH 096/535] add tests --- .../0_stateless/01463_test_alter_live_view_refresh.sql | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql diff --git a/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql new file mode 100644 index 00000000000..36e8c9a9785 --- /dev/null +++ b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql @@ -0,0 +1,9 @@ +CREATE TABLE test0 ( + c0 UInt64 + ) ENGINE = MergeTree() PARTITION BY c0 ORDER BY c0; + +SET allow_experimental_live_view=1; + +CREATE LIVE VIEW live1 AS SELECT * FROM test0; + +ALTER LIVE VIEW live1 REFRESH; -- success From 6cb893f0736ba04196eb43c0da566baa90d6c36e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Sep 2020 09:38:23 +0300 Subject: [PATCH 097/535] Draft. --- docker/test/integration/base/Dockerfile | 1 + docker/test/stateless_unbundled/Dockerfile | 68 ++++++++++++++++--- tests/integration/helpers/cluster.py | 14 ++-- tests/integration/runner | 2 + .../configs/dns_update_short.xml | 2 +- tests/integration/test_host_ip_change/test.py | 7 +- 6 files changed, 80 insertions(+), 14 deletions(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 53627c78208..2b8877437f8 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -17,6 +17,7 @@ RUN apt-get update \ odbc-postgresql \ sqlite3 \ curl \ + bind9-host \ tar RUN rm -rf \ /var/lib/apt/lists/* \ diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index 7de29fede72..4978252d556 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -1,12 +1,56 @@ # docker build -t yandex/clickhouse-stateless-unbundled-test . -FROM yandex/clickhouse-test-base +FROM ubuntu:20.04 ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz" -RUN apt-get --allow-unauthenticated update -y \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get --allow-unauthenticated install --yes --no-install-recommends \ - alien \ +ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 + +RUN apt-get update \ + && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ + --yes --no-install-recommends --verbose-versions \ + && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ + && wget -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ + && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ + && apt-key add /tmp/llvm-snapshot.gpg.key \ + && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ + && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ + /etc/apt/sources.list + +# initial packages +RUN apt-get update \ + && apt-get install \ + bash \ + fakeroot \ + ccache \ + curl \ + software-properties-common \ + --yes --no-install-recommends + +# Special dpkg-deb (https://github.com/ClickHouse-Extras/dpkg) version which is able +# to compress files using pigz (https://zlib.net/pigz/) instead of gzip. +# Significantly increase deb packaging speed and compatible with old systems +RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb \ + && chmod +x dpkg-deb \ + && cp dpkg-deb /usr/bin + + +RUN apt-get update \ + && apt-get install \ + clang-${LLVM_VERSION} \ + debhelper \ + devscripts \ + gdb \ + git \ + gperf \ + lcov \ + llvm-${LLVM_VERSION} \ + moreutils \ + perl \ + perl \ + pigz \ + pkg-config \ + tzdata \ + alien \ brotli \ cmake \ devscripts \ @@ -56,8 +100,6 @@ RUN apt-get --allow-unauthenticated update -y \ pkg-config \ python \ python-lxml \ - python-requests \ - python-termcolor \ qemu-user-static \ sudo \ telnet \ @@ -68,7 +110,10 @@ RUN apt-get --allow-unauthenticated update -y \ wget \ zlib1g-dev \ zookeeper \ - zookeeperd + zookeeperd \ + --yes --no-install-recommends + + RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ @@ -77,6 +122,13 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ && rm -rf /tmp/clickhouse-odbc-tmp +# Sanitizer options +RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ + echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ + echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ + ln -s /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; + + ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f421f979947..88a2611774a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -111,6 +111,7 @@ class ClickHouseCluster: custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST') self.docker_api_version = os.environ.get("DOCKER_API_VERSION") + self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG") self.base_cmd = ['docker-compose'] if custom_dockerd_host: @@ -165,7 +166,7 @@ class ClickHouseCluster: with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, - hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", + hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag=None, stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None, zookeeper_docker_compose_path=None, zookeeper_use_tmpfs=True, minio_certs_dir=None): """Add an instance to the cluster. @@ -183,13 +184,16 @@ class ClickHouseCluster: if name in self.instances: raise Exception("Can\'t add instance `%s': there is already an instance with the same name!" % name) + if tag is None: + tag = self.docker_base_tag + instance = ClickHouseInstance( self, self.base_dir, name, config_dir, main_configs or [], user_configs or [], macros or {}, with_zookeeper, self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, self.base_configs_dir, self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, - env_variables=env_variables or {}, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, + env_variables=env_variables or {}, image=image, tag=tag, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, tmpfs=tmpfs or []) @@ -704,7 +708,7 @@ DOCKER_COMPOSE_TEMPLATE = ''' version: '2.3' services: {name}: - image: {image} + image: {image}:{tag} hostname: {hostname} volumes: - {configs_dir}:/etc/clickhouse-server/ @@ -739,7 +743,7 @@ class ClickHouseInstance: with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, base_configs_dir, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, - image="yandex/clickhouse-integration-test", + image="yandex/clickhouse-integration-test", tag="latest", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None): self.name = name @@ -783,6 +787,7 @@ class ClickHouseInstance: self.client = None self.default_timeout = 20.0 # 20 sec self.image = image + self.tag = tag self.stay_alive = stay_alive self.ipv4_address = ipv4_address self.ipv6_address = ipv6_address @@ -1160,6 +1165,7 @@ class ClickHouseInstance: with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, + tag=self.tag, name=self.name, hostname=self.hostname, binary_volume=binary_volume, diff --git a/tests/integration/runner b/tests/integration/runner index e5d6eabe794..67a174eccfa 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -154,6 +154,8 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) elif image == "yandex/clickhouse-postgresql-java-client": env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-integration-test": + env_tags += "-e {}={}".format("DOCKER_BASE_TAG", tag) else: raise Exception("Unknown image {}".format(image)) diff --git a/tests/integration/test_host_ip_change/configs/dns_update_short.xml b/tests/integration/test_host_ip_change/configs/dns_update_short.xml index 2bfafe2ef21..3317f709b4a 100644 --- a/tests/integration/test_host_ip_change/configs/dns_update_short.xml +++ b/tests/integration/test_host_ip_change/configs/dns_update_short.xml @@ -1,3 +1,3 @@ - 2 + 1 diff --git a/tests/integration/test_host_ip_change/test.py b/tests/integration/test_host_ip_change/test.py index ac35478277c..e3e8c08c848 100644 --- a/tests/integration/test_host_ip_change/test.py +++ b/tests/integration/test_host_ip_change/test.py @@ -106,11 +106,16 @@ def test_ip_change_update_dns_cache(cluster_with_dns_cache_update): # Put some data to source node3 node3.query("INSERT INTO test_table_update VALUES ('2018-10-01', 5), ('2018-10-02', 6), ('2018-10-03', 7)") + + # Check that data is placed on node3 assert node3.query("SELECT count(*) from test_table_update") == "6\n" + result = node4.exec_in_container(["bash", "-c", "/usr/bin/host node3"]) + print("HOST RESULT %s", result) + # Because of DNS cache update, ip of node3 would be updated - assert_eq_with_retry(node4, "SELECT count(*) from test_table_update", "6") + assert_eq_with_retry(node4, "SELECT count(*) from test_table_update", "6", sleep_time=3) # Just to be sure check one more time node3.query("INSERT INTO test_table_update VALUES ('2018-10-01', 8)") From d04cda03677b5d3151f6d2eb24f63f181892e8e2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Sep 2020 02:22:33 +0300 Subject: [PATCH 098/535] Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter Example of such queries are distributed queries, which creates local InterpreterSelectQuery, which will have it's own QueryPlan but returns Pipes that has that IQueryPlanStep attached. After EXPLAIN PIPELINE graph=1 tries to use them, and will get SIGSEGV. - TSAN:
``` ==2782113==ERROR: AddressSanitizer: heap-use-after-free on address 0x6120000223c0 at pc 0x00002b8f3f3e bp 0x7fff18cfbff0 sp 0x7fff18cfbfe8 READ of size 8 at 0x6120000223c0 thread T22 (TCPHandler) #0 0x2b8f3f3d in DB::printPipelineCompact(std::__1::vector, std::__1::allocator > > const&, DB::WriteBuffer&, bool) /build/obj-x86_64-linux-gnu/../src/Processors/printPipeline.cpp:116:53 #1 0x29ee698c in DB::InterpreterExplainQuery::executeImpl() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:275:17 #2 0x29ee2e40 in DB::InterpreterExplainQuery::execute() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:73:14 #3 0x2a7b44a2 in DB::executeQueryImpl(char const*, char const*, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool, DB::ReadBuffer*) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:389:28 #4 0x2a7b1cb3 in DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:675:30 #5 0x2b7993b2 in DB::TCPHandler::runImpl() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:253:24 #6 0x2b7b649a in DB::TCPHandler::run() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1217:9 #7 0x31d9c57e in Poco::Net::TCPServerConnection::start() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:43:3 #8 0x31d9d281 in Poco::Net::TCPServerDispatcher::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:114:20 #9 0x3206b5d5 in Poco::PooledThread::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:199:14 #10 0x320657ad in Poco::ThreadImpl::runnableEntry(void*) /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:345:27 #11 0x7ffff7f853e8 in start_thread (/usr/lib/libpthread.so.0+0x93e8) #12 0x7ffff7ea2292 in clone (/usr/lib/libc.so.6+0x100292) 0x6120000223c0 is located 0 bytes inside of 272-byte region [0x6120000223c0,0x6120000224d0) freed by thread T22 (TCPHandler) here: #0 0x122f3b62 in operator delete(void*, unsigned long) (/src/ch/tmp/master-20200831/clickhouse+0x122f3b62) #1 0x2bd9e9fa in std::__1::default_delete::operator()(DB::IQueryPlanStep*) const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2363:5 #2 0x2bd9e9fa in std::__1::unique_ptr >::reset(DB::IQueryPlanStep*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2618:7 #3 0x2bd9e9fa in std::__1::unique_ptr >::~unique_ptr() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2572:19 #4 0x2bd9e9fa in DB::QueryPlan::Node::~Node() /build/obj-x86_64-linux-gnu/../src/Processors/QueryPlan/QueryPlan.h:66:12 #5 0x2bd9e9fa in void std::__1::allocator_traits > >::__destroy(std::__1::integral_constant, std::__1::allocator >&, DB::QueryPlan::Node*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1798:23 #6 0x2bd9e9fa in void std::__1::allocator_traits > >::destroy(std::__1::allocator >&, DB::QueryPlan::Node*) /build/obj-x86_64-lin ux-gnu/../contrib/libcxx/include/memory:1630:14 #7 0x2bd9e9fa in std::__1::__list_imp >::clear() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/list:762:13 #8 0x29fece08 in DB::InterpreterSelectQuery::execute() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:492:1 #9 0x2abf7484 in DB::ClusterProxy::(anonymous namespace)::createLocalStream(std::__1::shared_ptr const&, DB::Block const&, DB::Context const&, DB::QueryProcessingStage::Enum) /build/obj-x86_64-linux-gnu/../src/Interpreters/ClusterProxy/SelectStreamFactory.cpp: 78:33 #10 0x2abea85d in DB::ClusterProxy::SelectStreamFactory::createForShard(DB::Cluster::ShardInfo const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::shared_ptr const&, DB::Context const&, std::__1::shar ed_ptr const&, DB::SelectQueryInfo const&, std::__1::vector >&)::$_0::operator()() const /build/obj-x86_64-linux-gnu/../src/Interpreters/ClusterProxy/SelectStreamFactory.cpp:133:51 #11 0x2abea85d in DB::ClusterProxy::SelectStreamFactory::createForShard(DB::Cluster::ShardInfo const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::shared_ptr const&, DB::Context const&, std::__1::shar ed_ptr const&, DB::SelectQueryInfo const&, std::__1::vector >&) /build/obj-x86_64-linux-gnu/../src/Interpreters/ClusterProxy/SelectStreamFactory.cpp:189:13 #12 0x2abe6d99 in DB::ClusterProxy::executeQuery(DB::ClusterProxy::IStreamFactory&, std::__1::shared_ptr const&, Poco::Logger*, std::__1::shared_ptr const&, DB::Context const&, DB::Settings const&, DB::SelectQueryInfo const&) /build/obj-x86_64-lin ux-gnu/../src/Interpreters/ClusterProxy/executeQuery.cpp:107:24 #13 0x2abc4b74 in DB::StorageDistributed::read(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, s td::__1::shared_ptr const&, DB::SelectQueryInfo const&, DB::Context const&, DB::QueryProcessingStage::Enum, unsigned long, unsigned int) /build/obj-x86_64-linux-gnu/../src/Storages/StorageDistributed.cpp:514:12 #14 0x2bda1c5a in DB::ReadFromStorageStep::ReadFromStorageStep(std::__1::shared_ptr, std::__1::shared_ptr&, DB::SelectQueryOptions, std::__1::shared_ptr, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, DB::SelectQueryInfo const&, std::__1::shared_ptr, DB::QueryProcessingStage ::Enum, unsigned long, unsigned long) /build/obj-x86_64-linux-gnu/../src/Processors/QueryPlan/ReadFromStorageStep.cpp:39:26 #15 0x2a01ca70 in std::__1::__unique_if::__unique_single std::__1::make_unique&, std::__1::shared_ptr&, DB::SelectQueryOptions&, std ::__1::shared_ptr&, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > >&, DB::SelectQueryInfo&, st d::__1::shared_ptr&, DB::QueryProcessingStage::Enum&, unsigned long&, unsigned long&>(std::__1::shared_ptr&, std::__1::shared_ptr&, DB::SelectQueryOptions&, std::__1::shared_ptr&, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > >&, DB::SelectQueryInfo&, std::__1::shared_ptr&, DB::QueryProcessingStage::Enum&, unsigned long&, unsigned long&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3028:32 #16 0x29ff556a in DB::InterpreterSelectQuery::executeFetchColumns(DB::QueryProcessingStage::Enum, DB::QueryPlan&, std::__1::shared_ptr const&, std::__1::vector, std::__1::allocator >, std:: __1::allocator, std::__1::allocator > > > const&) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:1383:26 #17 0x29fe6b83 in DB::InterpreterSelectQuery::executeImpl(DB::QueryPlan&, std::__1::shared_ptr const&, std::__1::optional) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:795:9 #18 0x29fe5771 in DB::InterpreterSelectQuery::buildQueryPlan(DB::QueryPlan&) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:473:5 #19 0x2a47d370 in DB::InterpreterSelectWithUnionQuery::buildQueryPlan(DB::QueryPlan&) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectWithUnionQuery.cpp:182:38 #20 0x29ee5bff in DB::InterpreterExplainQuery::executeImpl() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:265:21 #21 0x29ee2e40 in DB::InterpreterExplainQuery::execute() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:73:14 #22 0x2a7b44a2 in DB::executeQueryImpl(char const*, char const*, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool, DB::ReadBuffer*) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:389:28 #23 0x2a7b1cb3 in DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:675:30 #24 0x2b7993b2 in DB::TCPHandler::runImpl() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:253:24 #25 0x2b7b649a in DB::TCPHandler::run() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1217:9 #26 0x31d9c57e in Poco::Net::TCPServerConnection::start() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:43:3 #27 0x31d9d281 in Poco::Net::TCPServerDispatcher::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:114:20 #28 0x3206b5d5 in Poco::PooledThread::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:199:14 #29 0x320657ad in Poco::ThreadImpl::runnableEntry(void*) /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:345:27 #30 0x7ffff7f853e8 in start_thread (/usr/lib/libpthread.so.0+0x93e8) ```
--- .../ClusterProxy/SelectStreamFactory.cpp | 18 ++++++++++++------ src/Interpreters/InterpreterExplainQuery.cpp | 4 +++- src/Processors/Pipe.cpp | 2 ++ src/Processors/Pipe.h | 8 ++++++++ src/Processors/QueryPipeline.h | 3 +++ .../0_stateless/01470_explain.reference | 0 tests/queries/0_stateless/01470_explain.sql | 6 ++++++ 7 files changed, 34 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01470_explain.reference create mode 100644 tests/queries/0_stateless/01470_explain.sql diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 986de85d712..ed7bd2cf71f 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -68,14 +69,19 @@ SelectStreamFactory::SelectStreamFactory( namespace { -QueryPipeline createLocalStream( +auto createLocalPipe( const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage) { checkStackSize(); - InterpreterSelectQuery interpreter{query_ast, context, SelectQueryOptions(processed_stage)}; + InterpreterSelectQuery interpreter(query_ast, context, SelectQueryOptions(processed_stage)); + auto query_plan = std::make_unique(); - auto pipeline = interpreter.execute().pipeline; + interpreter.buildQueryPlan(*query_plan); + auto pipeline = std::move(*query_plan->buildQueryPipeline()); + + /// Avoid going it out-of-scope for EXPLAIN + pipeline.addQueryPlan(std::move(query_plan)); pipeline.addSimpleTransform([&](const Block & source_header) { @@ -94,7 +100,7 @@ QueryPipeline createLocalStream( /// return std::make_shared(stream); pipeline.setMaxThreads(1); - return pipeline; + return QueryPipeline::getPipe(std::move(pipeline)); } String formattedAST(const ASTPtr & ast) @@ -130,7 +136,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - pipes.emplace_back(QueryPipeline::getPipe(createLocalStream(modified_query_ast, header, context, processed_stage))); + pipes.emplace_back(createLocalPipe(modified_query_ast, header, context, processed_stage)); }; String modified_query = formattedAST(modified_query_ast); @@ -270,7 +276,7 @@ void SelectStreamFactory::createForShard( } if (try_results.empty() || local_delay < max_remote_delay) - return QueryPipeline::getPipe(createLocalStream(modified_query_ast, header, context, stage)); + return createLocalPipe(modified_query_ast, header, context, stage); else { std::vector connections; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 9960509a5d7..c936556ce39 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -269,7 +269,9 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.graph) { - auto processors = Pipe::detachProcessors(QueryPipeline::getPipe(std::move(*pipeline))); + /// Pipe holds QueryPlan, should not go out-of-scope + auto pipe = QueryPipeline::getPipe(std::move(*pipeline)); + const auto & processors = pipe.getProcessors(); if (settings.compact) printPipelineCompact(processors, buffer, settings.query_pipeline_options.header); diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 93dcd561c00..d28e54dae58 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -102,6 +102,8 @@ Pipe::Holder & Pipe::Holder::operator=(Holder && rhs) storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end()); interpreter_context.insert(interpreter_context.end(), rhs.interpreter_context.begin(), rhs.interpreter_context.end()); + for (auto & plan : rhs.query_plans) + query_plans.emplace_back(std::move(plan)); return *this; } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 28b64937aeb..f5f8b117db9 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -8,6 +9,8 @@ namespace DB class Pipe; using Pipes = std::vector; +class QueryPipeline; + class IStorage; using StoragePtr = std::shared_ptr; @@ -86,6 +89,8 @@ public: /// Get processors from Pipe. Use it with cautious, it is easy to loss totals and extremes ports. static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); } + /// Get processors from Pipe w/o destroying pipe (used for EXPLAIN to keep QueryPlan). + const Processors & getProcessors() const { return processors; } /// Specify quotas and limits for every ISourceWithProgress. void setLimits(const SourceWithProgress::LocalLimits & limits); @@ -96,6 +101,8 @@ public: /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. void addInterpreterContext(std::shared_ptr context) { holder.interpreter_context.emplace_back(std::move(context)); } void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } + /// For queries with nested interpreters (i.e. StorageDistributed) + void addQueryPlan(std::unique_ptr plan) { holder.query_plans.emplace_back(std::move(plan)); } private: /// Destruction order: processors, header, locks, temporary storages, local contexts @@ -113,6 +120,7 @@ private: std::vector> interpreter_context; std::vector storage_holders; std::vector table_locks; + std::vector> query_plans; }; Holder holder; diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 385cf77198e..94de753bebc 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -21,6 +21,8 @@ class QueryPipelineProcessorsCollector; struct AggregatingTransformParams; using AggregatingTransformParamsPtr = std::shared_ptr; +class QueryPlan; + class QueryPipeline { public: @@ -93,6 +95,7 @@ public: void addTableLock(const TableLockHolder & lock) { pipe.addTableLock(lock); } void addInterpreterContext(std::shared_ptr context) { pipe.addInterpreterContext(std::move(context)); } void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } + void addQueryPlan(std::unique_ptr plan) { pipe.addQueryPlan(std::move(plan)); } /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); diff --git a/tests/queries/0_stateless/01470_explain.reference b/tests/queries/0_stateless/01470_explain.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01470_explain.sql b/tests/queries/0_stateless/01470_explain.sql new file mode 100644 index 00000000000..8fd145e7f65 --- /dev/null +++ b/tests/queries/0_stateless/01470_explain.sql @@ -0,0 +1,6 @@ +-- +-- regressions +-- + +-- SIGSEGV regression due to QueryPlan lifetime +EXPLAIN PIPELINE graph=1 SELECT * FROM remote('127.{1,2}', system.one) FORMAT Null; From 679afe5ff2bc792c6eadafa8a705113a42ac2c1b Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 1 Sep 2020 10:06:23 +0300 Subject: [PATCH 099/535] Revert "Documentation about ReplacingMergeTree extended with type DateTime64 for column (#13498)" This reverts commit 896b561523fb54361ef2e6748219f2bcbf625e4b. --- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md index 109ae6c4601..684e7e28112 100644 --- a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md @@ -31,7 +31,7 @@ For a description of request parameters, see [statement description](../../../sq **ReplacingMergeTree Parameters** -- `ver` — column with version. Type `UInt*`, `Date`, `DateTime` or `DateTime64`. Optional parameter. +- `ver` — column with version. Type `UInt*`, `Date` or `DateTime`. Optional parameter. When merging, `ReplacingMergeTree` from all the rows with the same sorting key leaves only one: diff --git a/docs/es/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/es/engines/table-engines/mergetree-family/replacingmergetree.md index cb3c6aea34b..a1e95c5b5f4 100644 --- a/docs/es/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/es/engines/table-engines/mergetree-family/replacingmergetree.md @@ -33,7 +33,7 @@ Para obtener una descripción de los parámetros de solicitud, consulte [descrip **ReplacingMergeTree Parámetros** -- `ver` — column with version. Type `UInt*`, `Date`, `DateTime` o `DateTime64`. Parámetro opcional. +- `ver` — column with version. Type `UInt*`, `Date` o `DateTime`. Parámetro opcional. Al fusionar, `ReplacingMergeTree` de todas las filas con la misma clave primaria deja solo una: diff --git a/docs/fa/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/fa/engines/table-engines/mergetree-family/replacingmergetree.md index 4ece20461cb..0ace0e05afc 100644 --- a/docs/fa/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/fa/engines/table-engines/mergetree-family/replacingmergetree.md @@ -33,7 +33,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **پارامترهای جایگزین** -- `ver` — column with version. Type `UInt*`, `Date`, `DateTime` یا `DateTime64`. پارامتر اختیاری. +- `ver` — column with version. Type `UInt*`, `Date` یا `DateTime`. پارامتر اختیاری. هنگام ادغام, `ReplacingMergeTree` از تمام ردیف ها با همان کلید اصلی تنها یک برگ دارد: diff --git a/docs/fr/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/fr/engines/table-engines/mergetree-family/replacingmergetree.md index 755249c1a38..ac3c0f3b021 100644 --- a/docs/fr/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/fr/engines/table-engines/mergetree-family/replacingmergetree.md @@ -33,7 +33,7 @@ Pour une description des paramètres de requête, voir [demande de description]( **ReplacingMergeTree Paramètres** -- `ver` — column with version. Type `UInt*`, `Date`, `DateTime` ou `DateTime64`. Paramètre facultatif. +- `ver` — column with version. Type `UInt*`, `Date` ou `DateTime`. Paramètre facultatif. Lors de la fusion, `ReplacingMergeTree` de toutes les lignes avec la même clé primaire ne laisse qu'un: diff --git a/docs/ja/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/ja/engines/table-engines/mergetree-family/replacingmergetree.md index e2cce893e3a..c3df9559415 100644 --- a/docs/ja/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/ja/engines/table-engines/mergetree-family/replacingmergetree.md @@ -33,7 +33,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **ReplacingMergeTreeパラメータ** -- `ver` — column with version. Type `UInt*`, `Date`, `DateTime` または `DateTime64`. 任意パラメータ。 +- `ver` — column with version. Type `UInt*`, `Date` または `DateTime`. 任意パラメータ。 マージ時, `ReplacingMergeTree` 同じ主キーを持つすべての行から、一つだけを残します: diff --git a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md index fefc3c65b38..4aa1eb556f3 100644 --- a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md @@ -25,7 +25,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **Параметры ReplacingMergeTree** -- `ver` — столбец с версией, тип `UInt*`, `Date`, `DateTime` или `DateTime64`. Необязательный параметр. +- `ver` — столбец с версией, тип `UInt*`, `Date` или `DateTime`. Необязательный параметр. При слиянии, из всех строк с одинаковым значением ключа сортировки `ReplacingMergeTree` оставляет только одну: diff --git a/docs/tr/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/tr/engines/table-engines/mergetree-family/replacingmergetree.md index f586b97cb2f..a24c84e9a16 100644 --- a/docs/tr/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/tr/engines/table-engines/mergetree-family/replacingmergetree.md @@ -33,7 +33,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **ReplacingMergeTree Parametreleri** -- `ver` — column with version. Type `UInt*`, `Date`, `DateTime` veya `DateTime64`. İsteğe bağlı parametre. +- `ver` — column with version. Type `UInt*`, `Date` veya `DateTime`. İsteğe bağlı parametre. Birleş whenirken, `ReplacingMergeTree` aynı birincil anahtara sahip tüm satırlardan sadece bir tane bırakır: diff --git a/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md index 03b47172400..626597eeaf0 100644 --- a/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md @@ -25,7 +25,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **参数** -- `ver` — 版本列。类型为 `UInt*`, `Date`, `DateTime` 或 `DateTime64`。可选参数。 +- `ver` — 版本列。类型为 `UInt*`, `Date` 或 `DateTime`。可选参数。 合并的时候,`ReplacingMergeTree` 从所有具有相同主键的行中选择一行留下: - 如果 `ver` 列未指定,选择最后一条。 From 2d7cb031202ae1cdbee29d11048dfe89bb3d5acc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Sep 2020 10:08:54 +0300 Subject: [PATCH 100/535] Suppress superfluous wget (-nv) output Since for dowloading some of files wget logging may take 50% of overall log [1]. [1]: https://clickhouse-builds.s3.yandex.net/14315/c32ff4c98cb3b83a12f945eadd180415b7a3b269/clickhouse_build_check/build_log_761119955_1598923036.txt --- docker/builder/Dockerfile | 2 +- docker/packager/binary/Dockerfile | 9 ++++----- docker/packager/deb/Dockerfile | 4 ++-- docker/test/base/Dockerfile | 2 +- docker/test/codebrowser/Dockerfile | 2 +- docker/test/fasttest/Dockerfile | 5 ++--- docker/test/fuzzer/run-fuzzer.sh | 4 ++-- docker/test/integration/runner/Dockerfile | 2 +- docker/test/pvs/Dockerfile | 8 ++++---- docker/test/stateless/Dockerfile | 2 +- docker/test/stateless_unbundled/Dockerfile | 2 +- docker/test/stateless_with_coverage/Dockerfile | 2 +- docker/test/testflows/runner/Dockerfile | 2 +- docs/en/interfaces/http.md | 2 +- docs/es/interfaces/http.md | 2 +- docs/fa/interfaces/http.md | 2 +- docs/fr/interfaces/http.md | 2 +- docs/ja/interfaces/http.md | 2 +- docs/ru/interfaces/http.md | 2 +- docs/tr/interfaces/http.md | 2 +- docs/zh/interfaces/http.md | 2 +- src/Functions/URL/tldLookup.sh | 2 +- utils/build/build_no_submodules.sh | 2 +- utils/ci/build-gcc-from-sources.sh | 2 +- utils/ci/docker-multiarch/update.sh | 10 +++++----- utils/clickhouse-docker | 2 +- 26 files changed, 39 insertions(+), 41 deletions(-) diff --git a/docker/builder/Dockerfile b/docker/builder/Dockerfile index b7dadc3ec6d..d4a121d13eb 100644 --- a/docker/builder/Dockerfile +++ b/docker/builder/Dockerfile @@ -6,7 +6,7 @@ RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ --yes --no-install-recommends --verbose-versions \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ + && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index b8650b945e1..e1133f337a9 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -7,7 +7,7 @@ RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ --yes --no-install-recommends --verbose-versions \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ + && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ @@ -55,7 +55,6 @@ RUN apt-get update \ cmake \ gdb \ rename \ - wget \ build-essential \ --yes --no-install-recommends @@ -83,14 +82,14 @@ RUN git clone https://github.com/tpoechtrager/cctools-port.git \ && rm -rf cctools-port # Download toolchain for Darwin -RUN wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz +RUN wget -nv https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz # Download toolchain for ARM # It contains all required headers and libraries. Note that it's named as "gcc" but actually we are using clang for cross compiling. -RUN wget "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en" -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz +RUN wget -nv "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en" -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz # Download toolchain for FreeBSD 11.3 -RUN wget https://clickhouse-datasets.s3.yandex.net/toolchains/toolchains/freebsd-11.3-toolchain.tar.xz +RUN wget -nv https://clickhouse-datasets.s3.yandex.net/toolchains/toolchains/freebsd-11.3-toolchain.tar.xz COPY build.sh / CMD ["/bin/bash", "/build.sh"] diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 6d0fdca2310..87f4582f8e2 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -7,7 +7,7 @@ RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ --yes --no-install-recommends --verbose-versions \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ + && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ @@ -34,7 +34,7 @@ RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb \ ENV APACHE_PUBKEY_HASH="bba6987b63c63f710fd4ed476121c588bc3812e99659d27a855f8c4d312783ee66ad6adfce238765691b04d62fa3688f" RUN export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ - && wget -O /tmp/arrow-keyring.deb "https://apache.bintray.com/arrow/ubuntu/apache-arrow-archive-keyring-latest-${CODENAME}.deb" \ + && wget -nv -O /tmp/arrow-keyring.deb "https://apache.bintray.com/arrow/ubuntu/apache-arrow-archive-keyring-latest-${CODENAME}.deb" \ && echo "${APACHE_PUBKEY_HASH} /tmp/arrow-keyring.deb" | sha384sum -c \ && dpkg -i /tmp/arrow-keyring.deb diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index c9b0700ecfc..8117d2907bc 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -7,7 +7,7 @@ RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ --yes --no-install-recommends --verbose-versions \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ + && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ diff --git a/docker/test/codebrowser/Dockerfile b/docker/test/codebrowser/Dockerfile index f9d239ef8ef..cb3462cad0e 100644 --- a/docker/test/codebrowser/Dockerfile +++ b/docker/test/codebrowser/Dockerfile @@ -15,7 +15,7 @@ RUN apt-get --allow-unauthenticated update -y \ gpg-agent \ git -RUN wget -O - https://apt.kitware.com/keys/kitware-archive-latest.asc 2>/dev/null | sudo apt-key add - +RUN wget -nv -O - https://apt.kitware.com/keys/kitware-archive-latest.asc | sudo apt-key add - RUN sudo apt-add-repository 'deb https://apt.kitware.com/ubuntu/ bionic main' RUN sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 49845d72f1d..9b4bb574f8f 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -7,7 +7,7 @@ RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ --yes --no-install-recommends --verbose-versions \ && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ + && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ @@ -61,7 +61,6 @@ RUN apt-get update \ software-properties-common \ tzdata \ unixodbc \ - wget \ --yes --no-install-recommends # This symlink required by gcc to find lld compiler @@ -70,7 +69,7 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz" RUN mkdir -p /tmp/clickhouse-odbc-tmp \ - && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ + && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ && cp /tmp/clickhouse-odbc-tmp/lib64/*.so /usr/local/lib/ \ && odbcinst -i -d -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbcinst.ini.sample \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 8cfe1a87408..a319033a232 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -32,10 +32,10 @@ function clone function download { -# wget -O- -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/performance/performance.tgz" \ +# wget -nv -O- -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/performance/performance.tgz" \ # | tar --strip-components=1 -zxv - wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-10_debug_none_bundled_unsplitted_disable_False_binary/clickhouse" + wget -nv -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-10_debug_none_bundled_unsplitted_disable_False_binary/clickhouse" chmod +x clickhouse ln -s ./clickhouse ./clickhouse-server ln -s ./clickhouse ./clickhouse-client diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 95ab516cdaa..bfbe8da816f 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -46,7 +46,7 @@ RUN set -eux; \ \ # this "case" statement is generated via "update.sh" \ - if ! wget -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ + if ! wget -nv -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \ exit 1; \ fi; \ diff --git a/docker/test/pvs/Dockerfile b/docker/test/pvs/Dockerfile index ebd9c105705..0aedb67e572 100644 --- a/docker/test/pvs/Dockerfile +++ b/docker/test/pvs/Dockerfile @@ -12,8 +12,8 @@ RUN apt-get update --yes \ strace \ --yes --no-install-recommends -#RUN wget -q -O - http://files.viva64.com/etc/pubkey.txt | sudo apt-key add - -#RUN sudo wget -O /etc/apt/sources.list.d/viva64.list http://files.viva64.com/etc/viva64.list +#RUN wget -nv -O - http://files.viva64.com/etc/pubkey.txt | sudo apt-key add - +#RUN sudo wget -nv -O /etc/apt/sources.list.d/viva64.list http://files.viva64.com/etc/viva64.list # #RUN apt-get --allow-unauthenticated update -y \ # && env DEBIAN_FRONTEND=noninteractive \ @@ -24,10 +24,10 @@ ENV PKG_VERSION="pvs-studio-latest" RUN set -x \ && export PUBKEY_HASHSUM="486a0694c7f92e96190bbfac01c3b5ac2cb7823981db510a28f744c99eabbbf17a7bcee53ca42dc6d84d4323c2742761" \ - && wget https://files.viva64.com/etc/pubkey.txt -O /tmp/pubkey.txt \ + && wget -nv https://files.viva64.com/etc/pubkey.txt -O /tmp/pubkey.txt \ && echo "${PUBKEY_HASHSUM} /tmp/pubkey.txt" | sha384sum -c \ && apt-key add /tmp/pubkey.txt \ - && wget "https://files.viva64.com/${PKG_VERSION}.deb" \ + && wget -nv "https://files.viva64.com/${PKG_VERSION}.deb" \ && { debsig-verify ${PKG_VERSION}.deb \ || echo "WARNING: Some file was just downloaded from the internet without any validation and we are installing it into the system"; } \ && dpkg -i "${PKG_VERSION}.deb" diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index d3bc03a8f92..409a1b07bef 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -26,7 +26,7 @@ RUN apt-get update -y \ zookeeperd RUN mkdir -p /tmp/clickhouse-odbc-tmp \ - && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ + && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ && cp /tmp/clickhouse-odbc-tmp/lib64/*.so /usr/local/lib/ \ && odbcinst -i -d -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbcinst.ini.sample \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index 7de29fede72..b05e46406da 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -71,7 +71,7 @@ RUN apt-get --allow-unauthenticated update -y \ zookeeperd RUN mkdir -p /tmp/clickhouse-odbc-tmp \ - && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ + && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ && cp /tmp/clickhouse-odbc-tmp/lib64/*.so /usr/local/lib/ \ && odbcinst -i -d -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbcinst.ini.sample \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ diff --git a/docker/test/stateless_with_coverage/Dockerfile b/docker/test/stateless_with_coverage/Dockerfile index f3539804852..77357d5142f 100644 --- a/docker/test/stateless_with_coverage/Dockerfile +++ b/docker/test/stateless_with_coverage/Dockerfile @@ -33,7 +33,7 @@ RUN apt-get update -y \ qemu-user-static RUN mkdir -p /tmp/clickhouse-odbc-tmp \ - && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ + && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ && cp /tmp/clickhouse-odbc-tmp/lib64/*.so /usr/local/lib/ \ && odbcinst -i -d -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbcinst.ini.sample \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile index 6b4ec12b80c..898552ade56 100644 --- a/docker/test/testflows/runner/Dockerfile +++ b/docker/test/testflows/runner/Dockerfile @@ -44,7 +44,7 @@ RUN set -eux; \ \ # this "case" statement is generated via "update.sh" \ - if ! wget -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ + if ! wget -nv -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \ exit 1; \ fi; \ diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index a5e7ef22558..35c79b5ee02 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -36,7 +36,7 @@ Examples: $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 diff --git a/docs/es/interfaces/http.md b/docs/es/interfaces/http.md index abc5cf63188..ebce0ec7a51 100644 --- a/docs/es/interfaces/http.md +++ b/docs/es/interfaces/http.md @@ -38,7 +38,7 @@ Ejemplos: $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 diff --git a/docs/fa/interfaces/http.md b/docs/fa/interfaces/http.md index 774980cf8fb..9ce40c17e6f 100644 --- a/docs/fa/interfaces/http.md +++ b/docs/fa/interfaces/http.md @@ -38,7 +38,7 @@ Ok. $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 diff --git a/docs/fr/interfaces/http.md b/docs/fr/interfaces/http.md index 2de32747d4a..a414bba2c2f 100644 --- a/docs/fr/interfaces/http.md +++ b/docs/fr/interfaces/http.md @@ -38,7 +38,7 @@ Exemple: $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 diff --git a/docs/ja/interfaces/http.md b/docs/ja/interfaces/http.md index c76b1ba0827..31f2b54af6d 100644 --- a/docs/ja/interfaces/http.md +++ b/docs/ja/interfaces/http.md @@ -38,7 +38,7 @@ GETメソッドを使用する場合, ‘readonly’ 設定されています。 $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index afd4d083365..b1cc4c79b25 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -31,7 +31,7 @@ Ok. $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 diff --git a/docs/tr/interfaces/http.md b/docs/tr/interfaces/http.md index 2b92dd0ed9b..49d20ef6655 100644 --- a/docs/tr/interfaces/http.md +++ b/docs/tr/interfaces/http.md @@ -38,7 +38,7 @@ GET yöntemini kullanırken, ‘readonly’ ayar .lanmıştır. Başka bir deyi $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index 0fecb1873db..9feb8c5d69d 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -23,7 +23,7 @@ Ok. $ curl 'http://localhost:8123/?query=SELECT%201' 1 -$ wget -O- -q 'http://localhost:8123/?query=SELECT 1' +$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' 1 $ GET 'http://localhost:8123/?query=SELECT 1' diff --git a/src/Functions/URL/tldLookup.sh b/src/Functions/URL/tldLookup.sh index a61f2b09660..a7893c3a168 100755 --- a/src/Functions/URL/tldLookup.sh +++ b/src/Functions/URL/tldLookup.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -[ ! -f public_suffix_list.dat ] && wget -O public_suffix_list.dat https://publicsuffix.org/list/public_suffix_list.dat +[ ! -f public_suffix_list.dat ] && wget -nv -O public_suffix_list.dat https://publicsuffix.org/list/public_suffix_list.dat echo '%language=C++ %define lookup-function-name is_valid diff --git a/utils/build/build_no_submodules.sh b/utils/build/build_no_submodules.sh index 4bcbe0b2a17..f9e2b6032a5 100755 --- a/utils/build/build_no_submodules.sh +++ b/utils/build/build_no_submodules.sh @@ -11,7 +11,7 @@ ROOT_DIR=${CUR_DIR}/../../build_no_submodules mkdir -p $ROOT_DIR cd $ROOT_DIR URL=`git remote get-url origin | sed 's/.git$//'` -wget -O ch.zip $URL/archive/${BRANCH}.zip +wget -nv -O ch.zip $URL/archive/${BRANCH}.zip unzip -ou ch.zip # TODO: make disableable lz4 zstd diff --git a/utils/ci/build-gcc-from-sources.sh b/utils/ci/build-gcc-from-sources.sh index 06d9820a022..8886bb7afd7 100755 --- a/utils/ci/build-gcc-from-sources.sh +++ b/utils/ci/build-gcc-from-sources.sh @@ -18,7 +18,7 @@ THREADS=$(grep -c ^processor /proc/cpuinfo) mkdir "${WORKSPACE}/gcc" pushd "${WORKSPACE}/gcc" -wget https://ftpmirror.gnu.org/gcc/${GCC_SOURCES_VERSION}/${GCC_SOURCES_VERSION}.tar.xz +wget -nv https://ftpmirror.gnu.org/gcc/${GCC_SOURCES_VERSION}/${GCC_SOURCES_VERSION}.tar.xz tar xf ${GCC_SOURCES_VERSION}.tar.xz pushd ${GCC_SOURCES_VERSION} ./contrib/download_prerequisites diff --git a/utils/ci/docker-multiarch/update.sh b/utils/ci/docker-multiarch/update.sh index 6abcf339607..1348631bdcf 100755 --- a/utils/ci/docker-multiarch/update.sh +++ b/utils/ci/docker-multiarch/update.sh @@ -29,7 +29,7 @@ baseUrl="https://partner-images.canonical.com/core/$VERSION" # install qemu-user-static if [ -n "${QEMU_ARCH}" ]; then if [ ! -f x86_64_qemu-${QEMU_ARCH}-static.tar.gz ]; then - wget -N https://github.com/multiarch/qemu-user-static/releases/download/${QEMU_VER}/x86_64_qemu-${QEMU_ARCH}-static.tar.gz + wget -nv -N https://github.com/multiarch/qemu-user-static/releases/download/${QEMU_VER}/x86_64_qemu-${QEMU_ARCH}-static.tar.gz fi tar -xvf x86_64_qemu-${QEMU_ARCH}-static.tar.gz -C $ROOTFS/usr/bin/ fi @@ -37,13 +37,13 @@ fi # get the image if \ - wget -q --spider "$baseUrl/current" \ - && wget -q --spider "$baseUrl/current/$thisTar" \ + wget -nv --spider "$baseUrl/current" \ + && wget -nv --spider "$baseUrl/current/$thisTar" \ ; then baseUrl+='/current' fi -wget -qN "$baseUrl/"{{MD5,SHA{1,256}}SUMS{,.gpg},"$thisTarBase.manifest",'unpacked/build-info.txt'} || true -wget -N "$baseUrl/$thisTar" +wget -nv -N "$baseUrl/"{{MD5,SHA{1,256}}SUMS{,.gpg},"$thisTarBase.manifest",'unpacked/build-info.txt'} || true +wget -nv -N "$baseUrl/$thisTar" # check checksum if [ -f SHA256SUMS ]; then diff --git a/utils/clickhouse-docker b/utils/clickhouse-docker index a3354aadacb..383a82e6d2c 100755 --- a/utils/clickhouse-docker +++ b/utils/clickhouse-docker @@ -24,7 +24,7 @@ param="$1" if [ "${param}" = "list" ] then # https://stackoverflow.com/a/39454426/1555175 - wget -q https://registry.hub.docker.com/v1/repositories/yandex/clickhouse-server/tags -O - | sed -e 's/[][]//g' -e 's/"//g' -e 's/ //g' | tr '}' '\n' | awk -F: '{print $3}' + wget -nv https://registry.hub.docker.com/v1/repositories/yandex/clickhouse-server/tags -O - | sed -e 's/[][]//g' -e 's/"//g' -e 's/ //g' | tr '}' '\n' | awk -F: '{print $3}' else docker pull yandex/clickhouse-server:${param} tmp_dir=$(mktemp -d -t ci-XXXXXXXXXX) # older version require /nonexistent folder to exist to run clickhouse client :D From c2fb72abf8ec372df0dfe23af3223d74274708aa Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Aug 2020 16:34:16 +0000 Subject: [PATCH 101/535] Better mv, more comments --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 14 +- .../RabbitMQ/RabbitMQBlockInputStream.h | 4 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 5 +- .../ReadBufferFromRabbitMQConsumer.cpp | 9 +- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 1 + src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 139 ++++++++++----- src/Storages/RabbitMQ/StorageRabbitMQ.h | 12 +- .../WriteBufferToRabbitMQProducer.cpp | 29 ++-- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 57 ++++-- .../integration/test_storage_rabbitmq/test.py | 164 +++++++++--------- 10 files changed, 269 insertions(+), 165 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 4742ea2a33a..be634a4fa3d 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -16,11 +16,13 @@ RabbitMQBlockInputStream::RabbitMQBlockInputStream( const StorageMetadataPtr & metadata_snapshot_, Context & context_, const Names & columns, + size_t max_block_size_, bool ack_in_suffix_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) + , max_block_size(max_block_size_) , ack_in_suffix(ack_in_suffix_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header(metadata_snapshot->getSampleBlockForColumns( @@ -51,12 +53,14 @@ void RabbitMQBlockInputStream::readPrefixImpl() } -bool RabbitMQBlockInputStream::needManualChannelUpdate() +bool RabbitMQBlockInputStream::needChannelUpdate() { if (!buffer) return false; - return !buffer->channelUsable() && buffer->channelAllowed() && storage.connectionRunning(); + ChannelPtr channel = buffer->getChannel(); + + return !channel || !channel->usable(); } @@ -83,7 +87,7 @@ Block RabbitMQBlockInputStream::readImpl() MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, 1); + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); InputPort port(input_format->getPort().getHeader(), input_format.get()); connect(input_format->getPort(), port); @@ -164,7 +168,7 @@ Block RabbitMQBlockInputStream::readImpl() buffer->allowNext(); - if (buffer->queueEmpty() || !checkTimeLimit()) + if (total_rows >= max_block_size || buffer->queueEmpty() || buffer->consumerStopped() || !checkTimeLimit()) break; } @@ -189,7 +193,7 @@ void RabbitMQBlockInputStream::readSuffixImpl() bool RabbitMQBlockInputStream::sendAck() { - if (!buffer || !buffer->channelUsable()) + if (!buffer) return false; if (!buffer->ackMessages()) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h index 4f52d64189e..0cfd9c2e350 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h @@ -18,6 +18,7 @@ public: const StorageMetadataPtr & metadata_snapshot_, Context & context_, const Names & columns, + size_t max_block_size_, bool ack_in_suffix = true); ~RabbitMQBlockInputStream() override; @@ -29,8 +30,8 @@ public: Block readImpl() override; void readSuffixImpl() override; + bool needChannelUpdate(); void updateChannel(); - bool needManualChannelUpdate(); bool sendAck(); private: @@ -38,6 +39,7 @@ private: StorageMetadataPtr metadata_snapshot; Context context; Names column_names; + const size_t max_block_size; bool ack_in_suffix; bool finished = false; diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 110093ef2f3..2f8d6adfa16 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -2,7 +2,6 @@ #include - namespace DB { class ASTStorage; @@ -21,11 +20,11 @@ namespace DB M(String, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ M(String, rabbitmq_deadletter_exchange, "", "Exchange name to be passed as a dead-letter-exchange name.", 0) \ M(Bool, rabbitmq_persistent, false, "If set, delivery mode will be set to 2 (makes messages 'persistent', durable).", 0) \ + M(UInt64, rabbitmq_skip_broken_messages, 0, "Skip at least this number of broken messages from RabbitMQ per block", 0) \ M(UInt64, rabbitmq_max_block_size, 0, "Number of row collected before flushing data from RabbitMQ.", 0) \ M(Milliseconds, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \ - DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) - +DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) struct RabbitMQSettings : public BaseSettings { diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 80a630117d8..074f74c91aa 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -50,7 +50,6 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , row_delimiter(row_delimiter_) , stopped(stopped_) , received(QUEUE_SIZE * num_queues) - , last_inserted_record(AckTracker()) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) bindQueue(queue_id); @@ -165,10 +164,14 @@ bool ReadBufferFromRabbitMQConsumer::ackMessages() { /// Commit all received messages with delivery tags from last commited to last inserted if (!consumer_channel->ack(record.delivery_tag, AMQP::multiple)) + { + LOG_ERROR(log, "Failed to commit messages with delivery tags from last commited to {} on channel {}", + record.delivery_tag, channel_id); return false; + } prev_tag = record.delivery_tag; - LOG_TRACE(log, "Consumer acknowledged messages with deliveryTags up to {} on channel {}", record.delivery_tag, channel_id); + LOG_TRACE(log, "Consumer commited messages with deliveryTags up to {} on channel {}", record.delivery_tag, channel_id); } return true; @@ -207,6 +210,8 @@ void ReadBufferFromRabbitMQConsumer::setupChannel() consumer_channel->onError([&](const char * message) { + /// If here, then fatal error occured on the channel and it is not usable anymore, need to close it + consumer_channel->close(); LOG_ERROR(log, "Channel {} error: {}", channel_id, message); channel_error.store(true); diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 5524a5b52cc..28c67e0314e 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -59,6 +59,7 @@ public: bool channelUsable() { return !channel_error.load(); } /// Do not allow to update channel untill current channel is properly set up and subscribed bool channelAllowed() { return !wait_subscription.load(); } + bool consumerStopped() { return stopped; } ChannelPtr & getChannel() { return consumer_channel; } void setupChannel(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 4b013d11574..44c57a0db3f 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -107,8 +107,7 @@ StorageRabbitMQ::StorageRabbitMQ( setInMemoryMetadata(storage_metadata); rabbitmq_context.makeQueryContext(); - if (!schema_name.empty()) - rabbitmq_context.setSetting("format_schema", schema_name); + rabbitmq_context = addSettings(rabbitmq_context); /// One looping task for all consumers as they share the same connection == the same handler == the same event loop event_handler->updateLoopState(Loop::STOP); @@ -193,6 +192,19 @@ String StorageRabbitMQ::getTableBasedName(String name, const StorageID & table_i } +Context StorageRabbitMQ::addSettings(Context context) +{ + context.setSetting("input_format_skip_unknown_fields", true); + context.setSetting("input_format_allow_errors_ratio", 0.); + context.setSetting("input_format_allow_errors_num", rabbitmq_settings->rabbitmq_skip_broken_messages.value); + + if (!schema_name.empty()) + context.setSetting("format_schema", schema_name); + + return context; +} + + void StorageRabbitMQ::heartbeatFunc() { if (!stream_cancelled && event_handler->connectionRunning()) @@ -215,10 +227,11 @@ void StorageRabbitMQ::loopingFunc() */ void StorageRabbitMQ::deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop) { + if (stop_loop) + event_handler->updateLoopState(Loop::STOP); + if (task_mutex.try_lock()) { - if (stop_loop) - event_handler->updateLoopState(Loop::STOP); task->deactivate(); task_mutex.unlock(); @@ -232,6 +245,14 @@ void StorageRabbitMQ::deactivateTask(BackgroundSchedulePool::TaskHolder & task, } +size_t StorageRabbitMQ::getMaxBlockSize() + { + return rabbitmq_settings->rabbitmq_max_block_size.changed + ? rabbitmq_settings->rabbitmq_max_block_size.value + : (global_context.getSettingsRef().max_insert_block_size.value / num_consumers); + } + + void StorageRabbitMQ::initExchange() { /* Binding scheme is the following: client's exchange -> key bindings by routing key list -> bridge exchange (fanout) -> @@ -240,7 +261,15 @@ void StorageRabbitMQ::initExchange() setup_channel->declareExchange(exchange_name, exchange_type, AMQP::durable) .onError([&](const char * message) { - throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " + /* This error can be a result of attempt to declare exchange if it was already declared but + * 1) with different exchange type. In this case can + * - manually delete previously declared exchange and create a new one. + * - throw an error that the exchange with this name but another type is already declared and ask client to delete it himself + * if it is not needed anymore or use another exchange name. + * 2) with different exchange settings. This can only happen if client himself declared exchange with the same name and + * specified its own settings, which differ from this implementation. + */ + throw Exception("Unable to declare exchange (1). Make sure specified exchange is not already declared. Error: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); @@ -248,7 +277,8 @@ void StorageRabbitMQ::initExchange() setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) .onError([&](const char * message) { - throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); + /// This error is not supposed to happen as this exchange name is always unique to type and its settings + throw Exception("Unable to declare exchange (2). Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); if (!hash_exchange) @@ -267,13 +297,17 @@ void StorageRabbitMQ::initExchange() setup_channel->declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) .onError([&](const char * message) { - throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); + /* This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared + * to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter + * is bad. + */ + throw Exception("Unable to declare exchange (3). Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); setup_channel->bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unable to bind exchange (2). Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); consumer_exchange = sharding_exchange; @@ -302,7 +336,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unable to bind exchange (1). Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) @@ -314,7 +348,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unable to bind exchange (1). Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } else @@ -330,7 +364,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unable to bind exchange (1). Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } } @@ -348,7 +382,7 @@ bool StorageRabbitMQ::restoreConnection(bool reconnecting) if (reconnecting) { - deactivateTask(heartbeat_task, 0, 0); + deactivateTask(heartbeat_task, false, false); connection->close(); /// Connection might be unusable, but not closed /* Connection is not closed immediately (firstly, all pending operations are completed, and then @@ -393,8 +427,8 @@ void StorageRabbitMQ::unbindExchange() * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local * queues, then messages will go both ways and in one of them they will remain not consumed. So need to disconnect local exchange - * bindings to remove redunadant message copies, but after that mv cannot work unless thoso bindings recreated. Recreating them is not - * difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. + * bindings to remove redunadant message copies, but after that mv cannot work unless those bindings are recreated. Recreating them is + * not difficult but very ugly and as probably nobody will do such thing - bindings will not be recreated. */ std::call_once(flag, [&]() { @@ -435,20 +469,17 @@ Pipe StorageRabbitMQ::read( auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); - auto modified_context = context; - if (!schema_name.empty()) - modified_context.setSetting("format_schema", schema_name); + auto modified_context = addSettings(context); + auto block_size = getMaxBlockSize(); bool update_channels = false; if (!event_handler->connectionRunning()) { if (event_handler->loopRunning()) - { - event_handler->updateLoopState(Loop::STOP); - looping_task->deactivate(); - } + deactivateTask(looping_task, false, true); - if ((update_channels = restoreConnection(true))) + update_channels = restoreConnection(true); + if (update_channels) heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); } @@ -457,20 +488,20 @@ Pipe StorageRabbitMQ::read( for (size_t i = 0; i < num_created_consumers; ++i) { - auto rabbit_stream = std::make_shared(*this, metadata_snapshot, modified_context, column_names); + auto rabbit_stream = std::make_shared( + *this, metadata_snapshot, modified_context, column_names, block_size); /* It is a possible but rare case when channel gets into error state and does not also close connection, so need manual update. * But I believe that in current context and with local rabbitmq settings this will never happen and any channel error will also * close connection, but checking anyway (in second condition of if statement). This must be done here (and also in streamToViews()) * and not in readPrefix as it requires to stop heartbeats and looping tasks to avoid race conditions inside the library */ - if (update_channels || rabbit_stream->needManualChannelUpdate()) + if ((update_channels || rabbit_stream->needChannelUpdate()) && connection->usable()) { if (event_handler->loopRunning()) { - event_handler->updateLoopState(Loop::STOP); - looping_task->deactivate(); - heartbeat_task->deactivate(); + deactivateTask(looping_task, false, true); + deactivateTask(heartbeat_task, false, false); } rabbit_stream->updateChannel(); @@ -526,9 +557,9 @@ void StorageRabbitMQ::shutdown() stream_cancelled = true; wait_confirm.store(false); - deactivateTask(streaming_task, 1, 1); - deactivateTask(heartbeat_task, 1, 0); - deactivateTask(looping_task, 1, 1); + deactivateTask(streaming_task, true, false); + deactivateTask(heartbeat_task, true, false); + deactivateTask(looping_task, true, true); connection->close(); @@ -594,7 +625,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - producer_id.fetch_add(1), unique_strbase, persistent, wait_confirm, log, + producer_id.fetch_add(1), persistent, wait_confirm, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } @@ -683,19 +714,25 @@ bool StorageRabbitMQ::streamToViews() if (!event_handler->loopRunning() && event_handler->connectionRunning()) looping_task->activateAndSchedule(); + auto block_size = getMaxBlockSize(); + // Create a stream for each consumer and join them in a union stream BlockInputStreams streams; streams.reserve(num_created_consumers); for (size_t i = 0; i < num_created_consumers; ++i) { - auto stream = std::make_shared(*this, metadata_snapshot, rabbitmq_context, column_names, false); + auto stream = std::make_shared( + *this, metadata_snapshot, rabbitmq_context, column_names, block_size, false); streams.emplace_back(stream); // Limit read batch to maximum block size to allow DDL IBlockInputStream::LocalLimits limits; - limits.speed_limits.max_execution_time = global_context.getSettingsRef().stream_flush_interval_ms; + limits.speed_limits.max_execution_time = rabbitmq_settings->rabbitmq_flush_interval_ms.changed + ? rabbitmq_settings->rabbitmq_flush_interval_ms + : global_context.getSettingsRef().stream_flush_interval_ms; + limits.timeout_overflow_mode = OverflowMode::BREAK; stream->setLimits(limits); @@ -715,7 +752,7 @@ bool StorageRabbitMQ::streamToViews() * races inside the library, but only in case any error occurs or connection is lost while ack is being sent */ if (event_handler->loopRunning()) - deactivateTask(looping_task, 0, 1); + deactivateTask(looping_task, false, true); if (!event_handler->connectionRunning()) { @@ -733,20 +770,37 @@ bool StorageRabbitMQ::streamToViews() } else { - deactivateTask(heartbeat_task, 0, 0); + deactivateTask(heartbeat_task, false, false); /// Commit for (auto & stream : streams) { + /* false is returned by the sendAck function in only two cases: + * 1) if connection failed. In this case all channels will be closed and will be unable to send ack. Also ack is made based on + * delivery tags, which are unique to channels, so if channels fail, those delivery tags will become invalid and there is + * no way to send specific ack from a different channel. Actually once the server realises that it has messages in a queue + * waiting for confirm from a channel which suddenly closed, it will immediately make those messages accessible to other + * consumers. So in this case duplicates are inevitable. + * 2) size of the sent frame (libraries's internal request interface) exceeds max frame - internal library error. This is more + * common for message frames, but not likely to happen to ack frame I suppose. So I do not believe it is likely to happen. + * Also in this case if channel didn't get closed - it is ok if failed to send ack, because the next attempt to send ack on + * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this + * will ever happen. + */ if (!stream->as()->sendAck()) { - /* Almost any error with channel will lead to connection closure, but if so happens that channel errored and connection - * is not closed - also need to restore channels - */ - if (!stream->as()->needManualChannelUpdate()) - stream->as()->updateChannel(); + if (connection->usable()) + { + /* Almost any error with channel will lead to connection closure, but if so happens that channel errored and + * connection is not closed - also need to restore channels + */ + if (!stream->as()->needChannelUpdate()) + stream->as()->updateChannel(); + } else + { break; + } } } } @@ -809,8 +863,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) CHECK_RABBITMQ_STORAGE_ARGUMENT(11, rabbitmq_deadletter_exchange) CHECK_RABBITMQ_STORAGE_ARGUMENT(12, rabbitmq_persistent) - CHECK_RABBITMQ_STORAGE_ARGUMENT(13, rabbitmq_max_block_size) - CHECK_RABBITMQ_STORAGE_ARGUMENT(14, rabbitmq_flush_interval_ms) + CHECK_RABBITMQ_STORAGE_ARGUMENT(13, rabbitmq_skip_broken_messages) + CHECK_RABBITMQ_STORAGE_ARGUMENT(14, rabbitmq_max_block_size) + CHECK_RABBITMQ_STORAGE_ARGUMENT(15, rabbitmq_flush_interval_ms) #undef CHECK_RABBITMQ_STORAGE_ARGUMENT diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 522dfff9a23..e4e90abd98b 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -104,12 +104,13 @@ private: std::mutex mutex, task_mutex; std::vector buffers; /// available buffers for RabbitMQ consumers - String unique_strbase; + String unique_strbase; /// to make unique consumer channel id String sharding_exchange, bridge_exchange, consumer_exchange; - std::once_flag flag; - size_t consumer_id = 0; - std::atomic producer_id = 1; - std::atomic wait_confirm = true, exchange_removed = false; + std::once_flag flag; /// remove exchange only once + size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id + std::atomic producer_id = 1; /// counter for producer buffer, needed for channel id + std::atomic wait_confirm = true; /// needed to break waiting for confirmations for producer + std::atomic exchange_removed = false; ChannelPtr setup_channel; BackgroundSchedulePool::TaskHolder streaming_task; @@ -126,6 +127,7 @@ private: Names parseRoutingKeys(String routing_key_list); AMQP::ExchangeType defineExchangeType(String exchange_type_); + Context addSettings(Context context); size_t getMaxBlockSize(); String getTableBasedName(String name, const StorageID & table_id); void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 945ebd5ac9a..38f62ff39b2 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -18,6 +18,11 @@ static const auto RETRIES_MAX = 20; static const auto BATCH = 1000; static const auto RETURNED_LIMIT = 50000; +namespace ErrorCodes +{ + extern const int CANNOT_CONNECT_RABBITMQ; +} + WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address_, Context & global_context, @@ -26,7 +31,6 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const String & exchange_name_, const AMQP::ExchangeType exchange_type_, const size_t channel_id_base_, - const String channel_base_, const bool persistent_, std::atomic & wait_confirm_, Poco::Logger * log_, @@ -40,7 +44,6 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , exchange_name(exchange_name_) , exchange_type(exchange_type_) , channel_id_base(std::to_string(channel_id_base_)) - , channel_base(channel_base_) , persistent(persistent_) , wait_confirm(wait_confirm_) , payloads(BATCH) @@ -56,7 +59,16 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( event_handler = std::make_unique(loop.get(), log); if (setupConnection(false)) + { setupChannel(); + } + else + { + if (!connection->closed()) + connection->close(true); + + throw Exception("Cannot connect to RabbitMQ", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + } writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); @@ -175,7 +187,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->onReady([&]() { - channel_id = channel_id_base + std::to_string(channel_id_counter++) + "_" + channel_base; + channel_id = channel_id_base + std::to_string(channel_id_counter++); LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, @@ -187,17 +199,17 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->confirmSelect() .onAck([&](uint64_t acked_delivery_tag, bool multiple) { - removeConfirmed(acked_delivery_tag, multiple, false); + removeRecord(acked_delivery_tag, multiple, false); }) .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) { - removeConfirmed(nacked_delivery_tag, multiple, true); + removeRecord(nacked_delivery_tag, multiple, true); }); }); } -void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish) +void WriteBufferToRabbitMQProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) { auto record_iter = delivery_record.find(received_delivery_tag); @@ -292,7 +304,6 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable() && event_handler->connectionRunning()) || (!event_handler->connectionRunning() && setupConnection(true))) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 1ab90cb0b1d..a8e94070dbd 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -25,7 +25,6 @@ public: const String & exchange_name_, const AMQP::ExchangeType exchange_type_, const size_t channel_id_, - const String channel_base_, const bool persistent_, std::atomic & wait_confirm_, Poco::Logger * log_, @@ -46,7 +45,7 @@ private: void writingFunc(); bool setupConnection(bool reconnecting); void setupChannel(); - void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); + void removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish); void publish(ConcurrentBoundedQueue> & message, bool republishing); std::pair parsed_address; @@ -54,9 +53,12 @@ private: const Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - const String channel_id_base; - const String channel_base; + const String channel_id_base; /// Serial number of current producer buffer const bool persistent; + + /* false: when shutdown is called; needed because table might be dropped before all acks are received + * true: in all other cases + */ std::atomic & wait_confirm; AMQP::Table key_arguments; @@ -67,15 +69,48 @@ private: std::unique_ptr connection; std::unique_ptr producer_channel; - String channel_id; - ConcurrentBoundedQueue> payloads, returned; - UInt64 delivery_tag = 0; - std::atomic wait_all = true; - std::atomic wait_num = 0; - UInt64 payload_counter = 0; - std::map> delivery_record; + /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id UInt64 channel_id_counter = 0; + /// channel id which contains id of current producer buffer and serial number of recreated channel in this buffer + String channel_id; + + /* payloads.queue: + * - payloads are pushed to queue in countRow and poped by another thread in writingFunc, each payload gets into queue only once + * returned.queue: + * - payloads are pushed to queue: + * 1) inside channel->onError() callback if channel becomes unusable and the record of pending acknowledgements from server + * is non-empty. + * 2) inside removeRecord() if received nack() - negative acknowledgement from the server that message failed to be written + * to disk or it was unable to reach the queue. + * - payloads are poped from the queue once republished + */ + ConcurrentBoundedQueue> payloads, returned; + + /* Counter of current delivery on a current channel. Delivery tags are scoped per channel. The server attaches a delivery tag for each + * published message - a serial number of delivery on current channel. Delivery tag is a way of server to notify publisher if it was + * able or unable to process delivery, i.e. it sends back a responce with a corresponding delivery tag. + */ + UInt64 delivery_tag = 0; + + /* false: message delivery successfully ended: publisher received confirm from server that all published + * 1) persistent messages were written to disk + * 2) non-persistent messages reached the queue + * true: continue to process deliveries and returned messages + */ + bool wait_all = true; + + /* false: untill writeSuffix is called + * true: means payloads.queue will not grow anymore + */ + std::atomic wait_num = 0; + + /// Needed to fill messageID property + UInt64 payload_counter = 0; + + /// Record of pending acknowledgements from the server; its size never exceeds size of returned.queue + std::map> delivery_record; + Poco::Logger * log; const std::optional delim; const size_t max_rows; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 68f7bb506e6..9dbaddf33f4 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1547,91 +1547,6 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): assert TSV(result) == TSV(expected) -@pytest.mark.timeout(420) -def test_rabbitmq_no_loss_on_table_drop(rabbitmq_cluster): - instance.query(''' - CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'queue_resume', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_queue_base = 'queue_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq_queue_resume; - ''') - - i = [0] - messages_num = 10000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - for message in messages: - channel.basic_publish(exchange='queue_resume', routing_key='queue_resume', body=message, - properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - - threads = [] - threads_num = 20 - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while int(instance.query('SELECT count() FROM test.view')) == 0: - time.sleep(1) - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_queue_resume; - ''') - - for thread in threads: - thread.join() - - collected = int(instance.query('SELECT count() FROM test.view')) - - instance.query(''' - CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'queue_resume', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_queue_base = 'queue_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - ''') - - while True: - result1 = instance.query('SELECT count() FROM test.view') - time.sleep(1) - if int(result1) == messages_num * threads_num: - break - - instance.query(''' - DROP TABLE test.rabbitmq_queue_resume; - DROP TABLE test.consumer; - DROP TABLE test.view; - ''') - - assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - @pytest.mark.timeout(420) def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): instance.query(''' @@ -1856,6 +1771,85 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) +@pytest.mark.timeout(300) +def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'block', + rabbitmq_format = 'JSONEachRow', + rabbitmq_queue_base = 'block', + rabbitmq_max_block_size = 100, + rabbitmq_row_delimiter = '\\n'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.rabbitmq; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + cancel = threading.Event() + + i = [0] + def produce(): + while not cancel.is_set(): + messages = [] + for _ in range(101): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='block', routing_key='', body=message) + + rabbitmq_thread = threading.Thread(target=produce) + rabbitmq_thread.start() + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + cancel.set() + + instance.query(''' + DROP TABLE test.rabbitmq; + ''') + + while int(instance.query("SELECT count() FROM system.tables WHERE database='test' AND name='rabbitmq'")) == 1: + time.sleep(1) + + instance.query(''' + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'block', + rabbitmq_format = 'JSONEachRow', + rabbitmq_max_block_size = 100, + rabbitmq_queue_base = 'block', + rabbitmq_row_delimiter = '\\n'; + ''') + + while int(instance.query('SELECT uniqExact(key) FROM test.view')) < i[0]: + time.sleep(1) + + result = int(instance.query('SELECT count() == uniqExact(key) FROM test.view')) + + instance.query(''' + DROP TABLE test.consumer; + DROP TABLE test.view; + ''') + + rabbitmq_thread.join() + connection.close() + + assert result == 1, 'Messages from RabbitMQ get duplicated!' + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 7b0713b12d68c00f3f5593636041c08ea7ad76ce Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Sep 2020 07:59:32 +0000 Subject: [PATCH 102/535] Update docs --- .../table-engines/integrations/rabbitmq.md | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 41429016898..3e8b34f7f41 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -27,9 +27,15 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [rabbitmq_exchange_type = 'exchange_type',] [rabbitmq_routing_key_list = 'key1,key2,...',] [rabbitmq_row_delimiter = 'delimiter_symbol',] + [rabbitmq_schema = '',] [rabbitmq_num_consumers = N,] [rabbitmq_num_queues = N,] - [rabbitmq_transactional_channel = 0] + [rabbitmq_queue_base = 'queue',] + [rabbitmq_deadletter_exchange = 'dl-exchange',] + [rabbitmq_persistent = 0,] + [rabbitmq_skip_broken_messages = N,] + [rabbitmq_max_block_size = N,] + [rabbitmq_flush_interval_ms = N] ``` Required parameters: @@ -43,12 +49,15 @@ Optional parameters: - `rabbitmq_exchange_type` – The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent_hash`. Default: `fanout`. - `rabbitmq_routing_key_list` – A comma-separated list of routing keys. - `rabbitmq_row_delimiter` – Delimiter character, which ends the message. +- `rabbitmq_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. - `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. - `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. -- `rabbitmq_transactional_channel` – Wrap insert queries in transactions. Default: `0`. -- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. +- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. By default, queues are declared unique to tables based on db and table names. - `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified. - `persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`. +- `rabbitmq_skip_broken_messages` – RabbitMQ message parser tolerance to schema-incompatible messages per block. Default: `0`. If `rabbitmq_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data). +- `rabbitmq_max_block_size` +- `rabbitmq_flush_interval_ms` Required configuration: @@ -96,16 +105,18 @@ Exchange type options: - `consistent-hash` - Data is evenly distributed between all bound tables (where exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. Setting `rabbitmq_queue_base` may be used for the following cases: -- to be able to restore reading from certain durable queues when not all messages were successfully consumed. Note: it makes sence only if messages are sent with delivery mode 2 - marked 'persistent', durable. To be able to resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables. +- to let different tables share queues, so that multiple consumers could be registered for the same queues, which makes a better performance. If using `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings, the exact match of queues is achieved in case these parameters are the same. +- to be able to restore reading from certain durable queues when not all messages were successfully consumed. To be able to resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables. Note: it makes sence only if messages are sent with delivery mode 2 - marked 'persistent', durable. - to reuse queues as they are declared durable and not auto-deleted. -- to let different tables share queues, so that multiple consumers could be registered for the same queues, which makes better performance. If using `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings, the exact match of queues is achieved in case these parameters are the same. + +To improve performance, received messages are grouped into blocks the size of [max\_insert\_block\_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). If the block wasn’t formed within [stream\_flush\_interval\_ms](../../../operations/server-configuration-parameters/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block. If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then: - `rabbitmq-consistent-hash-exchange` plugin must be enabled. - `message_id` property of the published messages must be specified (unique for each message/batch). -For insert query there is message metadata, which is added for each published message: messageID and republished flag - can be accessed via message headers. +For insert query there is message metadata, which is added for each published message: `messageID` and `republished` flag (true, if published more than once) - can be accessed via message headers. Do not use the same table for inserts and materialized views. @@ -134,6 +145,7 @@ Example: ## Virtual Columns {#virtual-columns} - `_exchange_name` - RabbitMQ exchange name. -- `_consumer_tag` - ConsumerTag of the consumer that received the message. -- `_delivery_tag` - DeliveryTag if the message. Scoped per consumer. +- `_channel_id` - ChannelID, on which consumer, who received the message, was declared. +- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel. - `_redelivered` - Redelivered flag of the message. +- `_message_id` - messageID of the received message; non-empty if was set, when message was published. From 602535396d0ac58c1885ef1d1d3e7c085335f059 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 16:36:27 +0800 Subject: [PATCH 103/535] Refactor, move function declarations --- src/Formats/FormatFactory.cpp | 100 +++++++++++++++++++++++++++++----- src/Formats/FormatFactory.h | 69 ----------------------- 2 files changed, 85 insertions(+), 84 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 935d31d6541..871098e00c0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -323,13 +323,85 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm target = std::move(file_segmentation_engine); } +/// File Segmentation Engines for parallel reading + +void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); +void registerFileSegmentationEngineCSV(FormatFactory & factory); +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); +void registerFileSegmentationEngineRegexp(FormatFactory & factory); +void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); + +/// Formats for both input/output. + +void registerInputFormatNative(FormatFactory & factory); +void registerOutputFormatNative(FormatFactory & factory); + +void registerInputFormatProcessorNative(FormatFactory & factory); +void registerOutputFormatProcessorNative(FormatFactory & factory); +void registerInputFormatProcessorRowBinary(FormatFactory & factory); +void registerOutputFormatProcessorRowBinary(FormatFactory & factory); +void registerInputFormatProcessorTabSeparated(FormatFactory & factory); +void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); +void registerInputFormatProcessorValues(FormatFactory & factory); +void registerOutputFormatProcessorValues(FormatFactory & factory); +void registerInputFormatProcessorCSV(FormatFactory & factory); +void registerOutputFormatProcessorCSV(FormatFactory & factory); +void registerInputFormatProcessorTSKV(FormatFactory & factory); +void registerOutputFormatProcessorTSKV(FormatFactory & factory); +void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProcessorProtobuf(FormatFactory & factory); +void registerOutputFormatProcessorProtobuf(FormatFactory & factory); +void registerInputFormatProcessorTemplate(FormatFactory & factory); +void registerOutputFormatProcessorTemplate(FormatFactory & factory); +void registerInputFormatProcessorMsgPack(FormatFactory & factory); +void registerOutputFormatProcessorMsgPack(FormatFactory & factory); +void registerInputFormatProcessorORC(FormatFactory & factory); +void registerOutputFormatProcessorORC(FormatFactory & factory); +void registerInputFormatProcessorParquet(FormatFactory & factory); +void registerOutputFormatProcessorParquet(FormatFactory & factory); +void registerInputFormatProcessorArrow(FormatFactory & factory); +void registerOutputFormatProcessorArrow(FormatFactory & factory); +void registerInputFormatProcessorAvro(FormatFactory & factory); +void registerOutputFormatProcessorAvro(FormatFactory & factory); + +/// Output only (presentational) formats. + +void registerOutputFormatNull(FormatFactory & factory); + +void registerOutputFormatProcessorPretty(FormatFactory & factory); +void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); +void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); +void registerOutputFormatProcessorVertical(FormatFactory & factory); +void registerOutputFormatProcessorJSON(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatProcessorXML(FormatFactory & factory); +void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); +void registerOutputFormatProcessorNull(FormatFactory & factory); +void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); +void registerOutputFormatProcessorMarkdown(FormatFactory & factory); +void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); + +/// Input only formats. + +void registerInputFormatProcessorRegexp(FormatFactory & factory); +void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorCapnProto(FormatFactory & factory); + FormatFactory::FormatFactory() { + registerFileSegmentationEngineTabSeparated(*this); + registerFileSegmentationEngineCSV(*this); + registerFileSegmentationEngineJSONEachRow(*this); + registerFileSegmentationEngineRegexp(*this); + registerFileSegmentationEngineJSONAsString(*this); + registerInputFormatNative(*this); registerOutputFormatNative(*this); - registerOutputFormatProcessorJSONEachRowWithProgress(*this); - registerInputFormatProcessorNative(*this); registerOutputFormatProcessorNative(*this); registerInputFormatProcessorRowBinary(*this); @@ -348,8 +420,11 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONCompactEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); + registerInputFormatProcessorTemplate(*this); + registerOutputFormatProcessorTemplate(*this); + registerInputFormatProcessorMsgPack(*this); + registerOutputFormatProcessorMsgPack(*this); #if !defined(ARCADIA_BUILD) - registerInputFormatProcessorCapnProto(*this); registerInputFormatProcessorORC(*this); registerOutputFormatProcessorORC(*this); registerInputFormatProcessorParquet(*this); @@ -359,18 +434,6 @@ FormatFactory::FormatFactory() registerInputFormatProcessorAvro(*this); registerOutputFormatProcessorAvro(*this); #endif - registerInputFormatProcessorTemplate(*this); - registerOutputFormatProcessorTemplate(*this); - registerInputFormatProcessorRegexp(*this); - registerInputFormatProcessorMsgPack(*this); - registerOutputFormatProcessorMsgPack(*this); - registerInputFormatProcessorJSONAsString(*this); - - registerFileSegmentationEngineTabSeparated(*this); - registerFileSegmentationEngineCSV(*this); - registerFileSegmentationEngineJSONEachRow(*this); - registerFileSegmentationEngineRegexp(*this); - registerFileSegmentationEngineJSONAsString(*this); registerOutputFormatNull(*this); @@ -380,12 +443,19 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorVertical(*this); registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); + registerOutputFormatProcessorJSONEachRowWithProgress(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); registerOutputFormatProcessorMySQLWire(*this); registerOutputFormatProcessorMarkdown(*this); registerOutputFormatProcessorPostgreSQLWire(*this); + + registerInputFormatProcessorRegexp(*this); + registerInputFormatProcessorJSONAsString(*this); +#if !defined(ARCADIA_BUILD) + registerInputFormatProcessorCapnProto(*this); +#endif } FormatFactory & FormatFactory::instance() diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index f0d2b7826a0..54bff1eefc6 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -141,73 +141,4 @@ private: const Creators & getCreators(const String & name) const; }; -/// Formats for both input/output. - -void registerInputFormatNative(FormatFactory & factory); -void registerOutputFormatNative(FormatFactory & factory); - -void registerInputFormatProcessorNative(FormatFactory & factory); -void registerOutputFormatProcessorNative(FormatFactory & factory); -void registerInputFormatProcessorRowBinary(FormatFactory & factory); -void registerOutputFormatProcessorRowBinary(FormatFactory & factory); -void registerInputFormatProcessorTabSeparated(FormatFactory & factory); -void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); -void registerInputFormatProcessorValues(FormatFactory & factory); -void registerOutputFormatProcessorValues(FormatFactory & factory); -void registerInputFormatProcessorCSV(FormatFactory & factory); -void registerOutputFormatProcessorCSV(FormatFactory & factory); -void registerInputFormatProcessorTSKV(FormatFactory & factory); -void registerOutputFormatProcessorTSKV(FormatFactory & factory); -void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorParquet(FormatFactory & factory); -void registerOutputFormatProcessorParquet(FormatFactory & factory); -void registerInputFormatProcessorArrow(FormatFactory & factory); -void registerOutputFormatProcessorArrow(FormatFactory & factory); -void registerInputFormatProcessorProtobuf(FormatFactory & factory); -void registerOutputFormatProcessorProtobuf(FormatFactory & factory); -void registerInputFormatProcessorAvro(FormatFactory & factory); -void registerOutputFormatProcessorAvro(FormatFactory & factory); -void registerInputFormatProcessorTemplate(FormatFactory & factory); -void registerOutputFormatProcessorTemplate(FormatFactory & factory); -void registerInputFormatProcessorMsgPack(FormatFactory & factory); -void registerOutputFormatProcessorMsgPack(FormatFactory & factory); -void registerInputFormatProcessorORC(FormatFactory & factory); -void registerOutputFormatProcessorORC(FormatFactory & factory); - - -/// File Segmentation Engines for parallel reading - -void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); -void registerFileSegmentationEngineCSV(FormatFactory & factory); -void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); -void registerFileSegmentationEngineRegexp(FormatFactory & factory); -void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); - -/// Output only (presentational) formats. - -void registerOutputFormatNull(FormatFactory & factory); - -void registerOutputFormatProcessorPretty(FormatFactory & factory); -void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); -void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); -void registerOutputFormatProcessorPrettyASCII(FormatFactory & factory); -void registerOutputFormatProcessorVertical(FormatFactory & factory); -void registerOutputFormatProcessorJSON(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorXML(FormatFactory & factory); -void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); -void registerOutputFormatProcessorNull(FormatFactory & factory); -void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); -void registerOutputFormatProcessorMarkdown(FormatFactory & factory); -void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); - -/// Input only formats. -void registerInputFormatProcessorCapnProto(FormatFactory & factory); -void registerInputFormatProcessorRegexp(FormatFactory & factory); -void registerInputFormatProcessorJSONAsString(FormatFactory & factory); - } From 86fa185bb6fbf8e1e6bc6044a7f4e523477e84db Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:06:28 +0800 Subject: [PATCH 104/535] Add JSONStrings formats --- src/Formats/FormatFactory.cpp | 6 + .../Impl/JSONStringsEachRowRowInputFormat.cpp | 245 ++++++++++++++++++ .../Impl/JSONStringsEachRowRowInputFormat.h | 54 ++++ .../JSONStringsEachRowRowOutputFormat.cpp | 117 +++++++++ .../Impl/JSONStringsEachRowRowOutputFormat.h | 45 ++++ .../Impl/JSONStringsRowOutputFormat.cpp | 93 +++++++ .../Formats/Impl/JSONStringsRowOutputFormat.h | 43 +++ src/Processors/ya.make | 3 + 8 files changed, 606 insertions(+) create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h create mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 871098e00c0..cb378fbea96 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -352,6 +352,8 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); @@ -378,6 +380,7 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatProcessorJSONStrings(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); void registerOutputFormatProcessorNull(FormatFactory & factory); @@ -418,6 +421,8 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONEachRow(*this); registerInputFormatProcessorJSONCompactEachRow(*this); registerOutputFormatProcessorJSONCompactEachRow(*this); + registerInputFormatProcessorJSONStringsEachRow(*this); + registerOutputFormatProcessorJSONStringsEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorTemplate(*this); @@ -444,6 +449,7 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); registerOutputFormatProcessorJSONEachRowWithProgress(*this); + registerOutputFormatProcessorJSONStrings(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp new file mode 100644 index 00000000000..fff44a204fb --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp @@ -0,0 +1,245 @@ +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int CANNOT_READ_ALL_DATA; +} + + +JSONStringsEachRowRowInputFormat::JSONStringsEachRowRowInputFormat(ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool with_names_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) +{ + const auto & sample = getPort().getHeader(); + size_t num_columns = sample.columns(); + + data_types.resize(num_columns); + column_indexes_by_names.reserve(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + const auto & column_info = sample.getByPosition(i); + + data_types[i] = column_info.type; + column_indexes_by_names.emplace(column_info.name, i); + } +} + +void JSONStringsEachRowRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + column_indexes_for_input_fields.clear(); + not_seen_columns.clear(); +} + +void JSONStringsEachRowRowInputFormat::readPrefix() +{ + /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. + skipBOMIfExists(in); + + if (with_names) + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, false); + + assertChar('[', in); + do + { + skipWhitespaceIfAny(in); + String column_name; + readJSONString(column_name, in); + addInputColumn(column_name); + skipWhitespaceIfAny(in); + } + while (checkChar(',', in)); + assertChar(']', in); + skipEndOfLine(); + + /// Type checking + assertChar('[', in); + for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) + { + skipWhitespaceIfAny(in); + String data_type; + readJSONString(data_type, in); + + if (column_indexes_for_input_fields[i] && + data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) + { + throw Exception( + "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name + + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + + ", not " + data_type, + ErrorCodes::INCORRECT_DATA + ); + } + + if (i != column_indexes_for_input_fields.size() - 1) + assertChar(',', in); + skipWhitespaceIfAny(in); + } + assertChar(']', in); + } + else + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, true); + column_indexes_for_input_fields.resize(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + column_indexes_for_input_fields[i] = i; + } + } + + for (size_t i = 0; i < read_columns.size(); ++i) + { + if (!read_columns[i]) + { + not_seen_columns.emplace_back(i); + } + } +} + +void JSONStringsEachRowRowInputFormat::addInputColumn(const String & column_name) +{ + names_of_columns.emplace_back(column_name); + + const auto column_it = column_indexes_by_names.find(column_name); + if (column_it == column_indexes_by_names.end()) + { + if (format_settings.skip_unknown_fields) + { + column_indexes_for_input_fields.push_back(std::nullopt); + return; + } + + throw Exception( + "Unknown field found in JSONStringsEachRow header: '" + column_name + "' " + + "at position " + std::to_string(column_indexes_for_input_fields.size()) + + "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", + ErrorCodes::INCORRECT_DATA + ); + } + + const auto column_index = column_it->second; + + if (read_columns[column_index]) + throw Exception("Duplicate field found while parsing JSONStringsEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); + + read_columns[column_index] = true; + column_indexes_for_input_fields.emplace_back(column_index); +} + +bool JSONStringsEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) +{ + skipEndOfLine(); + + if (in.eof()) + return false; + + size_t num_columns = columns.size(); + + read_columns.assign(num_columns, false); + + assertChar('[', in); + for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) + { + const auto & table_column = column_indexes_for_input_fields[file_column]; + if (table_column) + { + readField(*table_column, columns); + } + else + { + skipJSONField(in, StringRef(names_of_columns[file_column])); + } + + skipWhitespaceIfAny(in); + if (in.eof()) + throw Exception("Unexpected end of stream while parsing JSONStringsEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); + if (file_column + 1 != column_indexes_for_input_fields.size()) + { + assertChar(',', in); + skipWhitespaceIfAny(in); + } + } + assertChar(']', in); + + for (const auto & name : not_seen_columns) + columns[name]->insertDefault(); + + ext.read_columns = read_columns; + return true; +} + +void JSONStringsEachRowRowInputFormat::skipEndOfLine() +{ + skipWhitespaceIfAny(in); + if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) + ++in.position(); + + skipWhitespaceIfAny(in); +} + +void JSONStringsEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) +{ + try + { + read_columns[index] = true; + const auto & type = data_types[index]; + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } + catch (Exception & e) + { + e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); + throw; + } +} + +void JSONStringsEachRowRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(in); +} + +void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("JSONStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, false); + }); + + factory.registerInputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true); + }); +} + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h new file mode 100644 index 00000000000..ec0a0f7bad9 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h @@ -0,0 +1,54 @@ +#pragma once + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/** A stream for reading data in JSONStringsEachRow and JSONStringsEachRowWithNamesAndTypes formats +*/ +class JSONStringsEachRowRowInputFormat : public IRowInputFormat +{ +public: + JSONStringsEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); + + String getName() const override { return "JSONStringsEachRowRowInputFormat"; } + + + void readPrefix() override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + void resetParser() override; + +private: + void addInputColumn(const String & column_name); + void skipEndOfLine(); + void readField(size_t index, MutableColumns & columns); + + const FormatSettings format_settings; + + using IndexesMap = std::unordered_map; + IndexesMap column_indexes_by_names; + + using OptionalIndexes = std::vector>; + OptionalIndexes column_indexes_for_input_fields; + + DataTypes data_types; + std::vector read_columns; + std::vector not_seen_columns; + + /// This is for the correct exceptions in skipping unknown fields. + std::vector names_of_columns; + + bool with_names; +}; + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp new file mode 100644 index 00000000000..75007ea236e --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp @@ -0,0 +1,117 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +JSONStringsEachRowRowOutputFormat::JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool with_names_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) +{ + const auto & sample = getPort(PortKind::Main).getHeader(); + NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); +} + + +void JSONStringsEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); +} + + +void JSONStringsEachRowRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", out); +} + + +void JSONStringsEachRowRowOutputFormat::writeRowStartDelimiter() +{ + writeChar('[', out); +} + + +void JSONStringsEachRowRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("]\n", out); +} + +void JSONStringsEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + writeChar('\n', out); + size_t num_columns = columns.size(); + writeChar('[', out); + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + JSONStringsEachRowRowOutputFormat::writeFieldDelimiter(); + + JSONStringsEachRowRowOutputFormat::writeField(*columns[i], *types[i], row_num); + } + writeCString("]\n", out); +} + +void JSONStringsEachRowRowOutputFormat::writePrefix() +{ + if (with_names) + { + writeChar('[', out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeChar('\"', out); + writeString(fields[i].name, out); + writeChar('\"', out); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n[", out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeJSONString(fields[i].type->getName(), out, settings); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n", out); + } +} + +void JSONStringsEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) +{ + if (with_names) + IRowOutputFormat::consumeTotals(std::move(chunk)); +} + +void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( + WriteBuffer &buf, + const Block &sample, + FormatFactory::WriteCallback callback, + const FormatSettings &format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); + }); +} + + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h new file mode 100644 index 00000000000..1d43a333da1 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** The stream for outputting data in JSON format, by object per line. + * Does not validate UTF-8. + */ +class JSONStringsEachRowRowOutputFormat : public IRowOutputFormat +{ +public: + JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); + + String getName() const override { return "JSONStringsEachRowRowOutputFormat"; } + + void writePrefix() override; + + void writeBeforeTotals() override {} + void writeTotals(const Columns & columns, size_t row_num) override; + void writeAfterTotals() override {} + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + +protected: + void consumeTotals(Chunk) override; + /// No extremes. + void consumeExtremes(Chunk) override {} + +private: + FormatSettings settings; + + NamesAndTypes fields; + + bool with_names; +}; +} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp new file mode 100644 index 00000000000..6ccb315f73f --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp @@ -0,0 +1,93 @@ +#include +#include + +#include + + +namespace DB +{ + +JSONStringsRowOutputFormat::JSONStringsRowOutputFormat( + WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) + : JSONRowOutputFormat(out_, header, callback, settings_) +{ +} + + +void JSONStringsRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + ++field_number; +} + + +void JSONStringsRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", *ostr); +} + +void JSONStringsRowOutputFormat::writeTotalsFieldDelimiter() +{ + writeCString(",", *ostr); +} + + +void JSONStringsRowOutputFormat::writeRowStartDelimiter() +{ + writeCString("\t\t[", *ostr); +} + + +void JSONStringsRowOutputFormat::writeRowEndDelimiter() +{ + writeChar(']', *ostr); + field_number = 0; + ++row_count; +} + +void JSONStringsRowOutputFormat::writeBeforeTotals() +{ + writeCString(",\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"totals\": [", *ostr); +} + +void JSONStringsRowOutputFormat::writeAfterTotals() +{ + writeChar(']', *ostr); +} + +void JSONStringsRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) +{ + writeCString("\t\t\"", *ostr); + writeCString(title, *ostr); + writeCString("\": [", *ostr); + + size_t extremes_columns = columns.size(); + for (size_t i = 0; i < extremes_columns; ++i) + { + if (i != 0) + writeTotalsFieldDelimiter(); + + writeField(*columns[i], *types[i], row_num); + } + + writeChar(']', *ostr); +} + +void registerOutputFormatProcessorJSONStrings(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings); + }); +} + +} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h new file mode 100644 index 00000000000..b221bc9ee36 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +struct FormatSettings; + +/** The stream for outputting data in the JSONStrings format. + */ +class JSONStringsRowOutputFormat : public JSONRowOutputFormat +{ +public: + JSONStringsRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + + String getName() const override { return "JSONStringsRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + + void writeBeforeTotals() override; + void writeAfterTotals() override; + +protected: + void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; + + void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override + { + return writeField(column, type, row_num); + } + + void writeTotalsFieldDelimiter() override; + +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 30de38fedbd..27893674859 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -31,6 +31,9 @@ SRCS( Formats/Impl/JSONEachRowRowOutputFormat.cpp Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/JSONStringsEachRowRowInputFormat.cpp + Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp + Formats/Impl/JSONStringsRowOutputFormat.cpp Formats/Impl/MarkdownRowOutputFormat.cpp Formats/Impl/MsgPackRowInputFormat.cpp Formats/Impl/MsgPackRowOutputFormat.cpp From c25a99aaf58108651149930db5ef86e1313120c4 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:47:37 +0800 Subject: [PATCH 105/535] Add tests --- .../01446_JSONStringsEachRow.reference | 47 ++++++++++++++ .../0_stateless/01446_JSONStringsEachRow.sql | 63 +++++++++++++++++++ .../0_stateless/01447_JSONStrings.reference | 43 +++++++++++++ .../queries/0_stateless/01447_JSONStrings.sql | 8 +++ 4 files changed, 161 insertions(+) create mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.reference create mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.sql create mode 100644 tests/queries/0_stateless/01447_JSONStrings.reference create mode 100644 tests/queries/0_stateless/01447_JSONStrings.sql diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.reference b/tests/queries/0_stateless/01446_JSONStringsEachRow.reference new file mode 100644 index 00000000000..0b05f050b29 --- /dev/null +++ b/tests/queries/0_stateless/01446_JSONStringsEachRow.reference @@ -0,0 +1,47 @@ +1 +["1", "a"] +["2", "b"] +["3", "c"] +2 +["a", "1"] +["b", "1"] +["c", "1"] +3 +["value", "name"] +["UInt8", "String"] +["1", "a"] +["2", "b"] +["3", "c"] +4 +["name", "c"] +["String", "UInt64"] +["a", "1"] +["b", "1"] +["c", "1"] + +["", "3"] +5 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +6 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +7 +["16", "[15,16,17]", "['first','second','third']"] +8 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +9 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +10 +["first", "1", "16", "8"] +["second", "2", "32", "8"] +11 +["v1", "v2", "v3", "v4"] +["String", "UInt8", "UInt16", "UInt8"] +["", "2", "3", "1"] +12 +["v1", "n.id", "n.name"] +["UInt8", "Array(UInt8)", "Array(String)"] +["16", "[15,16,17]", "['first','second','third']"] diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql new file mode 100644 index 00000000000..f461b217fe4 --- /dev/null +++ b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; +SELECT 3; +/* Check JSONStringsEachRowWithNamesAndTypes Output */ +SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; +SELECT 4; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRowWithNamesAndTypes; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table_2; +SELECT 8; +/* Check JSONStringsEachRowWithNamesAndTypes Output */ +SET input_format_null_as_default = 0; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 9; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 10; +/* Check Header */ +TRUNCATE TABLE test_table; +SET input_format_skip_unknown_fields = 1; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 11; +TRUNCATE TABLE test_table; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] +SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; +SELECT 12; +/* Check Nested */ +INSERT INTO test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01447_JSONStrings.reference new file mode 100644 index 00000000000..58af593dc77 --- /dev/null +++ b/tests/queries/0_stateless/01447_JSONStrings.reference @@ -0,0 +1,43 @@ +{ + "meta": + [ + { + "name": "1", + "type": "UInt8" + }, + { + "name": "'a'", + "type": "String" + }, + { + "name": "[1, 2, 3]", + "type": "Array(UInt8)" + }, + { + "name": "tuple(1, 'a')", + "type": "Tuple(UInt8, String)" + }, + { + "name": "NULL", + "type": "Nullable(Nothing)" + }, + { + "name": "nan", + "type": "Float64" + } + ], + + "data": + [ + ["1", "a", "[1,2,3]", "(1,'a')", "ᴺᵁᴸᴸ", "nan"] + ], + + "rows": 1, + + "statistics": + { + "elapsed": 0.00068988, + "rows_read": 1, + "bytes_read": 1 + } +} diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_JSONStrings.sql new file mode 100644 index 00000000000..7d89f0f5087 --- /dev/null +++ b/tests/queries/0_stateless/01447_JSONStrings.sql @@ -0,0 +1,8 @@ +SELECT + 1, + 'a', + [1, 2, 3], + (1, 'a'), + null, + nan +FORMAT JSONStrings; From 6d37c9d2a68ca3f6ae39c9bc0bb99424d7fc236e Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:48:03 +0800 Subject: [PATCH 106/535] Update docs about formats --- docs/en/interfaces/formats.md | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 9d3965b4a9c..9c7c2dda8dc 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -26,7 +26,10 @@ The supported formats are: | [VerticalRaw](#verticalraw) | ✗ | ✔ | | [JSON](#json) | ✗ | ✔ | | [JSONCompact](#jsoncompact) | ✗ | ✔ | +| [JSONStrings](#jsonstrings) | ✗ | ✔ | | [JSONEachRow](#jsoneachrow) | ✔ | ✔ | +| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | +| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ | | [TSKV](#tskv) | ✔ | ✔ | | [Pretty](#pretty) | ✗ | ✔ | | [PrettyCompact](#prettycompact) | ✗ | ✔ | @@ -470,7 +473,7 @@ See also the [JSONEachRow](#jsoneachrow) format. ## JSONCompact {#jsoncompact} -Differs from JSON only in that data rows are output in arrays, not in objects. +Differs from JSON only in that data rows are output in arrays of any element type, not in objects. Example: @@ -514,17 +517,26 @@ Example: This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). See also the `JSONEachRow` format. -## JSONEachRow {#jsoneachrow} +## JSONStrings {#jsonstrings} -When using this format, ClickHouse outputs rows as separated, newline-delimited JSON objects, but the data as a whole is not valid JSON. +Differs from JSON and JSONCompact only in that data rows are output in arrays of strings. + +This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). +See also the `JSONEachRow` format. + +## JSONEachRow {#jsoneachrow} +## JSONCompactEachRow {#jsoncompacteachrow} +## JSONStringsEachRow {#jsonstringseachrow} + +When using these formats, ClickHouse outputs rows as separated, newline-delimited JSON values, but the data as a whole is not valid JSON. ``` json -{"SearchPhrase":"curtain designs","count()":"1064"} -{"SearchPhrase":"baku","count()":"1000"} -{"SearchPhrase":"","count()":"8267016"} +{"some_int":42,"some_str":"hello","some_tuple":[1,"a"]} // JSONEachRow +[42,"hello",[1,"a"]] // JSONCompactEachRow +["42","hello","(2,'a')"] // JSONStringsEachRow ``` -When inserting the data, you should provide a separate JSON object for each row. +When inserting the data, you should provide a separate JSON value for each row. ### Inserting Data {#inserting-data} From babd3beec09054d1dc4b1b8a35cf30da013f05af Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:48:19 +0800 Subject: [PATCH 107/535] Fix nullable data parsing --- src/DataTypes/DataTypeNullable.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 847047850fd..3318196b951 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -308,7 +308,10 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer const DataTypePtr & nested_data_type) { return safeDeserialize(column, *nested_data_type, - [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, + [&istr] + { + return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); + }, [&nested_data_type, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextQuoted(nested, istr, settings); }); } @@ -316,7 +319,11 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { safeDeserialize(column, *nested_data_type, - [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, + [&istr] + { + return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr) + || checkStringByFirstCharacterAndAssertTheRest("ᴺᵁᴸᴸ", istr); + }, [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); } From 8fac595428606115066647bc2f3b8a394931e16b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Sep 2020 13:29:10 +0300 Subject: [PATCH 108/535] Stop query execution if exception happened in PipelineExecutor itself. --- src/Processors/Executors/PipelineExecutor.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index cacd8fced8d..d445177f28e 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -469,7 +469,16 @@ void PipelineExecutor::wakeUpExecutor(size_t thread_num) void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads) { - executeStepImpl(thread_num, num_threads); + try + { + executeStepImpl(thread_num, num_threads); + } + catch (...) + { + /// In case of exception from executor itself, stop other threads. + finish(); + throw; + } #ifndef NDEBUG auto & context = executor_contexts[thread_num]; From f0dc5a30853ff4b40d0097f07068fbb0f49eb714 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Sep 2020 13:49:53 +0300 Subject: [PATCH 109/535] First working test --- src/DataStreams/TTLBlockInputStream.cpp | 28 +++++++++++++ src/DataStreams/TTLBlockInputStream.h | 2 + src/Interpreters/MutationsInterpreter.cpp | 8 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 41 +++++++++++++------ .../MergeTree/MergeTreeDataWriter.cpp | 7 +++- .../MergeTree/registerStorageMergeTree.cpp | 3 ++ src/Storages/StorageInMemoryMetadata.cpp | 5 ++- src/Storages/TTLDescription.cpp | 11 +++++ .../01465_ttl_recompression.reference | 10 +++++ .../0_stateless/01465_ttl_recompression.sql | 32 +++++++++++++++ 11 files changed, 142 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01465_ttl_recompression.reference create mode 100644 tests/queries/0_stateless/01465_ttl_recompression.sql diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 6d80e784c03..e1586286678 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -134,6 +134,7 @@ Block TTLBlockInputStream::readImpl() removeValuesWithExpiredColumnTTL(block); updateMovesTTL(block); + updateRecompressionTTL(block); return block; } @@ -395,6 +396,33 @@ void TTLBlockInputStream::updateMovesTTL(Block & block) block.erase(column); } + +void TTLBlockInputStream::updateRecompressionTTL(Block & block) +{ + std::vector columns_to_remove; + for (const auto & ttl_entry : metadata_snapshot->getRecompressionTTLs()) + { + auto & new_ttl_info = new_ttl_infos.recompression_ttl[ttl_entry.result_column]; + + if (!block.has(ttl_entry.result_column)) + { + columns_to_remove.push_back(ttl_entry.result_column); + ttl_entry.expression->execute(block); + } + + const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get(); + + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + new_ttl_info.update(cur_ttl); + } + } + + for (const String & column : columns_to_remove) + block.erase(column); +} + UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t ind) { if (const ColumnUInt16 * column_date = typeid_cast(column)) diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 3f37f35426c..18670021ec9 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -78,6 +78,8 @@ private: /// Updates TTL for moves void updateMovesTTL(Block & block); + void updateRecompressionTTL(Block & block); + UInt32 getTimestampByIndex(const IColumn * column, size_t ind); bool isTTLExpired(time_t ttl) const; }; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 9d35b339d94..3a397cb9b5a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -533,8 +533,16 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) /// Special step to recalculate affected indices and TTL expressions. stages.emplace_back(context); for (const auto & column : unchanged_columns) + { + std::cerr << "ADDING UNCHANGED COLUMN TO STAGE:" << column << std::endl; stages.back().column_to_updated.emplace( column, std::make_shared(column)); + std::cerr << "OUTPUT COLUMNS:" << stages.back().output_columns.size() << std::endl; + for (const auto & col : stages.back().output_columns) + { + std::cerr << "OUTPUT COLUMN:" << col << std::endl; + } + } } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b721cf4afbf..536d72d327a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3064,8 +3064,10 @@ CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_c auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + std::cerr << "RECOMPRESSION ENTRIES SIZE:" << recompression_ttl_entries.size() << std::endl; for (auto ttl_entry_it = recompression_ttl_entries.begin(); ttl_entry_it != recompression_ttl_entries.end(); ++ttl_entry_it) { + std::cerr << "RECOMPRESSION TTL SIZE:" << ttl_infos.recompression_ttl.size() << std::endl; auto ttl_info_it = ttl_infos.recompression_ttl.find(ttl_entry_it->result_column); /// Prefer TTL rule which went into action last. if (ttl_info_it != ttl_infos.recompression_ttl.end() @@ -3078,7 +3080,15 @@ CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_c } if (max_max_ttl) + { + std::cerr << "BEST ENTRY FOUND, MAX MAX:" << max_max_ttl << std::endl; + std::cerr << "RECOMPRESSION IS NULLPTR:" << (best_entry_it->recompression_codec == nullptr) << std::endl; return CompressionCodecFactory::instance().get(best_entry_it->recompression_codec, {}); + } + else + { + std::cerr << "NOT FOUND NEW RECOMPRESSION\n"; + } return global_context.chooseCompressionCodec( part_size_compressed, diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8cece66dafb..9a77115e777 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -659,9 +659,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// (which is locked in shared mode when input streams are created) and when inserting new data /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. - auto compression_codec = data.global_context.chooseCompressionCodec( - merge_entry->total_size_bytes_compressed, - static_cast (merge_entry->total_size_bytes_compressed) / data.getTotalActiveSizeInBytes()); + auto compression_codec = data.getCompressionCodecForPart(merge_entry->total_size_bytes_compressed, new_data_part->ttl_infos, time_of_merge); /// TODO: Should it go through IDisk interface? String rows_sources_file_path; @@ -1082,15 +1080,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto disk = new_data_part->volume->getDisk(); String new_part_tmp_path = new_data_part->getFullRelativePath(); - /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex - /// (which is locked in data.getTotalActiveSizeInBytes()) - /// (which is locked in shared mode when input streams are created) and when inserting new data - /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus - /// deadlock is impossible. - auto compression_codec = context.chooseCompressionCodec( - source_part->getBytesOnDisk(), - static_cast(source_part->getBytesOnDisk()) / data.getTotalActiveSizeInBytes()); - disk->createDirectories(new_part_tmp_path); /// Don't change granularity type while mutating subset of columns @@ -1100,11 +1089,27 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor bool need_remove_expired_values = false; if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) + { + std::cerr << "GOING TO MATERIALIZE TTL\n"; need_remove_expired_values = true; + } + else + { + std::cerr << "NOT GOING TO MATERIALIZE TTL\n"; + std::cerr << "IN IS NULL:" << (in == nullptr) << std::endl; + } /// All columns from part are changed and may be some more that were missing before in part if (!isWidePart(source_part) || (interpreter && interpreter->isAffectingAllColumns())) { + std::cerr << "MUTATING ALL PART COLUMNS\n"; + /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex + /// (which is locked in data.getTotalActiveSizeInBytes()) + /// (which is locked in shared mode when input streams are created) and when inserting new data + /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus + /// deadlock is impossible. + auto compression_codec = data.getCompressionCodecForPart(source_part->getBytesOnDisk(), source_part->ttl_infos, time_of_mutation); + auto part_indices = getIndicesForNewDataPart(metadata_snapshot->getSecondaryIndices(), for_file_renames); mutateAllPartColumns( new_data_part, @@ -1121,6 +1126,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } else /// TODO: check that we modify only non-key columns in this case. { + + std::cerr << "MUTATING SOME PART COLUMNS\n"; /// We will modify only some of the columns. Other columns and key values can be copied as-is. auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context); @@ -1128,7 +1135,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension); if (need_remove_expired_values) + { files_to_skip.insert("ttl.txt"); + } + for (const auto & name : files_to_skip) + { + std::cerr << "SKIPPING " << name << std::endl; + } /// Create hardlinks for unchanged files for (auto it = disk->iterateDirectory(source_part->getFullRelativePath()); it->isValid(); it->next()) @@ -1157,8 +1170,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor new_data_part->checksums = source_part->checksums; + auto compression_codec = source_part->default_codec; + if (in) { + std::cerr << "HEADER:" << updated_header.dumpStructure() << std::endl; + std::cerr << "IN HEADER:" << in->getHeader().dumpStructure() << std::endl; mutateSomePartColumns( source_part, metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 92bf5345d5a..5115666066a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -13,6 +13,7 @@ #include #include +#include namespace ProfileEvents { @@ -234,8 +235,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); + time_t current_time = time(nullptr); NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); + ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, current_time); VolumePtr volume = data.getStoragePolicy()->getVolume(0); auto new_data_part = data.createPart( @@ -306,7 +308,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. - auto compression_codec = data.global_context.chooseCompressionCodec(0, 0); + auto compression_codec = data.getCompressionCodecForPart(0, new_data_part->ttl_infos, current_time); + std::cerr << "SELECTED CODEC:" << queryToString(compression_codec->getCodecDesc()) << std::endl; const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 4526b0d4f9b..8706c1f3b37 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -558,8 +558,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, args.context); if (args.storage_def->ttl_table) + { + std::cerr << "Parsing table ttl in description\n"; metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( args.storage_def->ttl_table->ptr(), metadata.columns, args.context, metadata.primary_key); + } if (args.query.columns_list && args.query.columns_list->indices) for (auto & index : args.query.columns_list->indices->children) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index f611c1ec95d..f410fa34f59 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -124,7 +124,7 @@ TTLTableDescription StorageInMemoryMetadata::getTableTTLs() const bool StorageInMemoryMetadata::hasAnyTableTTL() const { - return hasAnyMoveTTL() || hasRowsTTL(); + return hasAnyMoveTTL() || hasRowsTTL() || hasAnyRecompressionTTL(); } TTLColumnsDescription StorageInMemoryMetadata::getColumnTTLs() const @@ -207,6 +207,9 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet } } + 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)) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 656baf39971..ca5ea714dd9 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -89,6 +89,7 @@ TTLDescription::TTLDescription(const TTLDescription & other) , aggregate_descriptions(other.aggregate_descriptions) , destination_type(other.destination_type) , destination_name(other.destination_name) + , recompression_codec(other.recompression_codec) { if (other.expression) expression = std::make_shared(*other.expression); @@ -125,6 +126,12 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) aggregate_descriptions = other.aggregate_descriptions; destination_type = other.destination_type; destination_name = other.destination_name; + + if (other.recompression_codec) + recompression_codec = other.recompression_codec->clone(); + else + recompression_codec.reset(); + return * this; } @@ -266,6 +273,7 @@ TTLDescription TTLDescription::getTTLFromAST( } else if (ttl_element->mode == TTLMode::RECOMPRESS) { + std::cerr << "GOT INTO RECOMPRESS\n"; result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( ttl_element->recompression_codec, {}, !context.getSettingsRef().allow_suspicious_codecs); @@ -283,6 +291,7 @@ TTLTableDescription::TTLTableDescription(const TTLTableDescription & other) : definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr) , rows_ttl(other.rows_ttl) , move_ttl(other.move_ttl) + , recompression_ttl(other.recompression_ttl) { } @@ -298,6 +307,7 @@ TTLTableDescription & TTLTableDescription::operator=(const TTLTableDescription & rows_ttl = other.rows_ttl; move_ttl = other.move_ttl; + recompression_ttl = other.recompression_ttl; return *this; } @@ -327,6 +337,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( } else if (ttl.mode == TTLMode::RECOMPRESS) { + std::cerr << "GOT RECOMPRESSIOn TTL\n"; result.recompression_ttl.emplace_back(std::move(ttl)); } else diff --git a/tests/queries/0_stateless/01465_ttl_recompression.reference b/tests/queries/0_stateless/01465_ttl_recompression.reference new file mode 100644 index 00000000000..2f1a2ea40b1 --- /dev/null +++ b/tests/queries/0_stateless/01465_ttl_recompression.reference @@ -0,0 +1,10 @@ +3000 +1_1_1_0 LZ4 +2_2_2_0 ZSTD(17) +3_3_3_0 LZ4HC(10) +1_1_1_0_4 LZ4 +2_2_2_0_4 ZSTD(17) +3_3_3_0_4 LZ4HC(10) +1_1_1_1_4 LZ4 +2_2_2_1_4 ZSTD(12) +3_3_3_1_4 ZSTD(12) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.sql b/tests/queries/0_stateless/01465_ttl_recompression.sql new file mode 100644 index 00000000000..0c72000c624 --- /dev/null +++ b/tests/queries/0_stateless/01465_ttl_recompression.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS recompression_table; + +CREATE TABLE recompression_table +( + dt DateTime, + key UInt64, + value String + +) ENGINE MergeTree() +ORDER BY tuple() +PARTITION BY key +TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)); + +INSERT INTO recompression_table SELECT now(), 1, toString(number) from numbers(1000); + +INSERT INTO recompression_table SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000); + +INSERT INTO recompression_table SELECT now() - INTERVAL 2 YEAR, 3, toString(number) from numbers(2000, 1000); + +SELECT COUNT() FROM recompression_table; + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; + +ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; + +OPTIMIZE TABLE recompression_table FINAL; + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; + +DROP TABLE IF EXISTS recompression_table; From c37a19f7b095a8f05ab1e9b65c6181e0e1fb6605 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Sep 2020 14:23:38 +0300 Subject: [PATCH 110/535] Better --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 9 --------- src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ .../0_stateless/01465_ttl_recompression.reference | 2 ++ tests/queries/0_stateless/01465_ttl_recompression.sql | 8 ++++++++ 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9a77115e777..f46fb7a79ef 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1126,8 +1126,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } else /// TODO: check that we modify only non-key columns in this case. { - - std::cerr << "MUTATING SOME PART COLUMNS\n"; /// We will modify only some of the columns. Other columns and key values can be copied as-is. auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context); @@ -1138,11 +1136,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { files_to_skip.insert("ttl.txt"); } - for (const auto & name : files_to_skip) - { - std::cerr << "SKIPPING " << name << std::endl; - } - /// Create hardlinks for unchanged files for (auto it = disk->iterateDirectory(source_part->getFullRelativePath()); it->isValid(); it->next()) { @@ -1174,8 +1167,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (in) { - std::cerr << "HEADER:" << updated_header.dumpStructure() << std::endl; - std::cerr << "IN HEADER:" << in->getHeader().dumpStructure() << std::endl; mutateSomePartColumns( source_part, metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 085c441aa90..6ac262ed35a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,8 +33,10 @@ struct Settings; M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_recompressions_in_queue, 1, "How many tasks of recompressiong parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_execute_ttl_recompression, 10, "When there is less than specified number of free entries in pool, do not execute part recompression according to TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ diff --git a/tests/queries/0_stateless/01465_ttl_recompression.reference b/tests/queries/0_stateless/01465_ttl_recompression.reference index 2f1a2ea40b1..c03c003d5b8 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.reference +++ b/tests/queries/0_stateless/01465_ttl_recompression.reference @@ -1,7 +1,9 @@ +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(17)), dt + toIntervalYear(1) RECOMPRESS CODEC(LZ4HC(10))\nSETTINGS index_granularity = 8192 3000 1_1_1_0 LZ4 2_2_2_0 ZSTD(17) 3_3_3_0 LZ4HC(10) +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(12))\nSETTINGS index_granularity = 8192 1_1_1_0_4 LZ4 2_2_2_0_4 ZSTD(17) 3_3_3_0_4 LZ4HC(10) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.sql b/tests/queries/0_stateless/01465_ttl_recompression.sql index 0c72000c624..92233f2d5cb 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.sql +++ b/tests/queries/0_stateless/01465_ttl_recompression.sql @@ -11,6 +11,10 @@ ORDER BY tuple() PARTITION BY key TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)); +SHOW CREATE TABLE recompression_table; + +SYSTEM STOP TTL MERGES recompression_table; + INSERT INTO recompression_table SELECT now(), 1, toString(number) from numbers(1000); INSERT INTO recompression_table SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000); @@ -23,8 +27,12 @@ SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompre ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; +SHOW CREATE TABLE recompression_table; + SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; +SYSTEM START TTL MERGES recompression_table; + OPTIMIZE TABLE recompression_table FINAL; SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; From f82b799f12baeb1047388cac0d1abc5a0b684c2f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Sep 2020 16:53:11 +0300 Subject: [PATCH 111/535] Update CreatingSetsTransform. --- src/Interpreters/SubqueryForSet.cpp | 10 +- src/Interpreters/SubqueryForSet.h | 5 +- .../Transforms/CreatingSetsTransform.cpp | 170 +++++++----------- .../Transforms/CreatingSetsTransform.h | 26 +-- 4 files changed, 83 insertions(+), 128 deletions(-) diff --git a/src/Interpreters/SubqueryForSet.cpp b/src/Interpreters/SubqueryForSet.cpp index ac5c1e3d9eb..038ecbbb0b6 100644 --- a/src/Interpreters/SubqueryForSet.cpp +++ b/src/Interpreters/SubqueryForSet.cpp @@ -12,10 +12,9 @@ void SubqueryForSet::makeSource(std::shared_ptr NamesWithAliases && joined_block_aliases_) { joined_block_aliases = std::move(joined_block_aliases_); - source = std::make_shared(interpreter->getSampleBlock(), - [interpreter]() mutable { return interpreter->execute().getInputStream(); }); + source = QueryPipeline::getPipe(interpreter->execute().pipeline); - sample_block = source->getHeader(); + sample_block = source.getHeader(); renameColumns(sample_block); } @@ -50,11 +49,10 @@ bool SubqueryForSet::insertJoinedBlock(Block & block) return join->addJoinedBlock(block); } -void SubqueryForSet::setTotals() +void SubqueryForSet::setTotals(Block totals) { - if (join && source) + if (join) { - Block totals = source->getTotals(); renameColumns(totals); join->setTotals(totals); } diff --git a/src/Interpreters/SubqueryForSet.h b/src/Interpreters/SubqueryForSet.h index 4c99e34a2fc..d268758c3e8 100644 --- a/src/Interpreters/SubqueryForSet.h +++ b/src/Interpreters/SubqueryForSet.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -18,7 +19,7 @@ using ExpressionActionsPtr = std::shared_ptr; struct SubqueryForSet { /// The source is obtained using the InterpreterSelectQuery subquery. - BlockInputStreamPtr source; + Pipe source; /// If set, build it from result. SetPtr set; @@ -37,7 +38,7 @@ struct SubqueryForSet void setJoinActions(ExpressionActionsPtr actions); bool insertJoinedBlock(Block & block); - void setTotals(); + void setTotals(Block totals); private: NamesWithAliases joined_block_aliases; /// Rename column from joined block from this list. diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 65cded62fde..1b308fd9a8b 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -1,6 +1,5 @@ #include -#include #include #include @@ -22,39 +21,56 @@ namespace ErrorCodes CreatingSetsTransform::CreatingSetsTransform( + Block in_header_, Block out_header_, - SubqueriesForSets subqueries_for_sets_, + SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, const Context & context_) - : IProcessor({}, {std::move(out_header_)}) - , subqueries_for_sets(std::move(subqueries_for_sets_)) - , cur_subquery(subqueries_for_sets.begin()) + : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) + , subquery(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , context(context_) { } -IProcessor::Status CreatingSetsTransform::prepare() +void CreatingSetsTransform::addTotalsPort() { - auto & output = outputs.front(); + if (inputs.size() > 1) + throw Exception("Totals port was already added to CreatingSetsTransform", ErrorCodes::LOGICAL_ERROR); - if (finished) - { - output.finish(); - return Status::Finished; - } - - /// Check can output. - if (output.isFinished()) - return Status::Finished; - - if (!output.canPush()) - return Status::PortFull; - - return Status::Ready; + inputs.emplace_back(getInputPort().getHeader(), this); } -void CreatingSetsTransform::startSubquery(SubqueryForSet & subquery) +IProcessor::Status CreatingSetsTransform::prepare() +{ + auto status = IAccumulatingTransform::prepare(); + if (status == IProcessor::Status::Finished && inputs.size() > 1) + { + auto & totals_input = inputs.back(); + if (totals_input.isFinished()) + return IProcessor::Status::Finished; + + totals_input.setNeeded(); + if (!totals_input.hasData()) + return IProcessor::Status::NeedData; + + auto totals = totals_input.pull(); + subquery.setTotals(getInputPort().getHeader().cloneWithColumns(totals.detachColumns())); + totals_input.close(); + } + + return status; +} + +void CreatingSetsTransform::work() +{ + if (!is_initialized) + init(); + + IAccumulatingTransform::work(); +} + +void CreatingSetsTransform::startSubquery() { if (subquery.set) LOG_TRACE(log, "Creating set."); @@ -63,8 +79,6 @@ void CreatingSetsTransform::startSubquery(SubqueryForSet & subquery) if (subquery.table) LOG_TRACE(log, "Filling temporary table."); - elapsed_nanoseconds = 0; - if (subquery.table) table_out = subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), context); @@ -79,25 +93,18 @@ void CreatingSetsTransform::startSubquery(SubqueryForSet & subquery) table_out->writePrefix(); } -void CreatingSetsTransform::finishSubquery(SubqueryForSet & subquery) +void CreatingSetsTransform::finishSubquery() { - size_t head_rows = 0; - const BlockStreamProfileInfo & profile_info = subquery.source->getProfileInfo(); - - head_rows = profile_info.rows; - - subquery.setTotals(); - - if (head_rows != 0) + if (read_rows != 0) { - auto seconds = elapsed_nanoseconds / 1e9; + auto seconds = watch.elapsedNanoseconds() / 1e9; if (subquery.set) - LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), head_rows, seconds); + LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), read_rows, seconds); if (subquery.join) - LOG_DEBUG(log, "Created Join with {} entries from {} rows in {} sec.", subquery.join->getTotalRowCount(), head_rows, seconds); + LOG_DEBUG(log, "Created Join with {} entries from {} rows in {} sec.", subquery.join->getTotalRowCount(), read_rows, seconds); if (subquery.table) - LOG_DEBUG(log, "Created Table with {} rows in {} sec.", head_rows, seconds); + LOG_DEBUG(log, "Created Table with {} rows in {} sec.", read_rows, seconds); } else { @@ -109,64 +116,17 @@ void CreatingSetsTransform::init() { is_initialized = true; - for (auto & elem : subqueries_for_sets) - if (elem.second.source && elem.second.set) - elem.second.set->setHeader(elem.second.source->getHeader()); + if (subquery.set) + subquery.set->setHeader(getInputPort().getHeader()); + + watch.restart(); + startSubquery(); } -void CreatingSetsTransform::work() +void CreatingSetsTransform::consume(Chunk chunk) { - if (!is_initialized) - init(); - - Stopwatch watch; - - while (cur_subquery != subqueries_for_sets.end() && cur_subquery->second.source == nullptr) - ++cur_subquery; - - if (cur_subquery == subqueries_for_sets.end()) - { - finished = true; - return; - } - - SubqueryForSet & subquery = cur_subquery->second; - - if (!started_cur_subquery) - { - startSubquery(subquery); - started_cur_subquery = true; - } - - auto finish_current_subquery = [&]() - { - if (subquery.set) - subquery.set->finishInsert(); - - if (table_out) - table_out->writeSuffix(); - - watch.stop(); - elapsed_nanoseconds += watch.elapsedNanoseconds(); - - finishSubquery(subquery); - - ++cur_subquery; - started_cur_subquery = false; - - while (cur_subquery != subqueries_for_sets.end() && cur_subquery->second.source == nullptr) - ++cur_subquery; - - if (cur_subquery == subqueries_for_sets.end()) - finished = true; - }; - - auto block = subquery.source->read(); - if (!block) - { - finish_current_subquery(); - return; - } + read_rows += chunk.getNumRows(); + auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); if (!done_with_set) { @@ -194,26 +154,20 @@ void CreatingSetsTransform::work() } if (done_with_set && done_with_join && done_with_table) - { - subquery.source->cancel(false); - finish_current_subquery(); - } - else - elapsed_nanoseconds += watch.elapsedNanoseconds(); + finishConsume(); } -void CreatingSetsTransform::setProgressCallback(const ProgressCallback & callback) +Chunk CreatingSetsTransform::generate() { - for (auto & elem : subqueries_for_sets) - if (elem.second.source) - elem.second.source->setProgressCallback(callback); -} + if (subquery.set) + subquery.set->finishInsert(); -void CreatingSetsTransform::setProcessListElement(QueryStatus * status) -{ - for (auto & elem : subqueries_for_sets) - if (elem.second.source) - elem.second.source->setProcessListElement(status); + if (table_out) + table_out->writeSuffix(); + + finishSubquery(); + finished = true; + return {}; } } diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index ac9ac7130f3..d31bef2438f 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include #include @@ -16,32 +16,34 @@ using ProgressCallback = std::function; /// Don't return any data. Sets are created when Finish status is returned. /// In general, several work() methods need to be called to finish. /// TODO: several independent processors can be created for each subquery. Make subquery a piece of pipeline. -class CreatingSetsTransform : public IProcessor +class CreatingSetsTransform : public IAccumulatingTransform { public: CreatingSetsTransform( + Block in_header_, Block out_header_, - SubqueriesForSets subqueries_for_sets_, + SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, const Context & context_); String getName() const override { return "CreatingSetsTransform"; } + Status prepare() override; void work() override; + void consume(Chunk chunk) override; + Chunk generate() override; - void setProgressCallback(const ProgressCallback & callback); - void setProcessListElement(QueryStatus * status); + void addTotalsPort(); protected: bool finished = false; private: - SubqueriesForSets subqueries_for_sets; - SubqueriesForSets::iterator cur_subquery; + SubqueryForSet subquery; - bool started_cur_subquery = false; BlockOutputStreamPtr table_out; - UInt64 elapsed_nanoseconds = 0; + UInt64 read_rows = 0; + Stopwatch watch; bool done_with_set = true; bool done_with_join = true; @@ -54,13 +56,13 @@ private: size_t bytes_to_transfer = 0; using Logger = Poco::Logger; - Poco::Logger * log = &Poco::Logger::get("CreatingSetsBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("CreatingSetsTransform"); bool is_initialized = false; void init(); - void startSubquery(SubqueryForSet & subquery); - void finishSubquery(SubqueryForSet & subquery); + void startSubquery(); + void finishSubquery(); }; } From 4e58f003053ba82053e61e620e6758014aa826d8 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 1 Sep 2020 16:57:13 +0300 Subject: [PATCH 112/535] Update docker/test/fuzzer/run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index a319033a232..0ac4859a1e2 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -35,7 +35,7 @@ function download # wget -nv -O- -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/performance/performance.tgz" \ # | tar --strip-components=1 -zxv - wget -nv -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-10_debug_none_bundled_unsplitted_disable_False_binary/clickhouse" + wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-10_debug_none_bundled_unsplitted_disable_False_binary/clickhouse" chmod +x clickhouse ln -s ./clickhouse ./clickhouse-server ln -s ./clickhouse ./clickhouse-client @@ -176,4 +176,3 @@ case "$stage" in exit $task_exit_code ;& esac - From 4620ac4c0d4d839aaf0554f6db94ea0ac24c214e Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 1 Sep 2020 16:57:20 +0300 Subject: [PATCH 113/535] Update docker/test/fuzzer/run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 0ac4859a1e2..66d3e840c4f 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -32,7 +32,7 @@ function clone function download { -# wget -nv -O- -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/performance/performance.tgz" \ +# wget -O- -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/performance/performance.tgz" \ # | tar --strip-components=1 -zxv wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-10_debug_none_bundled_unsplitted_disable_False_binary/clickhouse" From 34a2beab7c54cd5d726aa78f4efbbc8825f2ee20 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Tue, 1 Sep 2020 18:03:43 +0300 Subject: [PATCH 114/535] Edited EN description --- docs/en/engines/table-engines/integrations/rabbitmq.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 7d09c6f72a5..7fe99ca3678 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -7,7 +7,7 @@ toc_title: RabbitMQ This engine allows integrating ClickHouse with [RabbitMQ](https://www.rabbitmq.com). -RabbitMQ lets you: +`RabbitMQ` lets you: - Publish or subscribe to data flows. - Process streams as they become available. @@ -44,7 +44,7 @@ Optional parameters: - `rabbitmq_routing_key_list` – A comma-separated list of routing keys. - `rabbitmq_row_delimiter` – Delimiter character, which ends the message. - `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. -- `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. Single queue can contain up to 50K messages at the same time. +- `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. A single queue can contain up to 50K messages at the same time. - `rabbitmq_transactional_channel` – Wrap insert queries in transactions. Default: `0`. Required configuration: @@ -86,13 +86,13 @@ There can be no more than one exchange per table. One exchange can be shared bet Exchange type options: -- `direct` - Routing is based on exact matching of keys. Example table key list: `key1,key2,key3,key4,key5`, message key can eqaul any of them. +- `direct` - Routing is based on the exact matching of keys. Example table key list: `key1,key2,key3,key4,key5`, message key can equal any of them. - `fanout` - Routing to all tables (where exchange name is the same) regardless of the keys. - `topic` - Routing is based on patterns with dot-separated keys. Examples: `*.logs`, `records.*.*.2020`, `*.2018,*.2019,*.2020`. - `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`. -- `consistent-hash` - Data is evenly distributed between all bound tables (where exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. +- `consistent-hash` - Data is evenly distributed between all bound tables (where the exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. -If exchange type is not specified, then default is `fanout` and routing keys for data publishing must be randomized in range `[1, num_consumers]` for every message/batch (or in range `[1, num_consumers * num_queues]` if `rabbitmq_num_queues` is set). This table configuration works quicker then any other, especially when `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are set. +If exchange type is not specified, then default is `fanout` and routing keys for data publishing must be randomized in range `[1, num_consumers]` for every message/batch (or in range `[1, num_consumers * num_queues]` if `rabbitmq_num_queues` is set). This table configuration works quicker than any other, especially when `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are set. If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are specified along with `rabbitmq_exchange_type`, then: From 0b70abe54235b53cd1c909f56562cf32791eb344 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 1 Sep 2020 18:51:46 +0300 Subject: [PATCH 115/535] Don't let the fuzzer change max_execution_time --- docker/test/fuzzer/query-fuzzer-tweaks-users.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml index 8d430aa5c54..356d3212932 100644 --- a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml +++ b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml @@ -2,6 +2,15 @@ 10 + + + + 10 + + From 3cadc9033ae63d7faa851b1707b3c6f9ce1a36aa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 18:26:49 +0300 Subject: [PATCH 116/535] fsyncs for metadata files of part --- .../MergeTree/IMergeTreeDataPartWriter.h | 2 +- .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 13 +++++++-- .../MergeTree/MergedBlockOutputStream.h | 3 +- utils/durability-test/create_sync.sql | 1 + utils/durability-test/durability-test.sh | 28 ++++++++++--------- utils/durability-test/insert_sync.sql | 1 + 7 files changed, 32 insertions(+), 18 deletions(-) create mode 100644 utils/durability-test/create_sync.sql mode change 100644 => 100755 utils/durability-test/durability-test.sh create mode 100644 utils/durability-test/insert_sync.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 4d3602e732e..4a42a58a65b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -52,7 +52,7 @@ public: virtual void initPrimaryIndex() {} virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0; - virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} + virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} Columns releaseIndexColumns(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index dbe41144573..8295b881d87 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -332,7 +332,7 @@ void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization( checksums.files["primary.idx"].file_size = index_stream->count(); checksums.files["primary.idx"].file_hash = index_stream->getHash(); if (sync) - index_stream->sync(); + index_file_stream->sync(); index_stream = nullptr; } } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index fdef5d69688..bdc6bade259 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -111,7 +111,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( part_columns = *total_columns_list; if (new_part->isStoredOnDisk()) - finalizePartOnDisk(new_part, part_columns, checksums); + finalizePartOnDisk(new_part, part_columns, checksums, sync); new_part->setColumns(part_columns); new_part->rows_count = rows_count; @@ -126,7 +126,8 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( void MergedBlockOutputStream::finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, + bool sync) { if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { @@ -143,6 +144,8 @@ void MergedBlockOutputStream::finalizePartOnDisk( count_out_hashing.next(); checksums.files["count.txt"].file_size = count_out_hashing.count(); checksums.files["count.txt"].file_hash = count_out_hashing.getHash(); + if (sync) + count_out->sync(); } if (!new_part->ttl_infos.empty()) @@ -153,6 +156,8 @@ void MergedBlockOutputStream::finalizePartOnDisk( new_part->ttl_infos.write(out_hashing); checksums.files["ttl.txt"].file_size = out_hashing.count(); checksums.files["ttl.txt"].file_hash = out_hashing.getHash(); + if (sync) + out->sync(); } removeEmptyColumnsFromPart(new_part, part_columns, checksums); @@ -161,12 +166,16 @@ void MergedBlockOutputStream::finalizePartOnDisk( /// Write a file with a description of columns. auto out = volume->getDisk()->writeFile(part_path + "columns.txt", 4096); part_columns.writeText(*out); + if (sync) + out->sync(); } { /// Write file with checksums. auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096); checksums.write(*out); + if (sync) + out->sync(); } } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 0b500b93f01..87ff9dd1ded 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -59,7 +59,8 @@ private: void finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, - MergeTreeData::DataPart::Checksums & checksums); + MergeTreeData::DataPart::Checksums & checksums, + bool sync); private: NamesAndTypesList columns_list; diff --git a/utils/durability-test/create_sync.sql b/utils/durability-test/create_sync.sql new file mode 100644 index 00000000000..2cc88d2c943 --- /dev/null +++ b/utils/durability-test/create_sync.sql @@ -0,0 +1 @@ +CREATE TABLE test_sync (a Int, s String) ENGINE = MergeTree ORDER BY a SETTINGS fsync_after_insert = 1, min_compressed_bytes_to_fsync_after_merge = 1; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh old mode 100644 new mode 100755 index 1f47c900f49..c7f8936ec95 --- a/utils/durability-test/durability-test.sh +++ b/utils/durability-test/durability-test.sh @@ -17,12 +17,12 @@ fi function run() { - sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" + sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" 2>/dev/null } function copy() { - sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 + sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 2>/dev/null } function wait_vm_for_start() @@ -50,8 +50,8 @@ function wait_clickhouse_for_start() { echo "Waiting until ClickHouse started..." started=0 - for i in {0..15}; do - run "clickhouse client --query 'select 1'" + for i in {0..30}; do + run "clickhouse client --query 'select 1'" > /dev/null if [ $? -eq 0 ]; then started=1 break @@ -70,7 +70,7 @@ echo "Downloading image" curl -O $URL/$IMAGE qemu-img resize $IMAGE +10G -virt-customize -a $IMAGE --root-password password:$PASSWORD +virt-customize -a $IMAGE --root-password password:$PASSWORD > /dev/null 2>&1 virt-copy-in -a $IMAGE sshd_config /etc/ssh echo "Starting VM" @@ -93,8 +93,8 @@ if [[ -z $CLICKHOUSE_CONFIG_DIR ]]; then CLICKHOUSE_CONFIG_DIR=/etc/clickhouse-server fi -echo "Using ClickHouse binary: " $CLICKHOUSE_BINARY -echo "Using ClickHouse config from: " $CLICKHOUSE_CONFIG_DIR +echo "Using ClickHouse binary:" $CLICKHOUSE_BINARY +echo "Using ClickHouse config from:" $CLICKHOUSE_CONFIG_DIR copy $CLICKHOUSE_BINARY /usr/bin copy $CLICKHOUSE_CONFIG_DIR /etc @@ -104,23 +104,19 @@ echo "Prepared VM" echo "Starting ClickHouse" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & - wait_clickhouse_for_start -echo "Started ClickHouse" - query=`cat $CREATE_QUERY` echo "Executing query:" $query run "clickhouse client --query '$query'" query=`cat $INSERT_QUERY` echo "Will run in a loop query: " $query -run "clickhouse benchmark <<< '$query'" & +run "clickhouse benchmark <<< '$query' -c 8" & echo "Running queries" pid=`pidof qemu-system-x86_64` -sec=$(( (RANDOM % 3) + 25 )) - +sec=$(( (RANDOM % 5) + 25 )) ms=$(( RANDOM % 1000 )) echo "Will kill VM in $sec.$ms sec" @@ -130,6 +126,8 @@ kill -9 $pid echo "Restarting" +sleep 5s + ./startup.exp > qemu.log 2>&1 & wait_vm_for_start @@ -137,10 +135,12 @@ run "rm -r *data/system" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & wait_clickhouse_for_start +pid=`pidof qemu-system-x86_64` result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'"` if [[ -n $result ]]; then echo "FAIL. Can't attach table:" echo $result + kill -9 $pid exit 1 fi @@ -148,7 +148,9 @@ result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remov if [[ -n $result ]]; then echo "FAIL. Have broken parts:" echo $result + kill -9 $pid exit 1 fi +kill -9 $pid echo OK diff --git a/utils/durability-test/insert_sync.sql b/utils/durability-test/insert_sync.sql new file mode 100644 index 00000000000..a1ad2ff4ea5 --- /dev/null +++ b/utils/durability-test/insert_sync.sql @@ -0,0 +1 @@ +INSERT INTO test_sync SELECT number, toString(number) FROM numbers(10) From 120962b61a98ef1cafc043c51304070e727cde28 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Tue, 1 Sep 2020 05:09:48 -0700 Subject: [PATCH 117/535] fix tests --- .../0_stateless/01463_test_alter_live_view_refresh.reference | 1 + tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01463_test_alter_live_view_refresh.reference diff --git a/tests/queries/0_stateless/01463_test_alter_live_view_refresh.reference b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.reference new file mode 100644 index 00000000000..4d98c7b6838 --- /dev/null +++ b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.reference @@ -0,0 +1 @@ +ALTER LIVE VIEW live1 REFRESH diff --git a/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql index 36e8c9a9785..ab316a377fd 100644 --- a/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql +++ b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql @@ -6,4 +6,5 @@ SET allow_experimental_live_view=1; CREATE LIVE VIEW live1 AS SELECT * FROM test0; +select 'ALTER LIVE VIEW live1 REFRESH'; ALTER LIVE VIEW live1 REFRESH; -- success From d646ca1d0cd3f0f22226cd40e291625061f70d8e Mon Sep 17 00:00:00 2001 From: Dao Minh Thuc Date: Tue, 1 Sep 2020 23:07:26 +0700 Subject: [PATCH 118/535] Disable -fchar8_t for capnproto only --- contrib/capnproto-cmake/CMakeLists.txt | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt index e5d62c59327..b655ad3e5d9 100644 --- a/contrib/capnproto-cmake/CMakeLists.txt +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -29,10 +29,6 @@ set (KJ_SRCS ${CAPNPROTO_SOURCE_DIR}/kj/parse/char.c++ ) -if (CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-char8_t") -endif () - add_library(kj ${KJ_SRCS}) target_include_directories(kj SYSTEM PUBLIC ${CAPNPROTO_SOURCE_DIR}) @@ -82,8 +78,9 @@ if (COMPILER_GCC) -Wno-deprecated-declarations -Wno-class-memaccess) elseif (COMPILER_CLANG) set (SUPPRESS_WARNINGS -Wno-non-virtual-dtor -Wno-sign-compare -Wno-strict-aliasing -Wno-deprecated-declarations) + set (CAPNP_PRIVATE_CXX_FLAGS -fno-char8_t) endif () -target_compile_options(kj PRIVATE ${SUPPRESS_WARNINGS}) -target_compile_options(capnp PRIVATE ${SUPPRESS_WARNINGS}) -target_compile_options(capnpc PRIVATE ${SUPPRESS_WARNINGS}) +target_compile_options(kj PRIVATE ${SUPPRESS_WARNINGS} ${CAPNP_PRIVATE_CXX_FLAGS}) +target_compile_options(capnp PRIVATE ${SUPPRESS_WARNINGS} ${CAPNP_PRIVATE_CXX_FLAGS}) +target_compile_options(capnpc PRIVATE ${SUPPRESS_WARNINGS} ${CAPNP_PRIVATE_CXX_FLAGS}) From b67fde2b0415194978ab989e53c1443676e6a4e5 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 1 Sep 2020 19:20:42 +0300 Subject: [PATCH 119/535] DOCSUP-2031: Update by PR#1130 Added description of the partial_merge_join_optimizations and partial_merge_join_rows_in_right_blocks settings. --- docs/ru/operations/settings/settings.md | 29 +++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index ab64fb757f1..a6c868876ed 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -406,6 +406,35 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( - 0 — пустые ячейки заполняются значением по умолчанию соответствующего типа поля. - 1 — `JOIN` ведёт себя как в стандартном SQL. Тип соответствующего поля преобразуется в [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable), а пустые ячейки заполняются значениями [NULL](../../sql-reference/syntax.md). +## partial_merge_join_optimizations {#partial_merge_join_optimizations} + +Отключает все оптимизации для запросов [JOIN](../../sql-reference/statements/select/join.md) с частичным MergeJoin алгоритмом. + +По умолчанию оптимизации включены, что может привести к неправильным результатам. Если вы видите подозрительные результаты в своих запросах, отключите оптимизацию с помощью этого параметра. В различных версиях сервера ClickHouse, оптимизация может отличаться. + +Возможные значения: + +- 0 — Оптимизация отключена. +- 1 — Оптимизация включена. + +Значение по умолчанию: 1. + +## partial_merge_join_rows_in_right_blocks {#partial_merge_join_rows_in_right_blocks} + +Устанавливает предельные размеры блоков данных «правого» соединения, для запросов [JOIN](../../sql-reference/statements/select/join.md) с частичным MergeJoin алгоритмом. + +Сервер ClickHouse: + +1. Разделяет данные правого соединения на блоки с заданным числом строк. +2. Индексирует для каждого блока минимальное и максимальное значение. +3. Выгружает подготовленные блоки на диск, если это возможно. + +Возможные значения: + +- Положительное целое число. Рекомендуемый диапазон значений [1000, 100000]. + +Значение по умолчанию: 65536. + ## join_on_disk_max_files_to_merge {#join_on_disk_max_files_to_merge} Устанавливет количество файлов, разрешенных для параллельной сортировки, при выполнении операций MergeJoin на диске. From ac5877e601714450a369062abbf80f84485bc6f5 Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 00:58:39 +0800 Subject: [PATCH 120/535] Fix tests --- tests/queries/0_stateless/01447_JSONStrings.reference | 9 +-------- tests/queries/0_stateless/01447_JSONStrings.sql | 2 ++ 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01447_JSONStrings.reference index 58af593dc77..1c6f073c0d0 100644 --- a/tests/queries/0_stateless/01447_JSONStrings.reference +++ b/tests/queries/0_stateless/01447_JSONStrings.reference @@ -32,12 +32,5 @@ ["1", "a", "[1,2,3]", "(1,'a')", "ᴺᵁᴸᴸ", "nan"] ], - "rows": 1, - - "statistics": - { - "elapsed": 0.00068988, - "rows_read": 1, - "bytes_read": 1 - } + "rows": 1 } diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_JSONStrings.sql index 7d89f0f5087..45fc4a56d7a 100644 --- a/tests/queries/0_stateless/01447_JSONStrings.sql +++ b/tests/queries/0_stateless/01447_JSONStrings.sql @@ -1,3 +1,5 @@ +SET output_format_write_statistics = 0; + SELECT 1, 'a', From 9591ae59cc538100c9aa4440738abf47183cab15 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 1 Sep 2020 20:46:40 +0300 Subject: [PATCH 121/535] DOCSUP-2031: Update by PR#11065 Disable ANY RIGHT and ANY FULL JOINs by default --- docs/ru/operations/settings/settings.md | 28 +++++++++++++++++++ .../sql-reference/statements/select/join.md | 4 ++- 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index a6c868876ed..6f5f7ccf965 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -458,6 +458,34 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Значение по умолчанию: LZ4. +## any_join_distinct_right_table_keys {#any_join_distinct_right_table_keys} + +Включает устаревшее поведение сервера ClickHouse при выполнении операций `ANY INNER|LEFT JOIN`. + +!!! note "Внимание" + Используйте этот параметр только в целях обратной совместимости, если ваши варианты использования требуют устаревшего поведения `JOIN`. + +Когда включено устаревшее поведение: + +- Результаты операций "t1 ANY LEFT JOIN t2" и "t2 ANY RIGHT JOIN t1" не равны, поскольку ClickHouse использует логику с сопоставлением ключей таблицы "многие к одному слева направо". +- Результаты операций `ANY INNER JOIN` содержат все строки из левой таблицы, аналогично операции `SEMI LEFT JOIN`. + +Когда устаревшее поведение отключено: + +- Результаты операций `t1 ANY LEFT JOIN t2` и `t2 ANY RIGHT JOIN t1` равно, потому что ClickHouse использует логику сопоставления ключей один-ко-многим в операциях `ANY RIGHT JOIN`. +- Результаты операций `ANY INNER JOIN` содержат по одной строке на ключ из левой и правой таблиц. + +Возможные значения: + +- 0 — Устаревшее поведение отключено. +- 1 — Устаревшее поведение включено. + +Значение по умолчанию: 0. + +См. также: + +- [JOIN strictness](../../sql-reference/statements/select/join.md#select-join-strictness) + ## max\_block\_size {#setting-max_block_size} Данные в ClickHouse обрабатываются по блокам (наборам кусочков столбцов). Внутренние циклы обработки для одного блока достаточно эффективны, но есть заметные издержки на каждый блок. Настройка `max_block_size` — это рекомендация, какой размер блока (в количестве строк) загружать из таблиц. Размер блока не должен быть слишком маленьким, чтобы затраты на каждый блок были заметны, но не слишком велики, чтобы запрос с LIMIT, который завершается после первого блока, обрабатывался быстро. Цель состоит в том, чтобы не использовалось слишком много оперативки при вынимании большого количества столбцов в несколько потоков; чтобы оставалась хоть какая-нибудь кэш-локальность. diff --git a/docs/ru/sql-reference/statements/select/join.md b/docs/ru/sql-reference/statements/select/join.md index 2a5bcff0cbb..800f07a7c66 100644 --- a/docs/ru/sql-reference/statements/select/join.md +++ b/docs/ru/sql-reference/statements/select/join.md @@ -36,7 +36,9 @@ FROM !!! note "Примечание" Значение строгости по умолчанию может быть переопределено с помощью настройки [join\_default\_strictness](../../../operations/settings/settings.md#settings-join_default_strictness). - + +Поведение сервера ClickHouse для операций `ANY JOIN` зависит от параметра [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys). + ### Использование ASOF JOIN {#asof-join-usage} `ASOF JOIN` применим в том случае, когда необходимо объединять записи, которые не имеют точного совпадения. From f93edc5defd1141b2292614eab105b6b4371d9a2 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Tue, 1 Sep 2020 21:59:27 +0300 Subject: [PATCH 122/535] Edit more text in EN version --- docs/en/engines/table-engines/integrations/rabbitmq.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 7fe99ca3678..1bf1c1d3754 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -45,7 +45,7 @@ Optional parameters: - `rabbitmq_row_delimiter` – Delimiter character, which ends the message. - `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. - `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. A single queue can contain up to 50K messages at the same time. -- `rabbitmq_transactional_channel` – Wrap insert queries in transactions. Default: `0`. +- `rabbitmq_transactional_channel` – Wrap `INSERT` queries in transactions. Default: `0`. Required configuration: @@ -72,7 +72,7 @@ Example: ## Description {#description} -`SELECT` is not particularly useful for reading messages (except for debugging), because each message can be read only once. It is more practical to create real-time threads using materialized views. To do this: +`SELECT` is not particularly useful for reading messages (except for debugging), because each message can be read only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this: 1. Use the engine to create a RabbitMQ consumer and consider it a data stream. 2. Create a table with the desired structure. From 1259ded322fd27ae43c59423fb88a7639edd77b9 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Tue, 1 Sep 2020 22:02:11 +0300 Subject: [PATCH 123/535] Add RU version --- .../table-engines/integrations/rabbitmq.md | 122 ++++++++++++++++++ 1 file changed, 122 insertions(+) create mode 100644 docs/ru/engines/table-engines/integrations/rabbitmq.md diff --git a/docs/ru/engines/table-engines/integrations/rabbitmq.md b/docs/ru/engines/table-engines/integrations/rabbitmq.md new file mode 100644 index 00000000000..b6b239f0eee --- /dev/null +++ b/docs/ru/engines/table-engines/integrations/rabbitmq.md @@ -0,0 +1,122 @@ +--- +toc_priority: 6 +toc_title: RabbitMQ +--- + +# RabbitMQ {#rabbitmq-engine} + +Движок работает с [RabbitMQ](https://www.rabbitmq.com). + +`RabbitMQ` позволяет: + +- Публиковать/подписываться на потоки данных. +- Обрабатывать потоки по мере их появления. + +## Создание таблицы {#table_engine-rabbitmq-creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = RabbitMQ SETTINGS + rabbitmq_host_port = 'host:port', + rabbitmq_exchange_name = 'exchange_name', + rabbitmq_format = 'data_format'[,] + [rabbitmq_exchange_type = 'exchange_type',] + [rabbitmq_routing_key_list = 'key1,key2,...',] + [rabbitmq_row_delimiter = 'delimiter_symbol',] + [rabbitmq_num_consumers = N,] + [rabbitmq_num_queues = N,] + [rabbitmq_transactional_channel = 0] +``` + +Обязательные параметры: + +- `rabbitmq_host_port` – адрес сервера (`хост:порт`). Например: `localhost:5672`. +- `rabbitmq_exchange_name` – имя точки обмена в RabbitMQ. +- `rabbitmq_format` – формат сообщения. Используется такое же обозначение, как и в функции `FORMAT` в SQL, например, `JSONEachRow`. Подробнее см. в разделе [Форматы входных и выходных данных](../../../interfaces/formats.md). + +Дополнительные параметры: + +- `rabbitmq_exchange_type` – тип точки обмена в RabbitMQ: `direct`, `fanout`, `topic`, `headers`, `consistent-hash`. По умолчанию: `fanout`. +- `rabbitmq_routing_key_list` – список ключей маршрутизации, через запятую. +- `rabbitmq_row_delimiter` – символ-разделитель, который завершает сообщение. +- `rabbitmq_num_consumers` – количество потребителей на таблицу. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одного потребителя недостаточна. +- `rabbitmq_num_queues` – количество очередей на потребителя. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одной очереди на потребителя недостаточна. Одна очередь поддерживает до 50 тысяч сообщений одновременно. +- `rabbitmq_transactional_channel` – обернутые запросы `INSERT` в транзакциях. По умолчанию: `0`. + +Требуемая конфигурация: + +Конфигурация сервера RabbitMQ добавляется с помощью конфигурационного файла ClickHouse. + +``` xml + + root + clickhouse + +``` + +Example: + +``` sql + CREATE TABLE queue ( + key UInt64, + value UInt64 + ) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672', + rabbitmq_exchange_name = 'exchange1', + rabbitmq_format = 'JSONEachRow', + rabbitmq_num_consumers = 5; +``` + +## Описание {#description} + +Запрос `SELECT` не очень полезен для чтения сообщений (за исключением отладки), поскольку каждое сообщение может быть прочитано только один раз. Практичнее создавать потоки реального времени с помощью [материализованных преставлений](../../../sql-reference/statements/create/view.md). Для этого: + +1. Создайте потребителя RabbitMQ с помощью движка и рассматривайте его как поток данных. +2. Создайте таблицу с необходимой структурой. +3. Создайте материализованное представление, которое преобразует данные от движка и помещает их в ранее созданную таблицу. + +Когда к движку присоединяется материализованное представление, оно начинает в фоновом режиме собирать данные. Это позволяет непрерывно получать сообщения от RabbitMQ и преобразовывать их в необходимый формат с помощью `SELECT`. +У одной таблицы RabbitMQ может быть неограниченное количество материализованных представлений. + +Данные передаются с помощью параметров `rabbitmq_exchange_type` и `rabbitmq_routing_key_list`. +Может быть не более одной точки обмена на таблицу. Одна точка обмена может использоваться несколькими таблицами: это позволяет выполнять маршрутизацию по нескольким таблицам одновременно. + +Параметры точек обмена: + +- `direct` - маршрутизация основана на точном совпадении ключей. Пример списка ключей: `key1,key2,key3,key4,key5`. Ключ сообщения может совпадать с одним из них. +- `fanout` - маршрутизация по всем таблицам, где имя точки обмена совпадает, независимо от ключей. +- `topic` - маршрутизация основана на правилах с ключами, разделенными точками. Например: `*.logs`, `records.*.*.2020`, `*.2018,*.2019,*.2020`. +- `headers` - маршрутизация основана на совпадении `key=value` с настройкой `x-match=all` или `x-match=any`. Пример списка ключей таблицы: `x-match=all,format=logs,type=report,year=2020`. +- `consistent-hash` - данные равномерно распределяются между всеми связанными таблицами, где имя точки обмена совпадает. Обратите внимание, что этот тип обмена должен быть включен с помощью плагина RabbitMQ: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. + +Если тип точки обмена не задан, по умолчанию используется `fanout`. В таком случае ключи маршрутизации для публикации данных должны быть рандомизированы в диапазоне `[1, num_consumers]` за каждое сообщение/пакет (или в диапазоне `[1, num_consumers * num_queues]`, если `rabbitmq_num_queues` задано). Эта конфигурация таблицы работает быстрее, чем любая другая, особенно когда заданы параметры `rabbitmq_num_consumers` и/или `rabbitmq_num_queues`. + +Если параметры`rabbitmq_num_consumers` и/или `rabbitmq_num_queues` заданы вместе с параметром `rabbitmq_exchange_type`: + +- плагин `rabbitmq-consistent-hash-exchange` должен быть включен. +- свойство `message_id` должно быть определено (уникальное для каждого сообщения/пакета). + +Пример: + +``` sql + CREATE TABLE queue ( + key UInt64, + value UInt64 + ) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'localhost:5672', + rabbitmq_exchange_name = 'exchange1', + rabbitmq_exchange_type = 'headers', + rabbitmq_routing_key_list = 'format=logs,type=report,year=2020', + rabbitmq_format = 'JSONEachRow', + rabbitmq_num_consumers = 5; + + CREATE TABLE daily (key UInt64, value UInt64) + ENGINE = MergeTree(); + + CREATE MATERIALIZED VIEW consumer TO daily + AS SELECT key, value FROM queue; + + SELECT key, value FROM daily ORDER BY key; +``` From 6682c62a905cca886bb26c8856c0243420635248 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Sep 2020 14:11:34 +0000 Subject: [PATCH 124/535] Fixes --- .../ReadBufferFromRabbitMQConsumer.cpp | 22 ++++++++++++++----- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 3 ++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 14 +++++++----- src/Storages/RabbitMQ/StorageRabbitMQ.h | 8 ++++--- .../integration/test_storage_rabbitmq/test.py | 11 ++++++---- 5 files changed, 39 insertions(+), 19 deletions(-) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 074f74c91aa..5be1cfeedfa 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -19,8 +19,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static const auto QUEUE_SIZE = 50000; - ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, ChannelPtr setup_channel_, @@ -34,6 +32,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( bool hash_exchange_, size_t num_queues_, const String & deadletter_exchange_, + uint32_t queue_size_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer_channel(std::move(consumer_channel_)) @@ -48,8 +47,9 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , deadletter_exchange(deadletter_exchange_) , log(log_) , row_delimiter(row_delimiter_) + , queue_size(queue_size_) , stopped(stopped_) - , received(QUEUE_SIZE * num_queues) + , received(queue_size) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) bindQueue(queue_id); @@ -93,14 +93,24 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) auto error_callback([&](const char * message) { - throw Exception("Failed to declare queue. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); + /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a + * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different + * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously + * declared queues via any of the various cli tools. + */ + throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \ + specifying differently those settings or use a different queue_base or manually delete previously declared queues, \ + which were declared with the same names. ERROR reason: " + + std::string(message), ErrorCodes::LOGICAL_ERROR); }); AMQP::Table queue_settings; + + queue_settings["x-max-length"] = queue_size; + queue_settings["x-overflow"] = "reject-publish"; + if (!deadletter_exchange.empty()) - { queue_settings["x-dead-letter-exchange"] = deadletter_exchange; - } /* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one * specific queue when its name is specified in queue_base setting diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 28c67e0314e..7f4d25e7f18 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -35,6 +35,7 @@ public: bool hash_exchange_, size_t num_queues_, const String & deadletter_exchange_, + uint32_t queue_size_, const std::atomic & stopped_); ~ReadBufferFromRabbitMQConsumer() override; @@ -93,10 +94,10 @@ private: const bool hash_exchange; const size_t num_queues; const String deadletter_exchange; - Poco::Logger * log; char row_delimiter; bool allowed = true; + uint32_t queue_size; const std::atomic & stopped; String channel_id; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 44c57a0db3f..6d565ea7374 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -40,6 +40,7 @@ namespace DB static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; static const auto HEARTBEAT_RESCHEDULE_MS = 3000; +static const uint32_t QUEUE_SIZE = 100000; namespace ErrorCodes { @@ -89,6 +90,7 @@ StorageRabbitMQ::StorageRabbitMQ( global_context.getConfigRef().getString("rabbitmq.password"))) , semaphore(0, num_consumers) , unique_strbase(getRandomName()) + , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) { loop = std::make_unique(); uv_loop_init(loop.get()); @@ -473,7 +475,7 @@ Pipe StorageRabbitMQ::read( auto block_size = getMaxBlockSize(); bool update_channels = false; - if (!event_handler->connectionRunning()) + if (!connection->usable()) { if (event_handler->loopRunning()) deactivateTask(looping_task, false, true); @@ -558,8 +560,8 @@ void StorageRabbitMQ::shutdown() wait_confirm.store(false); deactivateTask(streaming_task, true, false); - deactivateTask(heartbeat_task, true, false); deactivateTask(looping_task, true, true); + deactivateTask(heartbeat_task, true, false); connection->close(); @@ -617,7 +619,7 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() return std::make_shared( consumer_channel, setup_channel, event_handler, consumer_exchange, ++consumer_id, unique_strbase, queue_base, log, row_delimiter, hash_exchange, num_queues, - deadletter_exchange, stream_cancelled); + deadletter_exchange, queue_size, stream_cancelled); } @@ -711,6 +713,10 @@ bool StorageRabbitMQ::streamToViews() auto column_names = block_io.out->getHeader().getNames(); auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + /* Need to use event_handler->connectionRunning() because connection might have failed and to start error callbacks need to start + * the loop, so it is important not to use connection->usable() method here. And need to use connection->usable() method in cases + * when loop is deactivated and connection check is needed. + */ if (!event_handler->loopRunning() && event_handler->connectionRunning()) looping_task->activateAndSchedule(); @@ -828,9 +834,7 @@ void registerStorageRabbitMQ(StorageFactory & factory) auto rabbitmq_settings = std::make_unique(); if (has_settings) - { rabbitmq_settings->loadFromQuery(*args.storage_def); - } // Check arguments and settings #define CHECK_RABBITMQ_STORAGE_ARGUMENT(ARG_NUM, ARG_NAME) \ diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index e4e90abd98b..eddb6b78ab4 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -105,6 +105,7 @@ private: std::vector buffers; /// available buffers for RabbitMQ consumers String unique_strbase; /// to make unique consumer channel id + uint32_t queue_size; String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; /// remove exchange only once size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id @@ -125,11 +126,12 @@ private: void heartbeatFunc(); void loopingFunc(); - Names parseRoutingKeys(String routing_key_list); - AMQP::ExchangeType defineExchangeType(String exchange_type_); + static Names parseRoutingKeys(String routing_key_list); + static AMQP::ExchangeType defineExchangeType(String exchange_type_); + static String getTableBasedName(String name, const StorageID & table_id); + Context addSettings(Context context); size_t getMaxBlockSize(); - String getTableBasedName(String name, const StorageID & table_id); void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop); void initExchange(); diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 636bee1245f..ad8ad5501c9 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -130,7 +130,6 @@ def rabbitmq_setup_teardown(): # Tests -@pytest.mark.skip(reason="Flaky") @pytest.mark.timeout(180) def test_rabbitmq_select(rabbitmq_cluster): instance.query(''' @@ -253,7 +252,6 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): rabbitmq_check_result(result, True) -@pytest.mark.skip(reason="Flaky") @pytest.mark.timeout(180) def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): instance.query(''' @@ -424,7 +422,6 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): rabbitmq_check_result(result, True) -@pytest.mark.skip(reason="Flaky") @pytest.mark.timeout(180) def test_rabbitmq_many_materialized_views(rabbitmq_cluster): instance.query(''' @@ -594,7 +591,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_read_only_combo(rabbitmq_cluster): +def test_rabbitmq_mv_combo(rabbitmq_cluster): NUM_MV = 5; NUM_CONSUMERS = 4 @@ -604,6 +601,7 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'combo', + rabbitmq_queue_base = 'combo', rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, rabbitmq_format = 'JSONEachRow', @@ -864,7 +862,11 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'over', + rabbitmq_queue_base = 'over', rabbitmq_exchange_type = 'direct', + rabbitmq_num_consumers = 5, + rabbitmq_num_queues = 2, + rabbitmq_max_block_size = 10000, rabbitmq_routing_key_list = 'over', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -1649,6 +1651,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'producer_reconnect', rabbitmq_format = 'JSONEachRow', + rabbitmq_num_consumers = 2, rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.consume; From 26d75f76026303b6f3769ab4ea39ff639ebe836a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Sep 2020 01:25:10 +0300 Subject: [PATCH 125/535] do fsync for WAL --- src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ .../MergeTree/MergeTreeWriteAheadLog.cpp | 32 +++++++++++++++++-- .../MergeTree/MergeTreeWriteAheadLog.h | 10 +++++- 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 1341526c38b..edf03710974 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,8 @@ struct Settings; M(UInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ M(Bool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ + M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index eda8579c76a..6f220fc7d5d 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -16,17 +17,23 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; } - MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( - const MergeTreeData & storage_, + MergeTreeData & storage_, const DiskPtr & disk_, const String & name_) : storage(storage_) , disk(disk_) , name(name_) , path(storage.getRelativeDataPath() + name_) + , pool(storage.global_context.getSchedulePool()) { init(); + sync_task = pool.createTask("MergeTreeWriteAheadLog::sync", [this] + { + std::lock_guard lock(write_mutex); + out->sync(); + sync_scheduled = false; + }); } void MergeTreeWriteAheadLog::init() @@ -38,6 +45,7 @@ void MergeTreeWriteAheadLog::init() block_out = std::make_unique(*out, 0, Block{}); min_block_number = std::numeric_limits::max(); max_block_number = -1; + bytes_at_last_sync = 0; } void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_name) @@ -53,6 +61,7 @@ void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_na writeStringBinary(part_name, *out); block_out->write(block); block_out->flush(); + sync(lock); auto max_wal_bytes = storage.getSettings()->write_ahead_log_max_bytes; if (out->count() > max_wal_bytes) @@ -66,6 +75,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) writeIntBinary(static_cast(0), *out); writeIntBinary(static_cast(ActionType::DROP_PART), *out); writeStringBinary(part_name, *out); + sync(lock); } void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) @@ -175,6 +185,24 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor return result; } +void MergeTreeWriteAheadLog::sync(const std::lock_guard &) +{ + size_t bytes_to_sync = storage.getSettings()->write_ahead_log_bytes_to_fsync; + time_t time_to_sync = storage.getSettings()->write_ahead_log_interval_ms_to_fsync; + size_t current_bytes = out->count(); + + if (bytes_to_sync && current_bytes - bytes_at_last_sync > bytes_to_sync) + { + sync_task->schedule(); + bytes_at_last_sync = current_bytes; + } + else if (time_to_sync && !sync_scheduled) + { + sync_task->scheduleAfter(time_to_sync); + sync_scheduled = true; + } +} + std::optional MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(const String & filename) { diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 2cc3c2b4181..43abf3c04be 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -31,7 +32,7 @@ public: constexpr static auto WAL_FILE_EXTENSION = ".bin"; constexpr static auto DEFAULT_WAL_FILE_NAME = "wal.bin"; - MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, + MergeTreeWriteAheadLog(MergeTreeData & storage_, const DiskPtr & disk_, const String & name = DEFAULT_WAL_FILE_NAME); void addPart(const Block & block, const String & part_name); @@ -44,6 +45,7 @@ public: private: void init(); void rotate(const std::lock_guard & lock); + void sync(const std::lock_guard & lock); const MergeTreeData & storage; DiskPtr disk; @@ -56,6 +58,12 @@ private: Int64 min_block_number = std::numeric_limits::max(); Int64 max_block_number = -1; + BackgroundSchedulePool & pool; + BackgroundSchedulePoolTaskHolder sync_task; + + size_t bytes_at_last_sync = 0; + bool sync_scheduled = false; + mutable std::mutex write_mutex; }; From fa04b39d32e566c641009da6724a2b9dc4a5e1f6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 2 Sep 2020 02:06:53 +0300 Subject: [PATCH 126/535] Revert "Change query event filter and add integration test for empty GTID transaction" --- src/Core/MySQL/MySQLReplication.cpp | 8 ++--- .../materialize_with_ddl.py | 36 ------------------- .../test_materialize_mysql_database/test.py | 5 --- 3 files changed, 4 insertions(+), 45 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index f26436440b8..41afe3cde6a 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -103,17 +103,17 @@ namespace MySQLReplication = header.event_size - EVENT_HEADER_LENGTH - 4 - 4 - 1 - 2 - 2 - status_len - schema_len - 1 - CHECKSUM_CRC32_SIGNATURE_LENGTH; query.resize(len); payload.readStrict(reinterpret_cast(query.data()), len); - if (query.starts_with("BEGIN") || query.starts_with("COMMIT")) + if (query.rfind("BEGIN", 0) == 0 || query.rfind("COMMIT") == 0) { typ = QUERY_EVENT_MULTI_TXN_FLAG; } - else if (query.starts_with("XA")) + else if (query.rfind("XA", 0) == 0) { - if (query.starts_with("XA ROLLBACK")) + if (query.rfind("XA ROLLBACK", 0) == 0) throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::UNKNOWN_EXCEPTION); typ = QUERY_EVENT_XA; } - else if (query.starts_with("SAVEPOINT")) + else if (query.rfind("SAVEPOINT", 0) == 0) { throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::UNKNOWN_EXCEPTION); } diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 0dff05df3a1..18695f40e53 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -1,5 +1,4 @@ import time -import pymysql.cursors def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=3): @@ -322,38 +321,3 @@ def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_no clickhouse_node.query("DROP DATABASE test_database") mysql_node.query("DROP DATABASE test_database") - - -def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name): - mysql_node.query("CREATE DATABASE test_database") - - mysql_node.query("RESET MASTER") - mysql_node.query("CREATE TABLE test_database.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(1)") - - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( - service_name)) - - # Reject one empty GTID QUERY event with 'BEGIN' and 'COMMIT' - mysql_cursor = mysql_node.cursor(pymysql.cursors.DictCursor) - mysql_cursor.execute("SHOW MASTER STATUS") - (uuid, seqs) = mysql_cursor.fetchall()[0]["Executed_Gtid_Set"].split(":") - (seq_begin, seq_end) = seqs.split("-") - assert int(seq_begin) == 1 - assert int(seq_end) == 3 - next_gtid = uuid + ":" + str(int(seq_end) + 1) - mysql_node.query("SET gtid_next='" + next_gtid + "'") - mysql_node.query("BEGIN") - mysql_node.query("COMMIT") - mysql_node.query("SET gtid_next='AUTOMATIC'") - - # Reject one 'BEGIN' QUERY event and 'COMMIT' XID event. - mysql_node.query("/* start */ begin /* end */") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(2)") - mysql_node.query("/* start */ commit /* end */") - - check_query(clickhouse_node, "SELECT * FROM test_database.t1 ORDER BY a FORMAT TSV", - "1\tBEGIN\n2\tBEGIN\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index d45a5e3ceaf..bfda4e7e840 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -120,8 +120,3 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") -def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7.alloc_connection(), "mysql5_7") - -def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0.alloc_connection(), "mysql8_0") From 6dfab8815660e967aec922ce5b6aaa1c11536933 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 2 Sep 2020 08:31:51 +0800 Subject: [PATCH 127/535] ISSUES-14235 change string.rfind to string starts_with and add some tests --- src/Core/MySQL/MySQLReplication.cpp | 13 +++---- .../materialize_with_ddl.py | 35 +++++++++++++++++++ .../test_materialize_mysql_database/test.py | 5 +++ 3 files changed, 47 insertions(+), 6 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 41afe3cde6a..104d2159f60 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_EXCEPTION; + extern const int LOGICAL_ERROR; } namespace MySQLReplication @@ -103,19 +104,19 @@ namespace MySQLReplication = header.event_size - EVENT_HEADER_LENGTH - 4 - 4 - 1 - 2 - 2 - status_len - schema_len - 1 - CHECKSUM_CRC32_SIGNATURE_LENGTH; query.resize(len); payload.readStrict(reinterpret_cast(query.data()), len); - if (query.rfind("BEGIN", 0) == 0 || query.rfind("COMMIT") == 0) + if (query.starts_with("BEGIN") || query.starts_with("COMMIT")) { typ = QUERY_EVENT_MULTI_TXN_FLAG; } - else if (query.rfind("XA", 0) == 0) + else if (query.starts_with("XA")) { - if (query.rfind("XA ROLLBACK", 0) == 0) - throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::UNKNOWN_EXCEPTION); + if (query.starts_with("XA ROLLBACK")) + throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR); typ = QUERY_EVENT_XA; } - else if (query.rfind("SAVEPOINT", 0) == 0) + else if (query.starts_with("SAVEPOINT")) { - throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::UNKNOWN_EXCEPTION); + throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR); } } diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 18695f40e53..eb3b0cdda4f 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -1,4 +1,5 @@ import time +import pymysql.cursors def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=3): @@ -321,3 +322,37 @@ def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_no clickhouse_node.query("DROP DATABASE test_database") mysql_node.query("DROP DATABASE test_database") + +def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name): + mysql_node.query("CREATE DATABASE test_database") + + mysql_node.query("RESET MASTER") + mysql_node.query("CREATE TABLE test_database.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") + mysql_node.query("INSERT INTO test_database.t1(a) VALUES(1)") + + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + service_name)) + + # Reject one empty GTID QUERY event with 'BEGIN' and 'COMMIT' + mysql_cursor = mysql_node.alloc_connection().cursor(pymysql.cursors.DictCursor) + mysql_cursor.execute("SHOW MASTER STATUS") + (uuid, seqs) = mysql_cursor.fetchall()[0]["Executed_Gtid_Set"].split(":") + (seq_begin, seq_end) = seqs.split("-") + assert int(seq_begin) == 1 + assert int(seq_end) == 3 + next_gtid = uuid + ":" + str(int(seq_end) + 1) + mysql_node.query("SET gtid_next='" + next_gtid + "'") + mysql_node.query("BEGIN") + mysql_node.query("COMMIT") + mysql_node.query("SET gtid_next='AUTOMATIC'") + + # Reject one 'BEGIN' QUERY event and 'COMMIT' XID event. + mysql_node.query("/* start */ begin /* end */") + mysql_node.query("INSERT INTO test_database.t1(a) VALUES(2)") + mysql_node.query("/* start */ commit /* end */") + + check_query(clickhouse_node, "SELECT * FROM test_database.t1 ORDER BY a FORMAT TSV", + "1\tBEGIN\n2\tBEGIN\n") + clickhouse_node.query("DROP DATABASE test_database") + mysql_node.query("DROP DATABASE test_database") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index bfda4e7e840..cc955da92a4 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -120,3 +120,8 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") +def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7): + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql5_7") + +def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0): + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql8_0") From 443ed33ab3def36559ace9f4d74b476faf193853 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2020 04:26:35 +0300 Subject: [PATCH 128/535] Less number of threads in builder --- debian/rules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/rules b/debian/rules index 5b271a8691f..e9882a09e76 100755 --- a/debian/rules +++ b/debian/rules @@ -18,7 +18,7 @@ ifeq ($(CCACHE_PREFIX),distcc) THREADS_COUNT=$(shell distcc -j) endif ifeq ($(THREADS_COUNT),) - THREADS_COUNT=$(shell nproc || grep -c ^processor /proc/cpuinfo || sysctl -n hw.ncpu || echo 4) + THREADS_COUNT=$(shell $$(( $$(nproc || grep -c ^processor /proc/cpuinfo || sysctl -n hw.ncpu || echo 8) / 2 )) ) endif DEB_BUILD_OPTIONS+=parallel=$(THREADS_COUNT) From 56bbac1569e8cc7b6853b3268ce451b791bf48c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2020 04:28:52 +0300 Subject: [PATCH 129/535] Trigger CI --- src/Dictionaries/BucketCache.h | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Dictionaries/BucketCache.h b/src/Dictionaries/BucketCache.h index 381110066a6..c7dec12c3e4 100644 --- a/src/Dictionaries/BucketCache.h +++ b/src/Dictionaries/BucketCache.h @@ -30,12 +30,13 @@ struct Int64Hasher }; -/* - Class for storing cache index. - It consists of two arrays. - The first one is split into buckets (each stores 8 elements (cells)) determined by hash of the element key. - The second one is split into 4bit numbers, which are positions in bucket for next element write (So cache uses FIFO eviction algorithm inside each bucket). -*/ +/** + * Class for storing cache index. + * It consists of two arrays. + * The first one is split into buckets (each stores 8 elements (cells)) determined by hash of the element key. + * The second one is split into 4bit numbers, which are positions in bucket for next element write + * (So cache uses FIFO eviction algorithm inside each bucket). + */ template class BucketCacheIndex { From 04c88ca9e434ade639889a3a1be244be71a07710 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 2 Sep 2020 05:06:21 +0300 Subject: [PATCH 130/535] Update AccessFlags.h --- src/Access/AccessFlags.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/AccessFlags.h b/src/Access/AccessFlags.h index 11d39585238..3cb92b6b855 100644 --- a/src/Access/AccessFlags.h +++ b/src/Access/AccessFlags.h @@ -100,7 +100,7 @@ public: /// The same as allFlags(). static AccessFlags allFlagsGrantableOnGlobalLevel(); - /// Returns all the flags which could be granted on the global level. + /// Returns all the flags which could be granted on the database level. /// Returns allDatabaseFlags() | allTableFlags() | allDictionaryFlags() | allColumnFlags(). static AccessFlags allFlagsGrantableOnDatabaseLevel(); From a80c1adee81631f770f642ad4430a8ff44ff46af Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 12:05:02 +0800 Subject: [PATCH 131/535] Add JSONCompactStrings formats --- docs/en/interfaces/formats.md | 288 +++++++++++------- src/Formats/FormatFactory.cpp | 6 - .../Impl/JSONCompactEachRowRowInputFormat.cpp | 49 ++- .../Impl/JSONCompactEachRowRowInputFormat.h | 11 +- .../JSONCompactEachRowRowOutputFormat.cpp | 37 ++- .../Impl/JSONCompactEachRowRowOutputFormat.h | 9 +- .../Impl/JSONCompactRowOutputFormat.cpp | 30 +- .../Formats/Impl/JSONCompactRowOutputFormat.h | 10 +- .../Impl/JSONEachRowRowInputFormat.cpp | 42 ++- .../Formats/Impl/JSONEachRowRowInputFormat.h | 9 +- .../Impl/JSONEachRowRowOutputFormat.cpp | 32 +- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 10 +- ...JSONEachRowWithProgressRowOutputFormat.cpp | 11 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 44 ++- .../Formats/Impl/JSONRowOutputFormat.h | 9 +- .../Impl/JSONStringsEachRowRowInputFormat.cpp | 245 --------------- .../Impl/JSONStringsEachRowRowInputFormat.h | 54 ---- .../JSONStringsEachRowRowOutputFormat.cpp | 117 ------- .../Impl/JSONStringsEachRowRowOutputFormat.h | 45 --- .../Impl/JSONStringsRowOutputFormat.cpp | 93 ------ .../Formats/Impl/JSONStringsRowOutputFormat.h | 43 --- .../0_stateless/01446_JSONStringsEachRow.sql | 63 ---- .../01446_json_strings_each_row.reference | 22 ++ .../01446_json_strings_each_row.sql | 38 +++ .../0_stateless/01447_json_strings.reference | 43 +++ ...JSONStrings.sql => 01447_json_strings.sql} | 0 ...8_json_compact_strings_each_row.reference} | 0 .../01448_json_compact_strings_each_row.sql | 63 ++++ ...e => 01449_json_compact_strings.reference} | 0 .../01449_json_compact_strings.sql | 10 + 30 files changed, 621 insertions(+), 812 deletions(-) delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h delete mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h delete mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.sql create mode 100644 tests/queries/0_stateless/01446_json_strings_each_row.reference create mode 100644 tests/queries/0_stateless/01446_json_strings_each_row.sql create mode 100644 tests/queries/0_stateless/01447_json_strings.reference rename tests/queries/0_stateless/{01447_JSONStrings.sql => 01447_json_strings.sql} (100%) rename tests/queries/0_stateless/{01446_JSONStringsEachRow.reference => 01448_json_compact_strings_each_row.reference} (100%) create mode 100644 tests/queries/0_stateless/01448_json_compact_strings_each_row.sql rename tests/queries/0_stateless/{01447_JSONStrings.reference => 01449_json_compact_strings.reference} (100%) create mode 100644 tests/queries/0_stateless/01449_json_compact_strings.sql diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 9c7c2dda8dc..bfe5b6218e4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -10,45 +10,51 @@ results of a `SELECT`, and to perform `INSERT`s into a file-backed table. The supported formats are: -| Format | Input | Output | -|-----------------------------------------------------------------|-------|--------| -| [TabSeparated](#tabseparated) | ✔ | ✔ | -| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | -| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | -| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | -| [Template](#format-template) | ✔ | ✔ | -| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | -| [CSV](#csv) | ✔ | ✔ | -| [CSVWithNames](#csvwithnames) | ✔ | ✔ | -| [CustomSeparated](#format-customseparated) | ✔ | ✔ | -| [Values](#data-format-values) | ✔ | ✔ | -| [Vertical](#vertical) | ✗ | ✔ | -| [VerticalRaw](#verticalraw) | ✗ | ✔ | -| [JSON](#json) | ✗ | ✔ | -| [JSONCompact](#jsoncompact) | ✗ | ✔ | -| [JSONStrings](#jsonstrings) | ✗ | ✔ | -| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | -| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | -| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ | -| [TSKV](#tskv) | ✔ | ✔ | -| [Pretty](#pretty) | ✗ | ✔ | -| [PrettyCompact](#prettycompact) | ✗ | ✔ | -| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | -| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | -| [PrettySpace](#prettyspace) | ✗ | ✔ | -| [Protobuf](#protobuf) | ✔ | ✔ | -| [Avro](#data-format-avro) | ✔ | ✔ | -| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | -| [Parquet](#data-format-parquet) | ✔ | ✔ | -| [Arrow](#data-format-arrow) | ✔ | ✔ | -| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | -| [ORC](#data-format-orc) | ✔ | ✗ | -| [RowBinary](#rowbinary) | ✔ | ✔ | -| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | -| [Native](#native) | ✔ | ✔ | -| [Null](#null) | ✗ | ✔ | -| [XML](#xml) | ✗ | ✔ | -| [CapnProto](#capnproto) | ✔ | ✗ | +| Format | Input | Output | +|-----------------------------------------------------------------------------------------|-------|--------| +| [TabSeparated](#tabseparated) | ✔ | ✔ | +| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | +| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | +| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [Template](#format-template) | ✔ | ✔ | +| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | +| [CSV](#csv) | ✔ | ✔ | +| [CSVWithNames](#csvwithnames) | ✔ | ✔ | +| [CustomSeparated](#format-customseparated) | ✔ | ✔ | +| [Values](#data-format-values) | ✔ | ✔ | +| [Vertical](#vertical) | ✗ | ✔ | +| [VerticalRaw](#verticalraw) | ✗ | ✔ | +| [JSON](#json) | ✗ | ✔ | +| [JSONString](#jsonstring) | ✗ | ✔ | +| [JSONCompact](#jsoncompact) | ✗ | ✔ | +| [JSONCompactString](#jsoncompactstring) | ✗ | ✔ | +| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | +| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ | +| [JSONStringEachRow](#jsonstringeachrow) | ✔ | ✔ | +| [JSONStringEachRowWithProgress](#jsonstringeachrowwithprogress) | ✗ | ✔ | +| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | +| [JSONCompactEachRowWithNamesAndTypes](#jsoncompacteachrowwithnamesandtypes) | ✔ | ✔ | +| [JSONCompactStringEachRow](#jsoncompactstringeachrow) | ✔ | ✔ | +| [JSONCompactStringEachRowWithNamesAndTypes](#jsoncompactstringeachrowwithnamesandtypes) | ✔ | ✔ | +| [TSKV](#tskv) | ✔ | ✔ | +| [Pretty](#pretty) | ✗ | ✔ | +| [PrettyCompact](#prettycompact) | ✗ | ✔ | +| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | +| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | +| [PrettySpace](#prettyspace) | ✗ | ✔ | +| [Protobuf](#protobuf) | ✔ | ✔ | +| [Avro](#data-format-avro) | ✔ | ✔ | +| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | +| [Parquet](#data-format-parquet) | ✔ | ✔ | +| [Arrow](#data-format-arrow) | ✔ | ✔ | +| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | +| [ORC](#data-format-orc) | ✔ | ✗ | +| [RowBinary](#rowbinary) | ✔ | ✔ | +| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | +| [Native](#native) | ✔ | ✔ | +| [Null](#null) | ✗ | ✔ | +| [XML](#xml) | ✗ | ✔ | +| [CapnProto](#capnproto) | ✔ | ✗ | You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section. @@ -395,62 +401,41 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA "meta": [ { - "name": "SearchPhrase", + "name": "'hello'", "type": "String" }, { - "name": "c", + "name": "multiply(42, number)", "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" } ], "data": [ { - "SearchPhrase": "", - "c": "8267016" + "'hello'": "hello", + "multiply(42, number)": "0", + "range(5)": [0,1,2,3,4] }, { - "SearchPhrase": "bathroom interior design", - "c": "2166" + "'hello'": "hello", + "multiply(42, number)": "42", + "range(5)": [0,1,2,3,4] }, { - "SearchPhrase": "yandex", - "c": "1655" - }, - { - "SearchPhrase": "spring 2014 fashion", - "c": "1549" - }, - { - "SearchPhrase": "freeform photos", - "c": "1480" + "'hello'": "hello", + "multiply(42, number)": "84", + "range(5)": [0,1,2,3,4] } ], - "totals": - { - "SearchPhrase": "", - "c": "8873898" - }, + "rows": 3, - "extremes": - { - "min": - { - "SearchPhrase": "", - "c": "1480" - }, - "max": - { - "SearchPhrase": "", - "c": "8267016" - } - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 + "rows_before_limit_at_least": 3 } ``` @@ -471,73 +456,166 @@ ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `n See also the [JSONEachRow](#jsoneachrow) format. -## JSONCompact {#jsoncompact} +## JSONString {#jsonstring} -Differs from JSON only in that data rows are output in arrays of any element type, not in objects. +Differs from JSON only in that data fields are output in strings, not in typed json values. Example: -``` json +```json { "meta": [ { - "name": "SearchPhrase", + "name": "'hello'", "type": "String" }, { - "name": "c", + "name": "multiply(42, number)", "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" } ], "data": [ - ["", "8267016"], - ["bathroom interior design", "2166"], - ["yandex", "1655"], - ["fashion trends spring 2014", "1549"], - ["freeform photo", "1480"] + { + "'hello'": "hello", + "multiply(42, number)": "0", + "range(5)": "[0,1,2,3,4]" + }, + { + "'hello'": "hello", + "multiply(42, number)": "42", + "range(5)": "[0,1,2,3,4]" + }, + { + "'hello'": "hello", + "multiply(42, number)": "84", + "range(5)": "[0,1,2,3,4]" + } ], - "totals": ["","8873898"], + "rows": 3, - "extremes": - { - "min": ["","1480"], - "max": ["","8267016"] - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 + "rows_before_limit_at_least": 3 } ``` -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -See also the `JSONEachRow` format. +## JSONCompact {#jsoncompact} +## JSONCompactString {#jsoncompactstring} -## JSONStrings {#jsonstrings} +Differs from JSON only in that data rows are output in arrays, not in objects. -Differs from JSON and JSONCompact only in that data rows are output in arrays of strings. +Example: -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -See also the `JSONEachRow` format. +``` json +// JSONCompact +{ + "meta": + [ + { + "name": "'hello'", + "type": "String" + }, + { + "name": "multiply(42, number)", + "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" + } + ], + + "data": + [ + ["hello", "0", [0,1,2,3,4]], + ["hello", "42", [0,1,2,3,4]], + ["hello", "84", [0,1,2,3,4]] + ], + + "rows": 3, + + "rows_before_limit_at_least": 3 +} +``` + +```json +// JSONCompactString +{ + "meta": + [ + { + "name": "'hello'", + "type": "String" + }, + { + "name": "multiply(42, number)", + "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" + } + ], + + "data": + [ + ["hello", "0", "[0,1,2,3,4]"], + ["hello", "42", "[0,1,2,3,4]"], + ["hello", "84", "[0,1,2,3,4]"] + ], + + "rows": 3, + + "rows_before_limit_at_least": 3 +} +``` ## JSONEachRow {#jsoneachrow} +## JSONStringEachRow {#jsonstringeachrow} ## JSONCompactEachRow {#jsoncompacteachrow} -## JSONStringsEachRow {#jsonstringseachrow} +## JSONCompactStringEachRow {#jsoncompactstringeachrow} When using these formats, ClickHouse outputs rows as separated, newline-delimited JSON values, but the data as a whole is not valid JSON. ``` json {"some_int":42,"some_str":"hello","some_tuple":[1,"a"]} // JSONEachRow [42,"hello",[1,"a"]] // JSONCompactEachRow -["42","hello","(2,'a')"] // JSONStringsEachRow +["42","hello","(2,'a')"] // JSONCompactStringsEachRow ``` When inserting the data, you should provide a separate JSON value for each row. +## JSONEachRowWithProgress {#jsoneachrowwithprogress} +## JSONStringEachRowWithProgress {#jsonstringeachrowwithprogress} + +Differs from JSONEachRow/JSONStringEachRow in that ClickHouse will also yield progress information as JSON objects. + +```json +{"row":{"'hello'":"hello","multiply(42, number)":"0","range(5)":[0,1,2,3,4]}} +{"row":{"'hello'":"hello","multiply(42, number)":"42","range(5)":[0,1,2,3,4]}} +{"row":{"'hello'":"hello","multiply(42, number)":"84","range(5)":[0,1,2,3,4]}} +{"progress":{"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"3"}} +``` + +## JSONCompactEachRowWithNamesAndTypes {#jsoncompacteachrowwithnamesandtypes} +## JSONCompactStringEachRowWithNamesAndTypes {#jsoncompactstringeachrowwithnamesandtypes} + +Differs from JSONCompactEachRow/JSONCompactStringEachRow in that the column names and types are written as the first two rows. + +```json +["'hello'", "multiply(42, number)", "range(5)"] +["String", "UInt64", "Array(UInt8)"] +["hello", "0", [0,1,2,3,4]] +["hello", "42", [0,1,2,3,4]] +["hello", "84", [0,1,2,3,4]] +``` + ### Inserting Data {#inserting-data} ``` sql diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index cb378fbea96..871098e00c0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -352,8 +352,6 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); @@ -380,7 +378,6 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorJSONStrings(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); void registerOutputFormatProcessorNull(FormatFactory & factory); @@ -421,8 +418,6 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONEachRow(*this); registerInputFormatProcessorJSONCompactEachRow(*this); registerOutputFormatProcessorJSONCompactEachRow(*this); - registerInputFormatProcessorJSONStringsEachRow(*this); - registerOutputFormatProcessorJSONStringsEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorTemplate(*this); @@ -449,7 +444,6 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); registerOutputFormatProcessorJSONEachRowWithProgress(*this); - registerOutputFormatProcessorJSONStrings(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 82e3cb795bf..eb697ce5318 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -19,8 +20,9 @@ JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(ReadBuffer & const Block & header_, Params params_, const FormatSettings & format_settings_, - bool with_names_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) + bool with_names_, + bool yield_strings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort().getHeader(); size_t num_columns = sample.columns(); @@ -200,10 +202,25 @@ void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & { read_columns[index] = true; const auto & type = data_types[index]; - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + + if (yield_strings) + { + // notice: null_as_default on "null" strings is not supported + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } else - type->deserializeAsTextJSON(*columns[index], in, format_settings); + { + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], in, format_settings); + } } catch (Exception & e) { @@ -225,7 +242,7 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings, false); + return std::make_shared(buf, sample, std::move(params), settings, false, false); }); factory.registerInputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( @@ -234,7 +251,25 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings, true); + return std::make_shared(buf, sample, std::move(params), settings, true, false); + }); + + factory.registerInputFormatProcessor("JSONCompactStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, false, true); + }); + + factory.registerInputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 5c864ebc751..593f297108c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -12,12 +12,18 @@ namespace DB class ReadBuffer; -/** A stream for reading data in JSONCompactEachRow and JSONCompactEachRowWithNamesAndTypes formats +/** A stream for reading data in JSONCompactEachRow- formats */ class JSONCompactEachRowRowInputFormat : public IRowInputFormat { public: - JSONCompactEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); + JSONCompactEachRowRowInputFormat( + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool with_names_, + bool yield_strings_); String getName() const override { return "JSONCompactEachRowRowInputFormat"; } @@ -49,6 +55,7 @@ private: std::vector names_of_columns; bool with_names; + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index e155dcb4247..ab8fd164c3c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -12,8 +12,9 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, - bool with_names_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) + bool with_names_, + bool yield_strings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -23,7 +24,15 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer void JSONCompactEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { - type.serializeAsTextJSON(column, row_num, out, settings); + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); + } + else + type.serializeAsTextJSON(column, row_num, out, settings); } @@ -97,7 +106,7 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false); + return std::make_shared(buf, sample, callback, format_settings, false, false); }); factory.registerOutputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( @@ -106,7 +115,25 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings &format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true); + return std::make_shared(buf, sample, callback, format_settings, true, false); + }); + + factory.registerOutputFormatProcessor("JSONCompactStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, false, true); + }); + + factory.registerOutputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + WriteBuffer &buf, + const Block &sample, + FormatFactory::WriteCallback callback, + const FormatSettings &format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index a7857a82d2d..56936783e78 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -15,7 +15,13 @@ namespace DB class JSONCompactEachRowRowOutputFormat : public IRowOutputFormat { public: - JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); + JSONCompactEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool with_names_, + bool yield_strings_); String getName() const override { return "JSONCompactEachRowRowOutputFormat"; } @@ -41,5 +47,6 @@ private: NamesAndTypes fields; bool with_names; + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index 7e56a4643da..c36942cff09 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -8,15 +8,28 @@ namespace DB { JSONCompactRowOutputFormat::JSONCompactRowOutputFormat( - WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : JSONRowOutputFormat(out_, header, callback, settings_) + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : JSONRowOutputFormat(out_, header, callback, settings_, yield_strings_) { } void JSONCompactRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { - type.serializeAsTextJSON(column, row_num, *ostr, settings); + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -83,7 +96,16 @@ void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONCompactStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index f4002f74287..6585016c44f 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -11,12 +11,17 @@ namespace DB struct FormatSettings; -/** The stream for outputting data in the JSONCompact format. +/** The stream for outputting data in the JSONCompact- formats. */ class JSONCompactRowOutputFormat : public JSONRowOutputFormat { public: - JSONCompactRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONCompactRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONCompactRowOutputFormat"; } @@ -37,7 +42,6 @@ protected: } void writeTotalsFieldDelimiter() override; - }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 6350db3b211..9ba82fbb009 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -29,8 +30,12 @@ enum JSONEachRowRowInputFormat::JSONEachRowRowInputFormat( - ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()) + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool yield_strings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()), yield_strings(yield_strings_) { /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. skipBOMIfExists(in); @@ -138,10 +143,25 @@ void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns { seen_columns[index] = read_columns[index] = true; const auto & type = getPort().getHeader().getByPosition(index).type; - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + + if (yield_strings) + { + // notice: null_as_default on "null" strings is not supported + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } else - type->deserializeAsTextJSON(*columns[index], in, format_settings); + { + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], in, format_settings); + } } catch (Exception & e) { @@ -318,13 +338,23 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings); + return std::make_shared(buf, sample, std::move(params), settings, false); + }); + + factory.registerInputFormatProcessor("JSONStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true); }); } void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) { factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl); + factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRowImpl); } } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index a0a4b735a3e..29a6ce6ecb8 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -20,7 +20,12 @@ class ReadBuffer; class JSONEachRowRowInputFormat : public IRowInputFormat { public: - JSONEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); + JSONEachRowRowInputFormat( + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool yield_strings_); String getName() const override { return "JSONEachRowRowInputFormat"; } @@ -75,6 +80,8 @@ private: bool data_in_square_brackets = false; bool allow_new_rows = true; + + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 910a9710de3..069499d99c1 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -8,8 +8,13 @@ namespace DB { -JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : IRowOutputFormat(header_, out_, callback), settings(settings_) +JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -27,7 +32,17 @@ void JSONEachRowRowOutputFormat::writeField(const IColumn & column, const IDataT { writeString(fields[field_number], out); writeChar(':', out); - type.serializeAsTextJSON(column, row_num, out, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); + } + else + type.serializeAsTextJSON(column, row_num, out, settings); + ++field_number; } @@ -59,7 +74,16 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index d2b6937cd01..5346a1ab19f 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -15,7 +15,12 @@ namespace DB class JSONEachRowRowOutputFormat : public IRowOutputFormat { public: - JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONEachRowRowOutputFormat"; } @@ -35,6 +40,9 @@ private: Names fields; FormatSettings settings; + +protected: + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index a611b5a129b..35720df9672 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -36,7 +36,16 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRowWithProgress", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index b3255f2894e..7dd7eb9953a 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -7,8 +7,13 @@ namespace DB { -JSONRowOutputFormat::JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : IRowOutputFormat(header, out_, callback), settings(settings_) +JSONRowOutputFormat::JSONRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : IRowOutputFormat(header, out_, callback), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -71,7 +76,17 @@ void JSONRowOutputFormat::writeField(const IColumn & column, const IDataType & t writeCString("\t\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - type.serializeAsTextJSON(column, row_num, *ostr, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -80,7 +95,17 @@ void JSONRowOutputFormat::writeTotalsField(const IColumn & column, const IDataTy writeCString("\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - type.serializeAsTextJSON(column, row_num, *ostr, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -249,7 +274,16 @@ void registerOutputFormatProcessorJSON(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index f9aea3a3e8b..4e9cceb717e 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -16,7 +16,12 @@ namespace DB class JSONRowOutputFormat : public IRowOutputFormat { public: - JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONRowOutputFormat"; } @@ -78,6 +83,8 @@ protected: Progress progress; Stopwatch watch; FormatSettings settings; + + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp deleted file mode 100644 index fff44a204fb..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp +++ /dev/null @@ -1,245 +0,0 @@ -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; - extern const int CANNOT_READ_ALL_DATA; -} - - -JSONStringsEachRowRowInputFormat::JSONStringsEachRowRowInputFormat(ReadBuffer & in_, - const Block & header_, - Params params_, - const FormatSettings & format_settings_, - bool with_names_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) -{ - const auto & sample = getPort().getHeader(); - size_t num_columns = sample.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = sample.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - } -} - -void JSONStringsEachRowRowInputFormat::resetParser() -{ - IRowInputFormat::resetParser(); - column_indexes_for_input_fields.clear(); - not_seen_columns.clear(); -} - -void JSONStringsEachRowRowInputFormat::readPrefix() -{ - /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. - skipBOMIfExists(in); - - if (with_names) - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, false); - - assertChar('[', in); - do - { - skipWhitespaceIfAny(in); - String column_name; - readJSONString(column_name, in); - addInputColumn(column_name); - skipWhitespaceIfAny(in); - } - while (checkChar(',', in)); - assertChar(']', in); - skipEndOfLine(); - - /// Type checking - assertChar('[', in); - for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) - { - skipWhitespaceIfAny(in); - String data_type; - readJSONString(data_type, in); - - if (column_indexes_for_input_fields[i] && - data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) - { - throw Exception( - "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name - + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + - ", not " + data_type, - ErrorCodes::INCORRECT_DATA - ); - } - - if (i != column_indexes_for_input_fields.size() - 1) - assertChar(',', in); - skipWhitespaceIfAny(in); - } - assertChar(']', in); - } - else - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, true); - column_indexes_for_input_fields.resize(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - column_indexes_for_input_fields[i] = i; - } - } - - for (size_t i = 0; i < read_columns.size(); ++i) - { - if (!read_columns[i]) - { - not_seen_columns.emplace_back(i); - } - } -} - -void JSONStringsEachRowRowInputFormat::addInputColumn(const String & column_name) -{ - names_of_columns.emplace_back(column_name); - - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - - throw Exception( - "Unknown field found in JSONStringsEachRow header: '" + column_name + "' " + - "at position " + std::to_string(column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (read_columns[column_index]) - throw Exception("Duplicate field found while parsing JSONStringsEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); - - read_columns[column_index] = true; - column_indexes_for_input_fields.emplace_back(column_index); -} - -bool JSONStringsEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) -{ - skipEndOfLine(); - - if (in.eof()) - return false; - - size_t num_columns = columns.size(); - - read_columns.assign(num_columns, false); - - assertChar('[', in); - for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) - { - const auto & table_column = column_indexes_for_input_fields[file_column]; - if (table_column) - { - readField(*table_column, columns); - } - else - { - skipJSONField(in, StringRef(names_of_columns[file_column])); - } - - skipWhitespaceIfAny(in); - if (in.eof()) - throw Exception("Unexpected end of stream while parsing JSONStringsEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); - if (file_column + 1 != column_indexes_for_input_fields.size()) - { - assertChar(',', in); - skipWhitespaceIfAny(in); - } - } - assertChar(']', in); - - for (const auto & name : not_seen_columns) - columns[name]->insertDefault(); - - ext.read_columns = read_columns; - return true; -} - -void JSONStringsEachRowRowInputFormat::skipEndOfLine() -{ - skipWhitespaceIfAny(in); - if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) - ++in.position(); - - skipWhitespaceIfAny(in); -} - -void JSONStringsEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) -{ - try - { - read_columns[index] = true; - const auto & type = data_types[index]; - - String str; - readJSONString(str, in); - - ReadBufferFromString buf(str); - - type->deserializeAsWholeText(*columns[index], buf, format_settings); - } - catch (Exception & e) - { - e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); - throw; - } -} - -void JSONStringsEachRowRowInputFormat::syncAfterError() -{ - skipToUnescapedNextLineOrEOF(in); -} - -void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory) -{ - factory.registerInputFormatProcessor("JSONStringsEachRow", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, false); - }); - - factory.registerInputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, true); - }); -} - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h deleted file mode 100644 index ec0a0f7bad9..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h +++ /dev/null @@ -1,54 +0,0 @@ -#pragma once - -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class ReadBuffer; - -/** A stream for reading data in JSONStringsEachRow and JSONStringsEachRowWithNamesAndTypes formats -*/ -class JSONStringsEachRowRowInputFormat : public IRowInputFormat -{ -public: - JSONStringsEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); - - String getName() const override { return "JSONStringsEachRowRowInputFormat"; } - - - void readPrefix() override; - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - bool allowSyncAfterError() const override { return true; } - void syncAfterError() override; - void resetParser() override; - -private: - void addInputColumn(const String & column_name); - void skipEndOfLine(); - void readField(size_t index, MutableColumns & columns); - - const FormatSettings format_settings; - - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; - - using OptionalIndexes = std::vector>; - OptionalIndexes column_indexes_for_input_fields; - - DataTypes data_types; - std::vector read_columns; - std::vector not_seen_columns; - - /// This is for the correct exceptions in skipping unknown fields. - std::vector names_of_columns; - - bool with_names; -}; - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp deleted file mode 100644 index 75007ea236e..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp +++ /dev/null @@ -1,117 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - - -JSONStringsEachRowRowOutputFormat::JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, - const Block & header_, - FormatFactory::WriteCallback callback, - const FormatSettings & settings_, - bool with_names_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) -{ - const auto & sample = getPort(PortKind::Main).getHeader(); - NamesAndTypesList columns(sample.getNamesAndTypesList()); - fields.assign(columns.begin(), columns.end()); -} - - -void JSONStringsEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) -{ - WriteBufferFromOwnString buf; - - type.serializeAsText(column, row_num, buf, settings); - writeJSONString(buf.str(), out, settings); -} - - -void JSONStringsEachRowRowOutputFormat::writeFieldDelimiter() -{ - writeCString(", ", out); -} - - -void JSONStringsEachRowRowOutputFormat::writeRowStartDelimiter() -{ - writeChar('[', out); -} - - -void JSONStringsEachRowRowOutputFormat::writeRowEndDelimiter() -{ - writeCString("]\n", out); -} - -void JSONStringsEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) -{ - writeChar('\n', out); - size_t num_columns = columns.size(); - writeChar('[', out); - for (size_t i = 0; i < num_columns; ++i) - { - if (i != 0) - JSONStringsEachRowRowOutputFormat::writeFieldDelimiter(); - - JSONStringsEachRowRowOutputFormat::writeField(*columns[i], *types[i], row_num); - } - writeCString("]\n", out); -} - -void JSONStringsEachRowRowOutputFormat::writePrefix() -{ - if (with_names) - { - writeChar('[', out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeChar('\"', out); - writeString(fields[i].name, out); - writeChar('\"', out); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n[", out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeJSONString(fields[i].type->getName(), out, settings); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n", out); - } -} - -void JSONStringsEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) -{ - if (with_names) - IRowOutputFormat::consumeTotals(std::move(chunk)); -} - -void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory) -{ - factory.registerOutputFormatProcessor("JSONStringsEachRow", []( - WriteBuffer & buf, - const Block & sample, - FormatFactory::WriteCallback callback, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, callback, format_settings, false); - }); - - factory.registerOutputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( - WriteBuffer &buf, - const Block &sample, - FormatFactory::WriteCallback callback, - const FormatSettings &format_settings) - { - return std::make_shared(buf, sample, callback, format_settings, true); - }); -} - - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h deleted file mode 100644 index 1d43a333da1..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h +++ /dev/null @@ -1,45 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -/** The stream for outputting data in JSON format, by object per line. - * Does not validate UTF-8. - */ -class JSONStringsEachRowRowOutputFormat : public IRowOutputFormat -{ -public: - JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); - - String getName() const override { return "JSONStringsEachRowRowOutputFormat"; } - - void writePrefix() override; - - void writeBeforeTotals() override {} - void writeTotals(const Columns & columns, size_t row_num) override; - void writeAfterTotals() override {} - - void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; - void writeFieldDelimiter() override; - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; - -protected: - void consumeTotals(Chunk) override; - /// No extremes. - void consumeExtremes(Chunk) override {} - -private: - FormatSettings settings; - - NamesAndTypes fields; - - bool with_names; -}; -} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp deleted file mode 100644 index 6ccb315f73f..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp +++ /dev/null @@ -1,93 +0,0 @@ -#include -#include - -#include - - -namespace DB -{ - -JSONStringsRowOutputFormat::JSONStringsRowOutputFormat( - WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : JSONRowOutputFormat(out_, header, callback, settings_) -{ -} - - -void JSONStringsRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) -{ - WriteBufferFromOwnString buf; - - type.serializeAsText(column, row_num, buf, settings); - writeJSONString(buf.str(), *ostr, settings); - ++field_number; -} - - -void JSONStringsRowOutputFormat::writeFieldDelimiter() -{ - writeCString(", ", *ostr); -} - -void JSONStringsRowOutputFormat::writeTotalsFieldDelimiter() -{ - writeCString(",", *ostr); -} - - -void JSONStringsRowOutputFormat::writeRowStartDelimiter() -{ - writeCString("\t\t[", *ostr); -} - - -void JSONStringsRowOutputFormat::writeRowEndDelimiter() -{ - writeChar(']', *ostr); - field_number = 0; - ++row_count; -} - -void JSONStringsRowOutputFormat::writeBeforeTotals() -{ - writeCString(",\n", *ostr); - writeChar('\n', *ostr); - writeCString("\t\"totals\": [", *ostr); -} - -void JSONStringsRowOutputFormat::writeAfterTotals() -{ - writeChar(']', *ostr); -} - -void JSONStringsRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) -{ - writeCString("\t\t\"", *ostr); - writeCString(title, *ostr); - writeCString("\": [", *ostr); - - size_t extremes_columns = columns.size(); - for (size_t i = 0; i < extremes_columns; ++i) - { - if (i != 0) - writeTotalsFieldDelimiter(); - - writeField(*columns[i], *types[i], row_num); - } - - writeChar(']', *ostr); -} - -void registerOutputFormatProcessorJSONStrings(FormatFactory & factory) -{ - factory.registerOutputFormatProcessor("JSONStrings", []( - WriteBuffer & buf, - const Block & sample, - FormatFactory::WriteCallback callback, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, callback, format_settings); - }); -} - -} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h deleted file mode 100644 index b221bc9ee36..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h +++ /dev/null @@ -1,43 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -struct FormatSettings; - -/** The stream for outputting data in the JSONStrings format. - */ -class JSONStringsRowOutputFormat : public JSONRowOutputFormat -{ -public: - JSONStringsRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); - - String getName() const override { return "JSONStringsRowOutputFormat"; } - - void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; - void writeFieldDelimiter() override; - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; - - void writeBeforeTotals() override; - void writeAfterTotals() override; - -protected: - void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; - - void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override - { - return writeField(column, type, row_num); - } - - void writeTotalsFieldDelimiter() override; - -}; - -} diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql deleted file mode 100644 index f461b217fe4..00000000000 --- a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql +++ /dev/null @@ -1,63 +0,0 @@ -DROP TABLE IF EXISTS test_table; -DROP TABLE IF EXISTS test_table_2; -SELECT 1; -/* Check JSONStringsEachRow Output */ -CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; -INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 2; -/* Check Totals */ -SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; -SELECT 3; -/* Check JSONStringsEachRowWithNamesAndTypes Output */ -SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; -SELECT 4; -/* Check Totals */ -SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRowWithNamesAndTypes; -DROP TABLE IF EXISTS test_table; -SELECT 5; -/* Check JSONStringsEachRow Input */ -CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; -INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 6; -/* Check input_format_null_as_default = 1 */ -SET input_format_null_as_default = 1; -INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 7; -/* Check Nested */ -CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; -INSERT INTO test_table_2 FORMAT JSONStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; -SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table_2; -SELECT 8; -/* Check JSONStringsEachRowWithNamesAndTypes Output */ -SET input_format_null_as_default = 0; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 9; -/* Check input_format_null_as_default = 1 */ -SET input_format_null_as_default = 1; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 10; -/* Check Header */ -TRUNCATE TABLE test_table; -SET input_format_skip_unknown_fields = 1; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 11; -TRUNCATE TABLE test_table; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] -SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; -SELECT 12; -/* Check Nested */ -INSERT INTO test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; -SELECT * FROM test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes; - -DROP TABLE IF EXISTS test_table; -DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.reference b/tests/queries/0_stateless/01446_json_strings_each_row.reference new file mode 100644 index 00000000000..84d41095b77 --- /dev/null +++ b/tests/queries/0_stateless/01446_json_strings_each_row.reference @@ -0,0 +1,22 @@ +1 +{"value":"1","name":"a"} +{"value":"2","name":"b"} +{"value":"3","name":"c"} +2 +{"name":"a","c":"1"} +{"name":"b","c":"1"} +{"name":"c","c":"1"} +3 +{"row":{"a":"1"}} +{"progress":{"read_rows":"1","read_bytes":"1","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +4 +{"row":{"a":"1"}} +{"progress":{"read_rows":"1","read_bytes":"1","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +5 +{"v1":"first","v2":"1","v3":"2","v4":"0"} +{"v1":"second","v2":"2","v3":"0","v4":"6"} +6 +{"v1":"first","v2":"1","v3":"2","v4":"0"} +{"v1":"second","v2":"2","v3":"0","v4":"6"} +7 +{"v1":"16","n.id":"[15,16,17]","n.name":"['first','second','third']"} diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.sql b/tests/queries/0_stateless/01446_json_strings_each_row.sql new file mode 100644 index 00000000000..98bd3e3ab47 --- /dev/null +++ b/tests/queries/0_stateless/01446_json_strings_each_row.sql @@ -0,0 +1,38 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; +SELECT 3; +/* Check JSONStringsEachRowWithProgress Output */ +SELECT 1 as a FROM system.one FORMAT JSONStringsEachRowWithProgress; +SELECT 4; +/* Check Totals */ +SELECT 1 as a FROM system.one GROUP BY a WITH TOTALS ORDER BY a FORMAT JSONStringsEachRowWithProgress; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONStringsEachRow {"v1": "first", "v2": "1", "v3": "2", "v4": "NULL"} {"v1": "second", "v2": "2", "v3": "null", "v4": "6"}; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRow {"v1": "first", "v2": "1", "v3": "2", "v4": "ᴺᵁᴸᴸ"} {"v1": "second", "v2": "2", "v3": "null", "v4": "6"}; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONStringsEachRow {"v1": "16", "n.id": "[15, 16, 17]", "n.name": "['first', 'second', 'third']"}; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table_2; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_json_strings.reference b/tests/queries/0_stateless/01447_json_strings.reference new file mode 100644 index 00000000000..ab88e2f3696 --- /dev/null +++ b/tests/queries/0_stateless/01447_json_strings.reference @@ -0,0 +1,43 @@ +{ + "meta": + [ + { + "name": "1", + "type": "UInt8" + }, + { + "name": "'a'", + "type": "String" + }, + { + "name": "[1, 2, 3]", + "type": "Array(UInt8)" + }, + { + "name": "tuple(1, 'a')", + "type": "Tuple(UInt8, String)" + }, + { + "name": "NULL", + "type": "Nullable(Nothing)" + }, + { + "name": "nan", + "type": "Float64" + } + ], + + "data": + [ + { + "1": "1", + "'a'": "a", + "[1, 2, 3]": "[1,2,3]", + "tuple(1, 'a')": "(1,'a')", + "NULL": "ᴺᵁᴸᴸ", + "nan": "nan" + } + ], + + "rows": 1 +} diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_json_strings.sql similarity index 100% rename from tests/queries/0_stateless/01447_JSONStrings.sql rename to tests/queries/0_stateless/01447_json_strings.sql diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.reference b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference similarity index 100% rename from tests/queries/0_stateless/01446_JSONStringsEachRow.reference rename to tests/queries/0_stateless/01448_json_compact_strings_each_row.reference diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql new file mode 100644 index 00000000000..c271de88434 --- /dev/null +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONCompactStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRow; +SELECT 3; +/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT 4; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONCompactStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table_2; +SELECT 8; +/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +SET input_format_null_as_default = 0; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 9; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 10; +/* Check Header */ +TRUNCATE TABLE test_table; +SET input_format_skip_unknown_fields = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 11; +TRUNCATE TABLE test_table; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT 12; +/* Check Nested */ +INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01449_json_compact_strings.reference similarity index 100% rename from tests/queries/0_stateless/01447_JSONStrings.reference rename to tests/queries/0_stateless/01449_json_compact_strings.reference diff --git a/tests/queries/0_stateless/01449_json_compact_strings.sql b/tests/queries/0_stateless/01449_json_compact_strings.sql new file mode 100644 index 00000000000..5b676e30347 --- /dev/null +++ b/tests/queries/0_stateless/01449_json_compact_strings.sql @@ -0,0 +1,10 @@ +SET output_format_write_statistics = 0; + +SELECT + 1, + 'a', + [1, 2, 3], + (1, 'a'), + null, + nan +FORMAT JSONCompactStrings; From 1f908af189d2693f87fa0aec6422ee9767f9958d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 2 Sep 2020 13:05:09 +0800 Subject: [PATCH 132/535] ISSUES-14114 fix create parse failure when mysql nullable primary key --- .../MySQL/InterpretersMySQLDDLQuery.cpp | 86 ++++++++++++------- .../MySQL/tests/gtest_create_rewritten.cpp | 52 ++++++++--- src/Parsers/MySQL/ASTDeclareColumn.cpp | 4 +- 3 files changed, 100 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 461dd997cd1..70916fe386d 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -124,8 +125,37 @@ static NamesAndTypesList getNames(const ASTFunction & expr, const Context & cont return expression->getRequiredColumnsWithTypes(); } +static NamesAndTypesList modifyPrimaryKeysToNonNullable(const NamesAndTypesList & primary_keys, NamesAndTypesList & columns) +{ + /// https://dev.mysql.com/doc/refman/5.7/en/create-table.html#create-table-indexes-keys + /// PRIMARY KEY: + /// A unique index where all key columns must be defined as NOT NULL. + /// If they are not explicitly declared as NOT NULL, MySQL declares them so implicitly (and silently). + /// A table can have only one PRIMARY KEY. The name of a PRIMARY KEY is always PRIMARY, + /// which thus cannot be used as the name for any other kind of index. + NamesAndTypesList non_nullable_primary_keys; + for (const auto & primary_key : primary_keys) + { + if (!primary_key.type->isNullable()) + non_nullable_primary_keys.emplace_back(primary_key); + else + { + non_nullable_primary_keys.emplace_back( + NameAndTypePair(primary_key.name, assert_cast(primary_key.type.get())->getNestedType())); + + for (auto & column : columns) + { + if (column.name == primary_key.name) + column.type = assert_cast(column.type.get())->getNestedType(); + } + } + } + + return non_nullable_primary_keys; +} + static inline std::tuple getKeys( - ASTExpressionList * columns_define, ASTExpressionList * indices_define, const Context & context, const NamesAndTypesList & columns) + ASTExpressionList * columns_define, ASTExpressionList * indices_define, const Context & context, NamesAndTypesList & columns) { NameSet increment_columns; auto keys = makeASTFunction("tuple"); @@ -171,8 +201,9 @@ static inline std::tuple ASTPtr { - ASTPtr column = std::make_shared(column_name); + if (type_max_size <= 1000) + return std::make_shared(column_name); - if (is_nullable) - column = makeASTFunction("assumeNotNull", column); - - return makeASTFunction("intDiv", column, std::make_shared(UInt64(type_max_size / 1000))); + return makeASTFunction("intDiv", std::make_shared(column_name), + std::make_shared(UInt64(type_max_size / 1000))); }; ASTPtr best_partition; @@ -219,16 +249,12 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) WhichDataType which(type); if (which.isNullable()) - { - type = (static_cast(*type)).getNestedType(); - which = WhichDataType(type); - } + throw Exception("LOGICAL ERROR: MySQL primary key must be not null, it is a bug.", ErrorCodes::LOGICAL_ERROR); if (which.isDateOrDateTime()) { /// In any case, date or datetime is always the best partitioning key - ASTPtr res = std::make_shared(primary_key.name); - return makeASTFunction("toYYYYMM", primary_key.type->isNullable() ? makeASTFunction("assumeNotNull", res) : res); + return makeASTFunction("toYYYYMM", std::make_shared(primary_key.name)); } if (type->haveMaximumSizeOfValue() && (!best_size || type->getSizeOfValueInMemory() < best_size)) @@ -236,25 +262,22 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) if (which.isInt8() || which.isUInt8()) { best_size = type->getSizeOfValueInMemory(); - best_partition = std::make_shared(primary_key.name); - - if (primary_key.type->isNullable()) - best_partition = makeASTFunction("assumeNotNull", best_partition); + best_partition = numbers_partition(primary_key.name, std::numeric_limits::max()); } else if (which.isInt16() || which.isUInt16()) { best_size = type->getSizeOfValueInMemory(); - best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits::max()); + best_partition = numbers_partition(primary_key.name, std::numeric_limits::max()); } else if (which.isInt32() || which.isUInt32()) { best_size = type->getSizeOfValueInMemory(); - best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits::max()); + best_partition = numbers_partition(primary_key.name, std::numeric_limits::max()); } else if (which.isInt64() || which.isUInt64()) { best_size = type->getSizeOfValueInMemory(); - best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits::max()); + best_partition = numbers_partition(primary_key.name, std::numeric_limits::max()); } } } @@ -266,12 +289,12 @@ static ASTPtr getOrderByPolicy( const NamesAndTypesList & primary_keys, const NamesAndTypesList & unique_keys, const NamesAndTypesList & keys, const NameSet & increment_columns) { NameSet order_by_columns_set; - std::deque> order_by_columns_list; + std::deque order_by_columns_list; const auto & add_order_by_expression = [&](const NamesAndTypesList & names_and_types) { - std::vector increment_keys; - std::vector non_increment_keys; + NamesAndTypesList increment_keys; + NamesAndTypesList non_increment_keys; for (const auto & [name, type] : names_and_types) { @@ -280,13 +303,13 @@ static ASTPtr getOrderByPolicy( if (increment_columns.count(name)) { - increment_keys.emplace_back(name); order_by_columns_set.emplace(name); + increment_keys.emplace_back(NameAndTypePair(name, type)); } else { order_by_columns_set.emplace(name); - non_increment_keys.emplace_back(name); + non_increment_keys.emplace_back(NameAndTypePair(name, type)); } } @@ -305,8 +328,13 @@ static ASTPtr getOrderByPolicy( for (const auto & order_by_columns : order_by_columns_list) { - for (const auto & order_by_column : order_by_columns) - order_by_expression->arguments->children.emplace_back(std::make_shared(order_by_column)); + for (const auto & [name, type] : order_by_columns) + { + order_by_expression->arguments->children.emplace_back(std::make_shared(name)); + + if (type->isNullable()) + order_by_expression->arguments->children.back() = makeASTFunction("assumeNotNull", order_by_expression->arguments->children.back()); + } } return order_by_expression; diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index b9bfe28ea1b..b940e4e0c95 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -103,21 +103,12 @@ TEST(MySQLCreateRewritten, PartitionPolicy) {"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"} }; - const auto & replace_string = [](const String & str, const String & old_str, const String & new_str) - { - String new_string = str; - size_t pos = new_string.find(old_str); - if (pos != std::string::npos) - new_string = new_string.replace(pos, old_str.size(), new_str); - return new_string; - }; - for (const auto & [test_type, mapped_type, partition_policy] : test_types) { EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " PRIMARY KEY)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Nullable(" + mapped_type + "), `_sign` Int8() MATERIALIZED 1, " - "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + replace_string(partition_policy, "key", "assumeNotNull(key)") + " ORDER BY tuple(key)"); + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " + "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY KEY)", context_holder.context)), @@ -126,6 +117,45 @@ TEST(MySQLCreateRewritten, PartitionPolicy) } } +TEST(MySQLCreateRewritten, OrderbyPolicy) +{ + tryRegisterFunctions(); + const auto & context_holder = getContext(); + + std::vector> test_types + { + {"TINYINT", "Int8", " PARTITION BY key"}, {"SMALLINT", "Int16", " PARTITION BY intDiv(key, 65)"}, + {"MEDIUMINT", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"INT", "Int32", " PARTITION BY intDiv(key, 4294967)"}, + {"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"}, + {"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""}, + {"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"}, + {"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"} + }; + + for (const auto & [test_type, mapped_type, partition_policy] : test_types) + { + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " PRIMARY KEY, `key2` " + test_type + " UNIQUE KEY)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` Nullable(" + mapped_type + "), `_sign` Int8() MATERIALIZED 1, " + "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, assumeNotNull(key2))"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY KEY, `key2` " + test_type + " NOT NULL UNIQUE KEY)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " + "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " KEY UNIQUE KEY)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " + "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + ", `key2` " + test_type + " UNIQUE KEY, PRIMARY KEY(`key`, `key2`))", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " + "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)"); + } +} + TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey) { tryRegisterFunctions(); diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index 56a92291f06..6d21f934858 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -46,10 +46,10 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, OptionDescribe("DEFAULT", "default", std::make_unique()), OptionDescribe("ON UPDATE", "on_update", std::make_unique()), OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_unique()), - OptionDescribe("UNIQUE", "unique_key", std::make_unique()), OptionDescribe("UNIQUE KEY", "unique_key", std::make_unique()), - OptionDescribe("KEY", "primary_key", std::make_unique()), OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), + OptionDescribe("UNIQUE", "unique_key", std::make_unique()), + OptionDescribe("KEY", "primary_key", std::make_unique()), OptionDescribe("COMMENT", "comment", std::make_unique()), OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), OptionDescribe("COLLATE", "collate", std::make_unique()), From be925f8d9c004aa8de4c6b1548916c7e2bd719f1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 29 Aug 2020 13:33:46 +0800 Subject: [PATCH 133/535] Introduce columns transformers. --- .../TranslateQualifiedNamesVisitor.cpp | 19 ++- src/Parsers/ASTAsterisk.cpp | 7 +- src/Parsers/ASTAsterisk.h | 3 + src/Parsers/ASTColumnsMatcher.cpp | 7 +- src/Parsers/ASTColumnsMatcher.h | 1 + src/Parsers/ASTColumnsTransformers.cpp | 158 ++++++++++++++++++ src/Parsers/ASTColumnsTransformers.h | 85 ++++++++++ src/Parsers/ASTQualifiedAsterisk.cpp | 5 + src/Parsers/ASTQualifiedAsterisk.h | 1 + src/Parsers/ExpressionElementParsers.cpp | 126 +++++++++++++- src/Parsers/ExpressionElementParsers.h | 9 + src/Parsers/ya.make | 1 + .../01470_columns_transformers.reference | 63 +++++++ .../01470_columns_transformers.sql | 36 ++++ 14 files changed, 515 insertions(+), 6 deletions(-) create mode 100644 src/Parsers/ASTColumnsTransformers.cpp create mode 100644 src/Parsers/ASTColumnsTransformers.h create mode 100644 tests/queries/0_stateless/01470_columns_transformers.reference create mode 100644 tests/queries/0_stateless/01470_columns_transformers.sql diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index fcc4948d88a..e28997f0ad6 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB @@ -135,8 +136,8 @@ void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, D void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk &, const ASTPtr & ast, Data & data) { - if (ast->children.size() != 1) - throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR); + if (ast->children.empty()) + throw Exception("Logical error: qualified asterisk must have children", ErrorCodes::LOGICAL_ERROR); auto & ident = ast->children[0]; @@ -242,6 +243,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt first_table = false; } + for (const auto & transformer : asterisk->children) + { + IASTColumnsTransformer::transform(transformer, node.children); + } } else if (const auto * asterisk_pattern = child->as()) { @@ -258,6 +263,11 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt first_table = false; } + // ColumnsMatcher's transformers start to appear at child 1 + for (auto it = asterisk_pattern->children.begin() + 1; it != asterisk_pattern->children.end(); ++it) + { + IASTColumnsTransformer::transform(*it, node.children); + } } else if (const auto * qualified_asterisk = child->as()) { @@ -274,6 +284,11 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt break; } } + // QualifiedAsterisk's transformers start to appear at child 1 + for (auto it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it) + { + IASTColumnsTransformer::transform(*it, node.children); + } } else node.children.emplace_back(child); diff --git a/src/Parsers/ASTAsterisk.cpp b/src/Parsers/ASTAsterisk.cpp index 9f38b955d00..95a63586685 100644 --- a/src/Parsers/ASTAsterisk.cpp +++ b/src/Parsers/ASTAsterisk.cpp @@ -13,9 +13,14 @@ ASTPtr ASTAsterisk::clone() const void ASTAsterisk::appendColumnName(WriteBuffer & ostr) const { ostr.write('*'); } -void ASTAsterisk::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +void ASTAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { settings.ostr << "*"; + for (const auto & child : children) + { + settings.ostr << ' '; + child->formatImpl(settings, state, frame); + } } } diff --git a/src/Parsers/ASTAsterisk.h b/src/Parsers/ASTAsterisk.h index 620394ec65a..9c4c9a2df6d 100644 --- a/src/Parsers/ASTAsterisk.h +++ b/src/Parsers/ASTAsterisk.h @@ -9,6 +9,9 @@ namespace DB struct AsteriskSemantic; struct AsteriskSemanticImpl; +/** SELECT * is expanded to all visible columns of the source table. + * Optional transformers can be attached to further manipulate these expanded columns. + */ class ASTAsterisk : public IAST { public: diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index b6eb4889a09..191ca52c0e8 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -28,10 +28,15 @@ void ASTColumnsMatcher::updateTreeHashImpl(SipHash & hash_state) const IAST::updateTreeHashImpl(hash_state); } -void ASTColumnsMatcher::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +void ASTColumnsMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(" << quoteString(original_pattern) << ")"; + for (ASTs::const_iterator it = children.begin() + 1; it != children.end(); ++it) + { + settings.ostr << ' '; + (*it)->formatImpl(settings, state, frame); + } } void ASTColumnsMatcher::setPattern(String pattern) diff --git a/src/Parsers/ASTColumnsMatcher.h b/src/Parsers/ASTColumnsMatcher.h index 3fa85769712..47a9b86a519 100644 --- a/src/Parsers/ASTColumnsMatcher.h +++ b/src/Parsers/ASTColumnsMatcher.h @@ -23,6 +23,7 @@ struct AsteriskSemanticImpl; /** SELECT COLUMNS('regexp') is expanded to multiple columns like * (asterisk). + * Optional transformers can be attached to further manipulate these expanded columns. */ class ASTColumnsMatcher : public IAST { diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp new file mode 100644 index 00000000000..29bc8420066 --- /dev/null +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -0,0 +1,158 @@ +#include "ASTColumnsTransformers.h" +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes) +{ + if (const auto * apply = transformer->as()) + { + apply->transform(nodes); + } + else if (const auto * except = transformer->as()) + { + except->transform(nodes); + } + else if (const auto * replace = transformer->as()) + { + replace->transform(nodes); + } +} + +void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << "(" << func_name << ")"; +} + +void ASTColumnsApplyTransformer::transform(ASTs & nodes) const +{ + for (auto & column : nodes) + { + column = makeASTFunction(func_name, column); + } +} + +void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (settings.hilite ? hilite_none : "") << "("; + + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + { + if (it != children.begin()) + { + settings.ostr << ", "; + } + (*it)->formatImpl(settings, state, frame); + } + + settings.ostr << ")"; +} + +void ASTColumnsExceptTransformer::transform(ASTs & nodes) const +{ + nodes.erase( + std::remove_if( + nodes.begin(), + nodes.end(), + [this](const ASTPtr & node_child) + { + if (const auto * id = node_child->as()) + { + for (const auto & except_child : children) + { + if (except_child->as().name == id->shortName()) + return true; + } + } + return false; + }), + nodes.end()); +} + +void ASTColumnsReplaceTransformer::Replacement::formatImpl( + const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + expr->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "") << name; +} + +void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (settings.hilite ? hilite_none : "") << "("; + + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + { + if (it != children.begin()) + { + settings.ostr << ", "; + } + (*it)->formatImpl(settings, state, frame); + } + + settings.ostr << ")"; +} + +void ASTColumnsReplaceTransformer::replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name) +{ + for (auto & child : node->children) + { + if (const auto * id = child->as()) + { + if (id->shortName() == name) + child = replacement; + } + else + replaceChildren(child, replacement, name); + } +} + +void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const +{ + std::map replace_map; + for (const auto & replace_child : children) + { + auto & replacement = replace_child->as(); + if (replace_map.find(replacement.name) != replace_map.end()) + throw Exception( + "Expressions in columns transformer REPLACE should not contain the same replacement more than once", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + replace_map.emplace(replacement.name, replacement.expr); + } + + for (auto & column : nodes) + { + if (const auto * id = column->as()) + { + auto replace_it = replace_map.find(id->shortName()); + if (replace_it != replace_map.end()) + { + column = replace_it->second; + column->setAlias(replace_it->first); + } + } + else if (auto * ast_with_alias = dynamic_cast(column.get())) + { + auto replace_it = replace_map.find(ast_with_alias->alias); + if (replace_it != replace_map.end()) + { + auto new_ast = replace_it->second->clone(); + ast_with_alias->alias = ""; // remove the old alias as it's useless after replace transformation + replaceChildren(new_ast, column, replace_it->first); + column = new_ast; + column->setAlias(replace_it->first); + } + } + } +} + +} diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h new file mode 100644 index 00000000000..ddf0d70dc35 --- /dev/null +++ b/src/Parsers/ASTColumnsTransformers.h @@ -0,0 +1,85 @@ +#pragma once + +#include + +namespace DB +{ +class IASTColumnsTransformer : public IAST +{ +public: + virtual void transform(ASTs & nodes) const = 0; + static void transform(const ASTPtr & transformer, ASTs & nodes); +}; + +class ASTColumnsApplyTransformer : public IASTColumnsTransformer +{ +public: + String getID(char) const override { return "ColumnsApplyTransformer"; } + ASTPtr clone() const override + { + auto res = std::make_shared(*this); + return res; + } + void transform(ASTs & nodes) const override; + String func_name; + +protected: + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +class ASTColumnsExceptTransformer : public IASTColumnsTransformer +{ +public: + String getID(char) const override { return "ColumnsExceptTransformer"; } + ASTPtr clone() const override + { + auto clone = std::make_shared(*this); + clone->cloneChildren(); + return clone; + } + void transform(ASTs & nodes) const override; + +protected: + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +class ASTColumnsReplaceTransformer : public IASTColumnsTransformer +{ +public: + class Replacement : public IAST + { + public: + String getID(char) const override { return "ColumnsReplaceTransformer::Replacement"; } + ASTPtr clone() const override + { + auto replacement = std::make_shared(*this); + replacement->name = name; + replacement->expr = expr->clone(); + replacement->children.push_back(replacement->expr); + return replacement; + } + + String name; + ASTPtr expr; + + protected: + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + }; + + String getID(char) const override { return "ColumnsReplaceTransformer"; } + ASTPtr clone() const override + { + auto clone = std::make_shared(*this); + clone->cloneChildren(); + return clone; + } + void transform(ASTs & nodes) const override; + +protected: + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + +private: + static void replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name); +}; + +} diff --git a/src/Parsers/ASTQualifiedAsterisk.cpp b/src/Parsers/ASTQualifiedAsterisk.cpp index cbde6d4f15d..0cda01cecac 100644 --- a/src/Parsers/ASTQualifiedAsterisk.cpp +++ b/src/Parsers/ASTQualifiedAsterisk.cpp @@ -16,6 +16,11 @@ void ASTQualifiedAsterisk::formatImpl(const FormatSettings & settings, FormatSta const auto & qualifier = children.at(0); qualifier->formatImpl(settings, state, frame); settings.ostr << ".*"; + for (ASTs::const_iterator it = children.begin() + 1; it != children.end(); ++it) + { + settings.ostr << ' '; + (*it)->formatImpl(settings, state, frame); + } } } diff --git a/src/Parsers/ASTQualifiedAsterisk.h b/src/Parsers/ASTQualifiedAsterisk.h index 2cead406647..2c3689d0ace 100644 --- a/src/Parsers/ASTQualifiedAsterisk.h +++ b/src/Parsers/ASTQualifiedAsterisk.h @@ -11,6 +11,7 @@ struct AsteriskSemanticImpl; /** Something like t.* * It will have qualifier as its child ASTIdentifier. + * Optional transformers can be attached to further manipulate these expanded columns. */ class ASTQualifiedAsterisk : public IAST { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e24bb9c4129..eee46c599b1 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -20,7 +20,9 @@ #include #include #include +#include +#include #include #include #include @@ -1172,17 +1174,131 @@ bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expect auto res = std::make_shared(); res->setPattern(regex_node->as().value.get()); res->children.push_back(regex_node); + ParserColumnsTransformers transformers_p; + ASTPtr transformer; + while (transformers_p.parse(pos, transformer, expected)) + { + res->children.push_back(transformer); + } node = std::move(res); return true; } -bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &) +bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword apply("APPLY"); + ParserKeyword except("EXCEPT"); + ParserKeyword replace("REPLACE"); + ParserKeyword as("AS"); + + if (apply.ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + String func_name; + if (!parseIdentifierOrStringLiteral(pos, expected, func_name)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto res = std::make_shared(); + res->func_name = func_name; + node = std::move(res); + return true; + } + else if (except.ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + ASTs identifiers; + auto parse_id = [&identifiers, &pos, &expected] + { + ASTPtr identifier; + if (!ParserIdentifier().parse(pos, identifier, expected)) + return false; + + identifiers.emplace_back(std::move(identifier)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_id, false)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto res = std::make_shared(); + res->children = std::move(identifiers); + node = std::move(res); + return true; + } + else if (replace.ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + ASTs replacements; + ParserExpression element_p; + ParserIdentifier ident_p; + auto parse_id = [&] + { + ASTPtr expr; + + if (!element_p.parse(pos, expr, expected)) + return false; + if (!as.ignore(pos, expected)) + return false; + + ASTPtr ident; + if (!ident_p.parse(pos, ident, expected)) + return false; + + auto replacement = std::make_shared(); + replacement->name = getIdentifierName(ident); + replacement->expr = std::move(expr); + replacements.emplace_back(std::move(replacement)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_id, false)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto res = std::make_shared(); + res->children = std::move(replacements); + node = std::move(res); + return true; + } + + return false; +} + + +bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (pos->type == TokenType::Asterisk) { ++pos; - node = std::make_shared(); + auto asterisk = std::make_shared(); + ParserColumnsTransformers transformers_p; + ASTPtr transformer; + while (transformers_p.parse(pos, transformer, expected)) + { + asterisk->children.push_back(transformer); + } + node = asterisk; return true; } return false; @@ -1204,6 +1320,12 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp auto res = std::make_shared(); res->children.push_back(node); + ParserColumnsTransformers transformers_p; + ASTPtr transformer; + while (transformers_p.parse(pos, transformer, expected)) + { + res->children.push_back(transformer); + } node = std::move(res); return true; } diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 13e3febcebe..702d757761a 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -88,6 +88,15 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/** *, t.*, db.table.*, COLUMNS('') APPLY(...) or EXCEPT(...) or REPLACE(...) + */ +class ParserColumnsTransformers : public IParserBase +{ +protected: + const char * getName() const override { return "COLUMNS transformers"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + /** A function, for example, f(x, y + 1, g(z)). * Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function. * Or a parametric aggregate function: quantile(0.9)(x + y). diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 1b03bae100b..b6ef322e426 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -10,6 +10,7 @@ SRCS( ASTAsterisk.cpp ASTColumnDeclaration.cpp ASTColumnsMatcher.cpp + ASTColumnsTransformers.cpp ASTConstraintDeclaration.cpp ASTCreateQuery.cpp ASTCreateQuotaQuery.cpp diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference new file mode 100644 index 00000000000..595d99b917f --- /dev/null +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -0,0 +1,63 @@ +220 18 347 +110 9 173.5 +1970-04-11 1970-01-11 1970-11-21 +2 3 +1 2 +18 347 +110 173.5 +1970-04-11 1970-01-11 1970-11-21 +222 18 347 +111 11 173.5 +1970-04-11 1970-01-11 1970-11-21 +SELECT + sum(i), + sum(j), + sum(k) +FROM columns_transformers +SELECT + avg(i), + avg(j), + avg(k) +FROM columns_transformers +SELECT + toDate(any(i)), + toDate(any(j)), + toDate(any(k)) +FROM columns_transformers AS a +SELECT + length(toString(j)), + length(toString(k)) +FROM columns_transformers +SELECT + sum(j), + sum(k) +FROM columns_transformers +SELECT + avg(i), + avg(k) +FROM columns_transformers +SELECT + toDate(any(i)), + toDate(any(j)), + toDate(any(k)) +FROM columns_transformers AS a +SELECT + sum(i + 1 AS i), + sum(j), + sum(k) +FROM columns_transformers +SELECT + avg(i + 1 AS i), + avg(j + 2 AS j), + avg(k) +FROM columns_transformers +SELECT + toDate(any(i)), + toDate(any(j)), + toDate(any(k)) +FROM columns_transformers AS a +SELECT + (i + 1) + 1 AS i, + j, + k +FROM columns_transformers diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql new file mode 100644 index 00000000000..de6a1a89d81 --- /dev/null +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -0,0 +1,36 @@ +DROP TABLE IF EXISTS columns_transformers; + +CREATE TABLE columns_transformers (i Int64, j Int16, k Int64) Engine=TinyLog; +INSERT INTO columns_transformers VALUES (100, 10, 324), (120, 8, 23); + +SELECT * APPLY(sum) from columns_transformers; +SELECT columns_transformers.* APPLY(avg) from columns_transformers; +SELECT a.* APPLY(toDate) APPLY(any) from columns_transformers a; +SELECT COLUMNS('[jk]') APPLY(toString) APPLY(length) from columns_transformers; + +SELECT * EXCEPT(i) APPLY(sum) from columns_transformers; +SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers; +-- EXCEPT after APPLY will not match anything +SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; + +SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; +SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; +SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 } +-- REPLACE after APPLY will not match anything +SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; + +EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers; +EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers; +EXPLAIN SYNTAX SELECT a.* APPLY(toDate) APPLY(any) from columns_transformers a; +EXPLAIN SYNTAX SELECT COLUMNS('[jk]') APPLY(toString) APPLY(length) from columns_transformers; +EXPLAIN SYNTAX SELECT * EXCEPT(i) APPLY(sum) from columns_transformers; +EXPLAIN SYNTAX SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers; +EXPLAIN SYNTAX SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; +EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; +EXPLAIN SYNTAX SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; +EXPLAIN SYNTAX SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; + +-- Multiple REPLACE in a row +EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) REPLACE(i + 1 AS i) from columns_transformers; + +DROP TABLE columns_transformers; From 13fdcfada9f34f693595b117063caac156b9b47b Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 2 Sep 2020 15:41:09 +0800 Subject: [PATCH 134/535] Try to fix query_event_with_empty_transaction failed --- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index eb3b0cdda4f..f8111ae9508 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -339,8 +339,6 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name mysql_cursor.execute("SHOW MASTER STATUS") (uuid, seqs) = mysql_cursor.fetchall()[0]["Executed_Gtid_Set"].split(":") (seq_begin, seq_end) = seqs.split("-") - assert int(seq_begin) == 1 - assert int(seq_end) == 3 next_gtid = uuid + ":" + str(int(seq_end) + 1) mysql_node.query("SET gtid_next='" + next_gtid + "'") mysql_node.query("BEGIN") From 6bd753d85d0a87a965f46317305348bdf7ec8556 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 28 Aug 2020 22:07:14 +0800 Subject: [PATCH 135/535] TableFunction view. --- src/Interpreters/QueryNormalizer.cpp | 8 ++++ src/Parsers/ASTFunction.cpp | 13 ++++++ src/Parsers/ASTFunction.h | 1 + src/Parsers/ExpressionElementParsers.cpp | 32 +++++++++++++ src/Storages/StorageView.cpp | 8 +++- src/TableFunctions/TableFunctionView.cpp | 45 +++++++++++++++++++ src/TableFunctions/TableFunctionView.h | 27 +++++++++++ src/TableFunctions/registerTableFunctions.cpp | 2 + src/TableFunctions/registerTableFunctions.h | 2 + src/TableFunctions/ya.make | 1 + .../01415_table_function_view.reference | 10 +++++ .../0_stateless/01415_table_function_view.sql | 5 +++ 12 files changed, 153 insertions(+), 1 deletion(-) create mode 100644 src/TableFunctions/TableFunctionView.cpp create mode 100644 src/TableFunctions/TableFunctionView.h create mode 100644 tests/queries/0_stateless/01415_table_function_view.reference create mode 100644 tests/queries/0_stateless/01415_table_function_view.sql diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 324c401eb8a..59233218a50 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes extern const int TOO_DEEP_AST; extern const int CYCLIC_ALIASES; extern const int UNKNOWN_QUERY_PARAMETER; + extern const int BAD_ARGUMENTS; } @@ -151,6 +152,13 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data) { if (const auto * func_node = node->as()) { + if (func_node->query) + { + if (func_node->name != "view") + throw Exception("Query argument can only be used in the `view` TableFunction", ErrorCodes::BAD_ARGUMENTS); + /// Don't go into query argument. + return; + } /// We skip the first argument. We also assume that the lambda function can not have parameters. size_t first_pos = 0; if (func_node->name == "lambda") diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index e8e6efc7fd9..07429c8104f 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -48,6 +48,7 @@ ASTPtr ASTFunction::clone() const auto res = std::make_shared(*this); res->children.clear(); + if (query) { res->query = query->clone(); res->children.push_back(res->query); } if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); } if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); } @@ -118,6 +119,18 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format nested_need_parens.need_parens = true; nested_dont_need_parens.need_parens = false; + if (query) + { + std::string nl_or_nothing = settings.one_line ? "" : "\n"; + std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); + settings.ostr << (settings.hilite ? hilite_function : "") << name << "(" << nl_or_nothing; + FormatStateStacked frame_nested = frame; + frame_nested.need_parens = false; + ++frame_nested.indent; + query->formatImpl(settings, state, frame_nested); + settings.ostr << nl_or_nothing << indent_str << ")"; + return; + } /// Should this function to be written as operator? bool written = false; if (arguments && !parameters) diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 127f50ee586..b94614426d8 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -13,6 +13,7 @@ class ASTFunction : public ASTWithAlias { public: String name; + ASTPtr query; // It's possible for a function to accept a query as its only argument. ASTPtr arguments; /// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'. ASTPtr parameters; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e24bb9c4129..149d0195dff 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -18,6 +18,8 @@ #include #include #include +#include +#include #include #include @@ -217,10 +219,12 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserIdentifier id_parser; ParserKeyword distinct("DISTINCT"); ParserExpressionList contents(false); + ParserSelectWithUnionQuery select; bool has_distinct_modifier = false; ASTPtr identifier; + ASTPtr query; ASTPtr expr_list_args; ASTPtr expr_list_params; @@ -231,8 +235,36 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; ++pos; + if (distinct.ignore(pos, expected)) has_distinct_modifier = true; + else + { + auto old_pos = pos; + auto maybe_an_subquery = pos->type == TokenType::OpeningRoundBracket; + + if (select.parse(pos, query, expected)) + { + auto & select_ast = query->as(); + if (select_ast.list_of_selects->children.size() == 1 && maybe_an_subquery) + { + // It's an subquery. Bail out. + pos = old_pos; + } + else + { + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + auto function_node = std::make_shared(); + tryGetIdentifierNameInto(identifier, function_node->name); + function_node->query = query; + function_node->children.push_back(function_node->query); + node = function_node; + return true; + } + } + } const char * contents_begin = pos->begin; if (!contents.parse(pos, expr_list_args, expected)) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 1a95b7ea21f..4b7733c1cd2 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -104,7 +104,13 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_ ASTTableExpression * table_expression = getFirstTableExpression(outer_query); if (!table_expression->database_and_table_name) - throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); + { + // If it's a view table function, add a fake db.table name. + if (table_expression->table_function && table_expression->table_function->as()->name == "view") + table_expression->database_and_table_name = std::make_shared("__view"); + else + throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); + } DatabaseAndTableWithAlias db_table(table_expression->database_and_table_name); String alias = db_table.alias.empty() ? db_table.table : db_table.alias; diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp new file mode 100644 index 00000000000..6166fa56f47 --- /dev/null +++ b/src/TableFunctions/TableFunctionView.cpp @@ -0,0 +1,45 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerTableFunctions.h" + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +StoragePtr TableFunctionView::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + if (const auto * function = ast_function->as()) + { + if (function->query) + { + if (auto * select = function->query->as()) + { + auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->query, context); + auto columns = ColumnsDescription(sample.getNamesAndTypesList()); + ASTCreateQuery create; + create.select = select; + auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); + res->startup(); + return res; + } + } + } + throw Exception("Table function '" + getName() + "' requires a query argument.", ErrorCodes::BAD_ARGUMENTS); +} + +void registerTableFunctionView(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/TableFunctions/TableFunctionView.h b/src/TableFunctions/TableFunctionView.h new file mode 100644 index 00000000000..49f51823735 --- /dev/null +++ b/src/TableFunctions/TableFunctionView.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/* view(query) + * Turning subquery into a table. + * Useful for passing subquery around. + */ +class TableFunctionView : public ITableFunction +{ +public: + static constexpr auto name = "view"; + std::string getName() const override { return name; } +private: + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + const char * getStorageTypeName() const override { return "View"; } + + UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; +}; + + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index d312fa2085d..25a495a9185 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -30,6 +30,8 @@ void registerTableFunctions() registerTableFunctionODBC(factory); registerTableFunctionJDBC(factory); + registerTableFunctionView(factory); + #if USE_MYSQL registerTableFunctionMySQL(factory); #endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index a695c1926a0..8ff64a22fea 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -30,6 +30,8 @@ void registerTableFunctionHDFS(TableFunctionFactory & factory); void registerTableFunctionODBC(TableFunctionFactory & factory); void registerTableFunctionJDBC(TableFunctionFactory & factory); +void registerTableFunctionView(TableFunctionFactory & factory); + #if USE_MYSQL void registerTableFunctionMySQL(TableFunctionFactory & factory); #endif diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index 3f73df7e3e2..e87c96073bd 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -21,6 +21,7 @@ SRCS( TableFunctionRemote.cpp TableFunctionURL.cpp TableFunctionValues.cpp + TableFunctionView.cpp TableFunctionZeros.cpp ) diff --git a/tests/queries/0_stateless/01415_table_function_view.reference b/tests/queries/0_stateless/01415_table_function_view.reference new file mode 100644 index 00000000000..2b5eef0300e --- /dev/null +++ b/tests/queries/0_stateless/01415_table_function_view.reference @@ -0,0 +1,10 @@ +1 +1 +SELECT `1` +FROM view( + SELECT 1 +) +SELECT `1` +FROM remote(\'127.0.0.1\', view( + SELECT 1 +)) diff --git a/tests/queries/0_stateless/01415_table_function_view.sql b/tests/queries/0_stateless/01415_table_function_view.sql new file mode 100644 index 00000000000..0beeb64c02d --- /dev/null +++ b/tests/queries/0_stateless/01415_table_function_view.sql @@ -0,0 +1,5 @@ +SELECT * FROM view(SELECT 1); +SELECT * FROM remote('127.0.0.1', view(SELECT 1)); + +EXPLAIN SYNTAX SELECT * FROM view(SELECT 1); +EXPLAIN SYNTAX SELECT * FROM remote('127.0.0.1', view(SELECT 1)); From 45e54f81c7d0fe7f8f440a1a9253b64a3911a5f8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 2 Sep 2020 11:07:46 +0300 Subject: [PATCH 136/535] better resolv.conf, add DNSCacheUpdater logs --- programs/server/Server.cpp | 1 + src/Interpreters/DNSCacheUpdater.cpp | 1 + tests/integration/helpers/cluster.py | 9 +++++++-- tests/integration/test_host_ip_change/test.py | 9 +++++++-- 4 files changed, 16 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3a975325851..f24ba444203 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -716,6 +716,7 @@ int Server::main(const std::vector & /*args*/) { /// Disable DNS caching at all DNSResolver::instance().setDisableCacheFlag(); + LOG_DEBUG(log, "DNS caching disabled"); } else { diff --git a/src/Interpreters/DNSCacheUpdater.cpp b/src/Interpreters/DNSCacheUpdater.cpp index e5a97dc76d9..248c0ffa4dd 100644 --- a/src/Interpreters/DNSCacheUpdater.cpp +++ b/src/Interpreters/DNSCacheUpdater.cpp @@ -42,6 +42,7 @@ void DNSCacheUpdater::run() void DNSCacheUpdater::start() { + LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "Update period {} seconds", update_period_seconds); task_handle->activateAndSchedule(); } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 88a2611774a..11c08312c72 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -727,6 +727,11 @@ services: - {env_file} security_opt: - label:disable + dns_opt: + - attempts:2 + - timeout:1 + - inet6 + - rotate {networks} {app_net} {ipv4_address} @@ -740,8 +745,8 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, - with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, - base_configs_dir, server_bin_path, odbc_bridge_bin_path, + with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, + with_redis, with_minio, with_cassandra, base_configs_dir, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag="latest", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None): diff --git a/tests/integration/test_host_ip_change/test.py b/tests/integration/test_host_ip_change/test.py index e3e8c08c848..4bc07e0bf63 100644 --- a/tests/integration/test_host_ip_change/test.py +++ b/tests/integration/test_host_ip_change/test.py @@ -111,8 +111,13 @@ def test_ip_change_update_dns_cache(cluster_with_dns_cache_update): # Check that data is placed on node3 assert node3.query("SELECT count(*) from test_table_update") == "6\n" - result = node4.exec_in_container(["bash", "-c", "/usr/bin/host node3"]) - print("HOST RESULT %s", result) + curl_result = node4.exec_in_container(["bash", "-c", "curl -s 'node3:8123'"]) + assert curl_result == 'Ok.\n' + cat_resolv = node4.exec_in_container(["bash", "-c", "cat /etc/resolv.conf"]) + print("RESOLV {}".format(cat_resolv)) + + + assert node4.query("SELECT * FROM remote('node3', 'system', 'one')") == "0\n" # Because of DNS cache update, ip of node3 would be updated assert_eq_with_retry(node4, "SELECT count(*) from test_table_update", "6", sleep_time=3) From b08056fa8c0f84670bab96b5643dd36850db0d8a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 11:18:50 +0300 Subject: [PATCH 137/535] Better selection of Merges with TTL --- src/Storages/MergeTree/MergeList.cpp | 2 ++ src/Storages/MergeTree/MergeList.h | 2 ++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 32 +++++++++++++++---- .../MergeTree/MergeTreeDataMergerMutator.h | 7 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++- src/Storages/MergeTree/MergeType.cpp | 27 ++++++++++++++++ src/Storages/MergeTree/MergeType.h | 17 ++++++++++ .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 13 ++++++++ .../MergeTree/ReplicatedMergeTreeLogEntry.h | 2 ++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 11 +++++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 9 +++++- src/Storages/StorageMergeTree.cpp | 10 ++++-- src/Storages/StorageReplicatedMergeTree.cpp | 31 +++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 3 +- src/Storages/System/StorageSystemMerges.cpp | 2 ++ 15 files changed, 145 insertions(+), 27 deletions(-) create mode 100644 src/Storages/MergeTree/MergeType.cpp create mode 100644 src/Storages/MergeTree/MergeType.h diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index e9d955f5395..5e7b7046c85 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -21,6 +21,7 @@ MergeListElement::MergeListElement(const std::string & database_, const std::str , result_data_version{future_part.part_info.getDataVersion()} , num_parts{future_part.parts.size()} , thread_id{getThreadId()} + , merge_type{toString(future_part.merge_type)} { for (const auto & source_part : future_part.parts) { @@ -70,6 +71,7 @@ MergeInfo MergeListElement::getInfo() const res.columns_written = columns_written.load(std::memory_order_relaxed); res.memory_usage = memory_tracker.get(); res.thread_id = thread_id; + res.merge_type = merge_type; for (const auto & source_part_name : source_part_names) res.source_part_names.emplace_back(source_part_name); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 4ee8a75a868..e6ae0407ec0 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -45,6 +45,7 @@ struct MergeInfo UInt64 columns_written; UInt64 memory_usage; UInt64 thread_id; + std::string merge_type; }; struct FutureMergedMutatedPart; @@ -88,6 +89,7 @@ struct MergeListElement : boost::noncopyable UInt64 thread_id; + const std::string merge_type; MergeListElement(const std::string & database, const std::string & table, const FutureMergedMutatedPart & future_part); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 673ad02bfb6..a0ab7866402 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -158,15 +158,15 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(MergeType merge_type) { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); - return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1); /// 1 is current thread + return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1, merge_type); /// 1 is current thread } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type) { if (pool_used > pool_size) throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); @@ -178,14 +178,21 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz /// One entry is probably the entry where this function is executed. /// This will protect from bad settings. + + size_t lowering_setting; + if (merge_type == MergeType::TTL_DELETE) + lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl; + else + lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge; + UInt64 max_size = 0; - if (pool_used <= 1 || free_entries >= data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge) + if (pool_used <= 1 || free_entries >= lowering_setting) max_size = data_settings->max_bytes_to_merge_at_max_space_in_pool; else max_size = interpolateExponential( data_settings->max_bytes_to_merge_at_min_space_in_pool, data_settings->max_bytes_to_merge_at_max_space_in_pool, - static_cast(free_entries) / data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge); + static_cast(free_entries) / lowering_setting); return std::min(max_size, static_cast(data.getStoragePolicy()->getMaxUnreservedFreeSpace() / DISK_USAGE_COEFFICIENT_TO_SELECT)); } @@ -213,6 +220,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, + size_t max_total_size_to_merge_with_ttl, String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); @@ -284,7 +292,9 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( current_time, data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); - parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge); + + parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge_with_ttl); + future_part.merge_type = MergeType::TTL_DELETE; } if (parts_to_merge.empty()) @@ -306,6 +316,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; return false; } + future_part.merge_type = MergeType::NORMAL; } MergeTreeData::DataPartsVector parts; @@ -385,6 +396,12 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); future_part.assign(std::move(parts)); + + if (final) + future_part.merge_type = MergeType::FINAL; + else + future_part.merge_type = MergeType::NORMAL; + available_disk_space -= required_disk_space; return true; } @@ -634,6 +651,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor new_data_part->partition.assign(future_part.getPartition()); new_data_part->is_temp = true; + if (future_part.merge_type == MergeType::TTL_DELETE && ttl_merges_blocker.isCancelled()) + throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); + bool need_remove_expired_values = false; for (const auto & part : parts) new_data_part->ttl_infos.update(part->ttl_infos); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index d5798fe3582..086a2a9cae2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -22,6 +23,7 @@ struct FutureMergedMutatedPart MergeTreeDataPartType type; MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; + MergeType merge_type = MergeType::NORMAL; const MergeTreePartition & getPartition() const { return parts.front()->partition; } @@ -57,12 +59,12 @@ public: /** Get maximum total size of parts to do merge, at current moment of time. * It depends on number of free threads in background_pool and amount of free space in disk. */ - UInt64 getMaxSourcePartsSizeForMerge(); + UInt64 getMaxSourcePartsSizeForMerge(MergeType merge_type); /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ - UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used); + UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type); /** Get maximum total size of parts to do mutation, at current moment of time. * It depends only on amount of free space in disk. @@ -81,6 +83,7 @@ public: bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge, + size_t max_total_size_to_merge_with_ttl, String * out_disable_reason = nullptr); /** Select all the parts in the specified partition for merge, if possible. diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 085c441aa90..e5707ff837c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,8 +33,10 @@ struct Settings; M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl, 14, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ @@ -83,7 +85,7 @@ struct Settings; M(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ M(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ M(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \ - M(Int64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ + M(Int64, merge_with_ttl_timeout, 0, "Minimal time in seconds, when merge with 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, 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) \ diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp new file mode 100644 index 00000000000..b58a0de4093 --- /dev/null +++ b/src/Storages/MergeTree/MergeType.cpp @@ -0,0 +1,27 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +String toString(MergeType merge_type) +{ + switch (merge_type) + { + case MergeType::NORMAL: + return "NORMAL"; + case MergeType::FINAL: + return "FINAL"; + case MergeType::TTL_DELETE: + return "TTL_DELETE"; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); +} + +} diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h new file mode 100644 index 00000000000..5d9abaa61b3 --- /dev/null +++ b/src/Storages/MergeTree/MergeType.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +enum class MergeType +{ + NORMAL, + FINAL, + TTL_DELETE, +}; + +String toString(MergeType merge_type); + +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index af6d980ad98..de8dd7f6097 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -36,6 +36,8 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << s << '\n'; out << "into\n" << new_part_name; out << "\ndeduplicate: " << deduplicate; + if (merge_type != MergeType::NORMAL) + out <<"\nmerge_type: " << static_cast(merge_type); break; case DROP_RANGE: @@ -149,7 +151,18 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) } in >> new_part_name; if (format_version >= 4) + { in >> "\ndeduplicate: " >> deduplicate; + in >> "\n"; + if (in.eof()) + trailing_newline_found = true; + else if (checkString("merge_type: ", in)) + { + UInt64 value; + in >> value; + merge_type = static_cast(value); + } + } } else if (type_str == "drop" || type_str == "detach") { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index ae5fad0b83c..bea796ce015 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -79,6 +80,7 @@ struct ReplicatedMergeTreeLogEntryData Strings source_parts; bool deduplicate = false; /// Do deduplicate on merge + MergeType merge_type = MergeType::NORMAL; String column_name; String index_name; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 8e2c3752212..c9b366a9ec8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1061,7 +1061,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge() + UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type) : merger_mutator.getMaxSourcePartSizeForMutation(); /** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed), * then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size, @@ -1312,21 +1312,26 @@ bool ReplicatedMergeTreeQueue::processEntry( } -std::pair ReplicatedMergeTreeQueue::countMergesAndPartMutations() const +ReplicatedMergeTreeQueue::OperationsInQueue ReplicatedMergeTreeQueue::countMergesAndPartMutations() const { std::lock_guard lock(state_mutex); size_t count_merges = 0; size_t count_mutations = 0; + size_t count_merges_with_ttl = 0; for (const auto & entry : queue) { if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS) + { ++count_merges; + if (entry->merge_type == MergeType::TTL_DELETE) + ++count_merges_with_ttl; + } else if (entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART) ++count_mutations; } - return std::make_pair(count_merges, count_mutations); + return OperationsInQueue{count_merges, count_mutations, count_merges_with_ttl}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 76f84da1ae8..c724701f1ff 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -46,6 +46,13 @@ private: } }; + struct OperationsInQueue + { + size_t merges = 0; + size_t mutations = 0; + size_t merges_with_ttl = 0; + }; + /// To calculate min_unprocessed_insert_time, max_processed_insert_time, for which the replica lag is calculated. using InsertsByTime = std::set; @@ -325,7 +332,7 @@ public: bool processEntry(std::function get_zookeeper, LogEntryPtr & entry, const std::function func); /// Count the number of merges and mutations of single parts in the queue. - std::pair countMergesAndPartMutations() const; + OperationsInQueue countMergesAndPartMutations() const; /// Count the total number of active mutations. size_t countMutations() const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7e4318a32f6..05f2f5254f0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -650,9 +650,14 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { - UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::NORMAL); + UInt64 max_source_parts_size_with_ttl = 0; + + if (!aggressive) + max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::TTL_DELETE); + if (max_source_parts_size > 0) - selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, out_disable_reason); + selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, max_source_parts_size_with_ttl, out_disable_reason); else if (out_disable_reason) *out_disable_reason = "Current value of max_source_parts_size is zero"; } @@ -724,6 +729,7 @@ bool StorageMergeTree::merge( try { + std::cerr << "FUTURE PART MERGE TYPE:" << toString(future_part.merge_type) << std::endl; new_part = merger_mutator.mergePartsToTemporaryPart( future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr), merging_tagger->reserved_space, deduplicate); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6058632d220..1c880c8c790 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2514,31 +2514,38 @@ void StorageReplicatedMergeTree::mergeSelectingTask() /// and in the same time, many small parts could be created and won't be merged. auto merges_and_mutations_queued = queue.countMergesAndPartMutations(); - size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second; + size_t merges_and_mutations_sum = merges_and_mutations_queued.merges + merges_and_mutations_queued.mutations; if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue) { LOG_TRACE(log, "Number of queued merges ({}) and part mutations ({})" " is greater than max_replicated_merges_in_queue ({}), so won't select new parts to merge or mutate.", - merges_and_mutations_queued.first, - merges_and_mutations_queued.second, + merges_and_mutations_queued.merges, + merges_and_mutations_queued.mutations, storage_settings_ptr->max_replicated_merges_in_queue); } else { UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::NORMAL); + + UInt64 max_source_parts_size_for_merge_with_ttl = 0; + if (merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue) + max_source_parts_size_for_merge_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge( + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::TTL_DELETE); + UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr)) + merger_mutator.selectPartsToMerge(future_merged_part, false, + max_source_parts_size_for_merge, merge_pred, max_source_parts_size_for_merge_with_ttl, nullptr)) { create_result = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, - future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion()); + future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion(), future_merged_part.merge_type); } /// If there are many mutations in queue, it may happen, that we cannot enqueue enough merges to merge all new parts else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 - && merges_and_mutations_queued.second < storage_settings_ptr->max_replicated_mutations_in_queue) + && merges_and_mutations_queued.mutations < storage_settings_ptr->max_replicated_mutations_in_queue) { /// Choose a part to mutate. DataPartsVector data_parts = getDataPartsVector(); @@ -2617,7 +2624,8 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c const MergeTreeDataPartType & merged_part_type, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry, - int32_t log_version) + int32_t log_version, + MergeType merge_type) { std::vector> exists_futures; exists_futures.reserve(parts.size()); @@ -2649,6 +2657,7 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c entry.source_replica = replica_name; entry.new_part_name = merged_name; entry.new_part_type = merged_part_type; + entry.merge_type = merge_type; entry.deduplicate = deduplicate; entry.create_time = time(nullptr); @@ -3584,7 +3593,7 @@ bool StorageReplicatedMergeTree::optimize( CreateMergeEntryResult create_result = createLogEntryToMergeParts( zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, - &merge_entry, can_merge.getVersion()); + &merge_entry, can_merge.getVersion(), future_merged_part.merge_type); if (create_result == CreateMergeEntryResult::MissingPart) return handle_noop("Can't create merge queue node in ZooKeeper, because some parts are missing"); @@ -3614,7 +3623,7 @@ bool StorageReplicatedMergeTree::optimize( if (!partition) { selected = merger_mutator.selectPartsToMerge( - future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason); + future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, 0, &disable_reason); } else { @@ -3639,7 +3648,7 @@ bool StorageReplicatedMergeTree::optimize( CreateMergeEntryResult create_result = createLogEntryToMergeParts( zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, - &merge_entry, can_merge.getVersion()); + &merge_entry, can_merge.getVersion(), future_merged_part.merge_type); if (create_result == CreateMergeEntryResult::MissingPart) return handle_noop("Can't create merge queue node in ZooKeeper, because some parts are missing"); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e9395f20f3f..2bc9265331d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -450,7 +450,8 @@ private: const MergeTreeDataPartType & merged_part_type, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry, - int32_t log_version); + int32_t log_version, + MergeType merge_type); CreateMergeEntryResult createLogEntryToMutatePart( const IMergeTreeDataPart & part, diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 39d22bd00ca..b3bd8f77a89 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -30,6 +30,7 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes() {"columns_written", std::make_shared()}, {"memory_usage", std::make_shared()}, {"thread_id", std::make_shared()}, + {"merge_type", std::make_shared()}, }; } @@ -65,6 +66,7 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & res_columns[i++]->insert(merge.columns_written); res_columns[i++]->insert(merge.memory_usage); res_columns[i++]->insert(merge.thread_id); + res_columns[i++]->insert(merge.merge_type); } } From fbb37c37df6c428579130772151492209742008e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 11:28:46 +0300 Subject: [PATCH 138/535] Simplier interface --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 6 +++--- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a0ab7866402..31d566c4e0e 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -158,15 +158,15 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(MergeType merge_type) +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(bool with_ttl) const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); - return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1, merge_type); /// 1 is current thread + return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1, with_ttl); /// 1 is current thread } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type) +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, bool with_ttl) const { if (pool_used > pool_size) throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); @@ -180,7 +180,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz size_t lowering_setting; - if (merge_type == MergeType::TTL_DELETE) + if (with_ttl) lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl; else lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge; @@ -198,7 +198,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() const { const auto data_settings = data.getSettings(); size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 086a2a9cae2..6b0e2e9be22 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -59,17 +59,17 @@ public: /** Get maximum total size of parts to do merge, at current moment of time. * It depends on number of free threads in background_pool and amount of free space in disk. */ - UInt64 getMaxSourcePartsSizeForMerge(MergeType merge_type); + UInt64 getMaxSourcePartsSizeForMerge(bool with_ttl) const; /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ - UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type); + UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, bool with_ttl) const; /** Get maximum total size of parts to do mutation, at current moment of time. * It depends only on amount of free space in disk. */ - UInt64 getMaxSourcePartSizeForMutation(); + UInt64 getMaxSourcePartSizeForMutation() const; /** Selects which parts to merge. Uses a lot of heuristics. * diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c9b366a9ec8..d1b4217401c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1061,7 +1061,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type) + UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type == MergeType::TTL_DELETE) : merger_mutator.getMaxSourcePartSizeForMutation(); /** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed), * then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 05f2f5254f0..07e373ac93c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -650,11 +650,11 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { - UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::NORMAL); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(false); UInt64 max_source_parts_size_with_ttl = 0; if (!aggressive) - max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::TTL_DELETE); + max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge(true); if (max_source_parts_size > 0) selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, max_source_parts_size_with_ttl, out_disable_reason); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1c880c8c790..e01926d39d1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2526,12 +2526,12 @@ void StorageReplicatedMergeTree::mergeSelectingTask() else { UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::NORMAL); + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, false); UInt64 max_source_parts_size_for_merge_with_ttl = 0; if (merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue) max_source_parts_size_for_merge_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::TTL_DELETE); + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, true); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); From 5d39b8ce0478e637ad5ac34b8f1c05b5f1aab3d8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 2 Sep 2020 11:39:16 +0300 Subject: [PATCH 139/535] unnecessary change --- docker/test/stateless_unbundled/Dockerfile | 68 +++------------------- 1 file changed, 8 insertions(+), 60 deletions(-) diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index 4978252d556..7de29fede72 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -1,56 +1,12 @@ # docker build -t yandex/clickhouse-stateless-unbundled-test . -FROM ubuntu:20.04 +FROM yandex/clickhouse-test-base ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz" -ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 - -RUN apt-get update \ - && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ - --yes --no-install-recommends --verbose-versions \ - && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ - && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ - && apt-key add /tmp/llvm-snapshot.gpg.key \ - && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ - && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ - /etc/apt/sources.list - -# initial packages -RUN apt-get update \ - && apt-get install \ - bash \ - fakeroot \ - ccache \ - curl \ - software-properties-common \ - --yes --no-install-recommends - -# Special dpkg-deb (https://github.com/ClickHouse-Extras/dpkg) version which is able -# to compress files using pigz (https://zlib.net/pigz/) instead of gzip. -# Significantly increase deb packaging speed and compatible with old systems -RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb \ - && chmod +x dpkg-deb \ - && cp dpkg-deb /usr/bin - - -RUN apt-get update \ - && apt-get install \ - clang-${LLVM_VERSION} \ - debhelper \ - devscripts \ - gdb \ - git \ - gperf \ - lcov \ - llvm-${LLVM_VERSION} \ - moreutils \ - perl \ - perl \ - pigz \ - pkg-config \ - tzdata \ - alien \ +RUN apt-get --allow-unauthenticated update -y \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get --allow-unauthenticated install --yes --no-install-recommends \ + alien \ brotli \ cmake \ devscripts \ @@ -100,6 +56,8 @@ RUN apt-get update \ pkg-config \ python \ python-lxml \ + python-requests \ + python-termcolor \ qemu-user-static \ sudo \ telnet \ @@ -110,10 +68,7 @@ RUN apt-get update \ wget \ zlib1g-dev \ zookeeper \ - zookeeperd \ - --yes --no-install-recommends - - + zookeeperd RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ @@ -122,13 +77,6 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ && rm -rf /tmp/clickhouse-odbc-tmp -# Sanitizer options -RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ - echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ - echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ - ln -s /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; - - ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone From 2c7d7f9f49bfa33382c9ae7ed81c875aeaeee0ba Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 2 Sep 2020 11:40:31 +0300 Subject: [PATCH 140/535] Update Dockerfile --- docker/test/integration/base/Dockerfile | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 2b8877437f8..53627c78208 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -17,7 +17,6 @@ RUN apt-get update \ odbc-postgresql \ sqlite3 \ curl \ - bind9-host \ tar RUN rm -rf \ /var/lib/apt/lists/* \ From 13e04641381461d7de2997bef7514f826a94a925 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Sep 2020 12:39:49 +0300 Subject: [PATCH 141/535] Stop query execution if exception happened in PipelineExecutor itself. --- src/Processors/Executors/PipelineExecutor.cpp | 27 +++++++++++-------- src/Processors/Executors/PipelineExecutor.h | 3 +++ 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index d445177f28e..c8774394cc3 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -432,6 +432,11 @@ bool PipelineExecutor::executeStep(std::atomic_bool * yield_flag) if (node->exception) std::rethrow_exception(node->exception); + /// Exception which happened in executing thread, but not at processor. + for (auto & executor_context : executor_contexts) + if (executor_context->exception) + std::rethrow_exception(executor_context->exception); + finalizeExecution(); return false; @@ -469,16 +474,7 @@ void PipelineExecutor::wakeUpExecutor(size_t thread_num) void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads) { - try - { - executeStepImpl(thread_num, num_threads); - } - catch (...) - { - /// In case of exception from executor itself, stop other threads. - finish(); - throw; - } + executeStepImpl(thread_num, num_threads); #ifndef NDEBUG auto & context = executor_contexts[thread_num]; @@ -735,7 +731,16 @@ void PipelineExecutor::executeImpl(size_t num_threads) CurrentThread::detachQueryIfNotDetached(); ); - executeSingleThread(thread_num, num_threads); + try + { + executeSingleThread(thread_num, num_threads); + } + catch (...) + { + /// In case of exception from executor itself, stop other threads. + finish(); + executor_contexts[thread_num]->exception = std::current_exception(); + } }); } diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 927b9d891e4..b457cca34b1 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -97,6 +97,9 @@ private: /// Currently processing node. ExecutingGraph::Node * node = nullptr; + /// Exception from executing thread itself. + std::exception_ptr exception; + #ifndef NDEBUG /// Time for different processing stages. UInt64 total_time_ns = 0; From 2180cdc400da23a1bfb3f4ddc34c10e699776a13 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 2 Sep 2020 13:00:49 +0300 Subject: [PATCH 142/535] system_events_show_zero_values setting added --- src/Core/Settings.h | 4 ++-- src/Storages/System/StorageSystemEvents.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8c4f6b8eb6f..b68561ad598 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -379,8 +379,8 @@ class IColumn; M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ - M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ - \ + M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ \ + M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(Bool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \ diff --git a/src/Storages/System/StorageSystemEvents.cpp b/src/Storages/System/StorageSystemEvents.cpp index 6a0992af052..aa442245396 100644 --- a/src/Storages/System/StorageSystemEvents.cpp +++ b/src/Storages/System/StorageSystemEvents.cpp @@ -15,13 +15,13 @@ NamesAndTypesList StorageSystemEvents::getNamesAndTypes() }; } -void StorageSystemEvents::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +void StorageSystemEvents::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { UInt64 value = ProfileEvents::global_counters[i]; - if (0 != value) + if (0 != value || context.getSettingsRef().system_events_show_zero_values)) { res_columns[0]->insert(ProfileEvents::getName(ProfileEvents::Event(i))); res_columns[1]->insert(value); From 6f18cd772c02b694cbc784f064ebb6aa44d96a47 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 2 Sep 2020 13:07:23 +0300 Subject: [PATCH 143/535] fixes --- src/Core/Settings.h | 3 ++- src/Storages/System/StorageSystemEvents.cpp | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b68561ad598..cb1150c17a3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -380,7 +380,8 @@ class IColumn; M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ \ - M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ + M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ \ + \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(Bool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \ diff --git a/src/Storages/System/StorageSystemEvents.cpp b/src/Storages/System/StorageSystemEvents.cpp index aa442245396..a877d7c5265 100644 --- a/src/Storages/System/StorageSystemEvents.cpp +++ b/src/Storages/System/StorageSystemEvents.cpp @@ -21,7 +21,7 @@ void StorageSystemEvents::fillData(MutableColumns & res_columns, const Context & { UInt64 value = ProfileEvents::global_counters[i]; - if (0 != value || context.getSettingsRef().system_events_show_zero_values)) + if (0 != value || context.getSettingsRef().system_events_show_zero_values) { res_columns[0]->insert(ProfileEvents::getName(ProfileEvents::Event(i))); res_columns[1]->insert(value); From 06b38a4d44fc579e1635f201caf0babe1c7c74fe Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 13:31:31 +0300 Subject: [PATCH 144/535] Remove cerr --- src/Interpreters/MutationsInterpreter.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3a397cb9b5a..ef95b25eb98 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -534,14 +534,14 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) stages.emplace_back(context); for (const auto & column : unchanged_columns) { - std::cerr << "ADDING UNCHANGED COLUMN TO STAGE:" << column << std::endl; + //std::cerr << "ADDING UNCHANGED COLUMN TO STAGE:" << column << std::endl; stages.back().column_to_updated.emplace( column, std::make_shared(column)); - std::cerr << "OUTPUT COLUMNS:" << stages.back().output_columns.size() << std::endl; - for (const auto & col : stages.back().output_columns) - { - std::cerr << "OUTPUT COLUMN:" << col << std::endl; - } + //std::cerr << "OUTPUT COLUMNS:" << stages.back().output_columns.size() << std::endl; + //for (const auto & col : stages.back().output_columns) + //{ + // std::cerr << "OUTPUT COLUMN:" << col << std::endl; + //} } } } From 0e4c9ff9a8a8b3dd216305cf558198ff5625fd37 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 2 Sep 2020 14:14:49 +0300 Subject: [PATCH 145/535] Added UUID to Obfuscator --- programs/obfuscator/Obfuscator.cpp | 54 ++++++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index acdab861ea3..321eb82e5a6 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -363,6 +363,20 @@ static void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UI } } +static void transformUUID(const UInt8 * src, UInt8 * dst, size_t size, UInt64 seed) +{ + SipHash hash; + hash.update(seed); + hash.update(reinterpret_cast(src), size); + seed = hash.get64(); + + /// Saving version and variant from an old UUID + hash.get128(reinterpret_cast(dst)); + dst[6] &= 0b00001111; + dst[6] |= src[6] & 0b11110000; + dst[8] &= 0b00011111; + dst[8] |= src[8] & 0b11100000; +} class FixedStringModel : public IModel { @@ -400,6 +414,43 @@ public: } }; +class UUIDModel : public IModel +{ +private: + UInt64 seed; + +public: + explicit UUIDModel(UInt64 seed_) : seed(seed_) {} + + void train(const IColumn &) override {} + void finalize() override {} + + ColumnPtr generate(const IColumn & column) override + { + const ColumnFixedString & column_fixed_string = assert_cast(column); + const size_t string_size = column_fixed_string.getN(); + assert(string_size == 16); + + const auto & src_data = column_fixed_string.getChars(); + size_t size = column_fixed_string.size(); + + auto res_column = ColumnFixedString::create(string_size); + auto & res_data = res_column->getChars(); + + res_data.resize(src_data.size()); + + for (size_t i = 0; i < size; ++i) + transformUUID(&src_data[i * string_size], &res_data[i * string_size], string_size, seed); + + return res_column; + } + + void updateSeed() override + { + seed = hash(seed); + } +}; + /// Leave date part as is and apply pseudorandom permutation to time difference with previous value within the same log2 class. class DateTimeModel : public IModel @@ -935,6 +986,9 @@ public: if (typeid_cast(&data_type)) return std::make_unique(seed); + if (typeid_cast(&data_type)) + return std::make_unique(seed); + if (const auto * type = typeid_cast(&data_type)) return std::make_unique(get(*type->getNestedType(), seed, markov_model_params)); From a04c8bb095eb20145e87c40a004497018e0ea902 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 2 Sep 2020 14:18:24 +0300 Subject: [PATCH 146/535] fixes --- programs/obfuscator/Obfuscator.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 321eb82e5a6..49528c1c2f5 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -372,10 +372,8 @@ static void transformUUID(const UInt8 * src, UInt8 * dst, size_t size, UInt64 se /// Saving version and variant from an old UUID hash.get128(reinterpret_cast(dst)); - dst[6] &= 0b00001111; - dst[6] |= src[6] & 0b11110000; - dst[8] &= 0b00011111; - dst[8] |= src[8] & 0b11100000; + dst[6] = (dst[6] & 0b00001111) | (src[6] & 0b11110000); + dst[8] = (dst[8] & 0b00011111) | (src[8] & 0b11100000); } class FixedStringModel : public IModel From c32c83f6532c29108991570ce4687d4a84c8f51b Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 2 Sep 2020 14:20:20 +0300 Subject: [PATCH 147/535] added include --- programs/obfuscator/Obfuscator.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 49528c1c2f5..4493842c45a 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include From 13ba8cd5a73710b929e227072bb82a1aa3728e69 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 2 Sep 2020 14:25:11 +0300 Subject: [PATCH 148/535] fixed spaces --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cb1150c17a3..4230a6474e8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -379,8 +379,8 @@ class IColumn; M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ - M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ \ - M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ \ + M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ + M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ From 128cb7ce22d9d763da462c5d41dbe90c237718f6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 15:16:12 +0300 Subject: [PATCH 149/535] Don't select already selected parts --- src/Storages/MergeTree/MergeSelector.h | 14 ++--- src/Storages/MergeTree/MergeTreeData.cpp | 59 +++---------------- src/Storages/MergeTree/MergeTreeData.h | 3 - .../MergeTree/MergeTreeDataMergerMutator.cpp | 28 ++++----- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 28 +++++++++ .../MergeTree/MergeTreeDataPartTTLInfo.h | 15 +++-- .../MergeTree/MergeTreePartsMover.cpp | 5 +- src/Storages/MergeTree/MergeType.cpp | 5 ++ src/Storages/MergeTree/MergeType.h | 2 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 4 +- src/Storages/MergeTree/TTLMergeSelector.cpp | 31 ++++++++-- src/Storages/MergeTree/TTLMergeSelector.h | 26 ++++++-- src/Storages/TTLDescription.cpp | 2 +- 13 files changed, 123 insertions(+), 99 deletions(-) diff --git a/src/Storages/MergeTree/MergeSelector.h b/src/Storages/MergeTree/MergeSelector.h index ae2c48fced1..285dc1a3660 100644 --- a/src/Storages/MergeTree/MergeSelector.h +++ b/src/Storages/MergeTree/MergeSelector.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include namespace DB @@ -40,17 +42,9 @@ public: /// Opaque pointer to avoid dependencies (it is not possible to do forward declaration of typedef). const void * data; - /// Minimal time, when we need to delete some data from this part. - time_t min_delete_ttl; + MergeTreeDataPartTTLInfos ttl_infos; - /// Maximum time, when we will need to drop this part altogether because all rows in it are expired. - time_t max_delete_ttl; - - /// Minimal time, when we need to recompress this part. - time_t min_recompress_ttl; - - /// Maximum time, when we need to recompress this part. - time_t max_recompress_ttl; + ASTPtr compression_codec_desc; }; /// Parts are belong to partitions. Only parts within same partition could be merged. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 16a08b180f9..f535a040535 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -507,6 +507,7 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta if (new_table_ttl.definition_ast) { + std::cerr << "MOVE TTL SIZE:" << new_table_ttl.move_ttl.size() << std::endl; for (const auto & move_ttl : new_table_ttl.move_ttl) { if (!getDestinationForTTL(move_ttl)) @@ -2975,9 +2976,11 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ { expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size); + auto metadata_snapshot = getInMemoryMetadataPtr(); ReservationPtr reservation; - auto ttl_entry = selectTTLEntryForTTLInfos(ttl_infos, time_of_move); + auto ttl_entry = selectTTLEntryForTTLInfos(metadata_snapshot->getMoveTTLs(), ttl_infos.moves_ttl, time_of_move, true); + if (ttl_entry) { SpacePtr destination_ptr = getDestinationForTTL(*ttl_entry); @@ -3031,64 +3034,16 @@ bool MergeTreeData::isPartInTTLDestination(const TTLDescription & ttl, const IMe return false; } -std::optional -MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const -{ - time_t max_max_ttl = 0; - TTLDescriptions::const_iterator best_entry_it; - auto metadata_snapshot = getInMemoryMetadataPtr(); - - const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs(); - for (auto ttl_entry_it = move_ttl_entries.begin(); ttl_entry_it != move_ttl_entries.end(); ++ttl_entry_it) - { - auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry_it->result_column); - /// Prefer TTL rule which went into action last. - if (ttl_info_it != ttl_infos.moves_ttl.end() - && ttl_info_it->second.max <= time_of_move - && max_max_ttl <= ttl_info_it->second.max) - { - best_entry_it = ttl_entry_it; - max_max_ttl = ttl_info_it->second.max; - } - } - - return max_max_ttl ? *best_entry_it : std::optional(); -} - - CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const { - time_t max_max_ttl = 0; - TTLDescriptions::const_iterator best_entry_it; auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); - //std::cerr << "RECOMPRESSION ENTRIES SIZE:" << recompression_ttl_entries.size() << std::endl; - for (auto ttl_entry_it = recompression_ttl_entries.begin(); ttl_entry_it != recompression_ttl_entries.end(); ++ttl_entry_it) - { - //std::cerr << "RECOMPRESSION TTL SIZE:" << ttl_infos.recompression_ttl.size() << std::endl; - auto ttl_info_it = ttl_infos.recompression_ttl.find(ttl_entry_it->result_column); - /// Prefer TTL rule which went into action last. - if (ttl_info_it != ttl_infos.recompression_ttl.end() - && ttl_info_it->second.max <= current_time - && max_max_ttl <= ttl_info_it->second.max) - { - best_entry_it = ttl_entry_it; - max_max_ttl = ttl_info_it->second.max; - } - } + auto best_ttl_entry = selectTTLEntryForTTLInfos(recompression_ttl_entries, ttl_infos.recompression_ttl, current_time, false); - if (max_max_ttl) - { - //std::cerr << "BEST ENTRY FOUND, MAX MAX:" << max_max_ttl << std::endl; - //std::cerr << "RECOMPRESSION IS NULLPTR:" << (best_entry_it->recompression_codec == nullptr) << std::endl; - return CompressionCodecFactory::instance().get(best_entry_it->recompression_codec, {}); - } - //else - //{ - // std::cerr << "NOT FOUND NEW RECOMPRESSION\n"; - //} + if (best_ttl_entry) + return CompressionCodecFactory::instance().get(best_ttl_entry->recompression_codec, {}); return global_context.chooseCompressionCodec( part_size_compressed, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ab115927e1e..14cefe9af1d 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -667,9 +667,6 @@ public: ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; - std::optional selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; - - CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const; /// Limiting parallel sends per one table, used in DataPartsExchange diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ce860a5b590..11bc6bbd46d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -225,6 +225,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); const auto data_settings = data.getSettings(); + auto metadata_snapshot = data.getInMemoryMetadataPtr(); if (data_parts.empty()) { @@ -268,10 +269,8 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( part_info.age = current_time - part->modification_time; part_info.level = part->info.level; part_info.data = ∂ - part_info.min_delete_ttl = part->ttl_infos.part_min_ttl; - part_info.max_delete_ttl = part->ttl_infos.part_max_ttl; - part_info.min_recompress_ttl = part->ttl_infos.getMinRecompressionTTL(); - part_info.max_recompress_ttl = part->ttl_infos.getMaxRecompressionTTL(); + part_info.ttl_infos = part->ttl_infos; + part_info.compression_codec_desc = part->default_codec->getCodecDesc(); partitions.back().emplace_back(part_info); @@ -287,7 +286,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( IMergeSelector::PartsInPartition parts_to_merge; - if (!ttl_merges_blocker.isCancelled()) + if (!ttl_merges_blocker.isCancelled() && metadata_snapshot->hasAnyTTL()) { TTLDeleteMergeSelector delete_ttl_selector( next_ttl_merge_times_by_partition, @@ -298,12 +297,13 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( parts_to_merge = delete_ttl_selector.select(partitions, max_total_size_to_merge_with_ttl); if (!parts_to_merge.empty()) future_part.merge_type = MergeType::TTL_DELETE; - else + else if (metadata_snapshot->hasAnyRecompressionTTL()) { TTLRecompressMergeSelector recompress_ttl_selector( next_ttl_merge_times_by_partition, current_time, - data_settings->merge_with_ttl_timeout); + data_settings->merge_with_ttl_timeout, + metadata_snapshot->getRecompressionTTLs()); parts_to_merge = recompress_ttl_selector.select(partitions, max_total_size_to_merge_with_ttl); if (!parts_to_merge.empty()) @@ -665,7 +665,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor new_data_part->partition.assign(future_part.getPartition()); new_data_part->is_temp = true; - if (future_part.merge_type == MergeType::TTL_DELETE && ttl_merges_blocker.isCancelled()) + if (isTTLMergeType(future_part.merge_type) && ttl_merges_blocker.isCancelled()) throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); bool need_remove_expired_values = false; @@ -840,8 +840,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (deduplicate) merged_stream = std::make_shared(merged_stream, sort_description, SizeLimits(), 0 /*limit_hint*/, Names()); - if (need_remove_expired_values) - merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, false); + if (need_remove_expired_values || (future_part.merge_type == MergeType::FINAL && !ttl_merges_blocker.isCancelled())) + merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, future_part.merge_type == MergeType::FINAL); if (metadata_snapshot->hasSecondaryIndices()) @@ -1123,19 +1123,19 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) { - std::cerr << "GOING TO MATERIALIZE TTL\n"; + //std::cerr << "GOING TO MATERIALIZE TTL\n"; need_remove_expired_values = true; } else { - std::cerr << "NOT GOING TO MATERIALIZE TTL\n"; - std::cerr << "IN IS NULL:" << (in == nullptr) << std::endl; + //std::cerr << "NOT GOING TO MATERIALIZE TTL\n"; + //std::cerr << "IN IS NULL:" << (in == nullptr) << std::endl; } /// All columns from part are changed and may be some more that were missing before in part if (!isWidePart(source_part) || (interpreter && interpreter->isAffectingAllColumns())) { - std::cerr << "MUTATING ALL PART COLUMNS\n"; + //std::cerr << "MUTATING ALL PART COLUMNS\n"; /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex /// (which is locked in data.getTotalActiveSizeInBytes()) /// (which is locked in shared mode when input streams are created) and when inserting new data diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 0664d3c5df0..42fc4be0fa5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -186,4 +186,32 @@ time_t MergeTreeDataPartTTLInfos::getMaxRecompressionTTL() const return max; } + +std::optional selectTTLEntryForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max) +{ + time_t best_ttl_time = 0; + TTLDescriptions::const_iterator best_entry_it; + for (auto ttl_entry_it = descriptions.begin(); ttl_entry_it != descriptions.end(); ++ttl_entry_it) + { + auto ttl_info_it = ttl_info_map.find(ttl_entry_it->result_column); + time_t ttl_time; + + if (use_max) + ttl_time = ttl_info_it->second.max; + else + ttl_time = ttl_info_it->second.min; + + /// Prefer TTL rule which went into action last. + if (ttl_info_it != ttl_info_map.end() + && ttl_time <= current_time + && best_ttl_time <= ttl_time) + { + best_entry_it = ttl_entry_it; + best_ttl_time = ttl_time; + } + } + + return best_ttl_time ? *best_entry_it : std::optional(); +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 0f46b4f97e8..d0738053d1d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include @@ -30,11 +31,13 @@ struct MergeTreeDataPartTTLInfo } }; +/// Order is important as it would be serialized and hashed for checksums +using TTLInfoMap = std::map; + /// PartTTLInfo for all columns and table with minimal ttl for whole part struct MergeTreeDataPartTTLInfos { - /// Order is important as it would be serialized and hashed for checksums - std::map columns_ttl; + TTLInfoMap columns_ttl; MergeTreeDataPartTTLInfo table_ttl; /// `part_min_ttl` and `part_max_ttl` are TTLs which are used for selecting parts @@ -42,11 +45,9 @@ struct MergeTreeDataPartTTLInfos time_t part_min_ttl = 0; time_t part_max_ttl = 0; - /// Order is important as it would be serialized and hashed for checksums - std::map moves_ttl; + TTLInfoMap moves_ttl; - /// Order is important as it would be serialized and hashed for checksums - std::map recompression_ttl; + TTLInfoMap recompression_ttl; time_t getMinRecompressionTTL() const; time_t getMaxRecompressionTTL() const; @@ -70,4 +71,6 @@ struct MergeTreeDataPartTTLInfos } }; +std::optional selectTTLEntryForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max); + } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index e84ff418bc3..92ea745c5df 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -121,6 +121,8 @@ bool MergeTreePartsMover::selectPartsForMove( time_t time_of_move = time(nullptr); + auto metadata_snapshot = data->getInMemoryMetadataPtr(); + for (const auto & part : data_parts) { String reason; @@ -128,7 +130,8 @@ bool MergeTreePartsMover::selectPartsForMove( if (!can_move(part, &reason)) continue; - auto ttl_entry = data->selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move); + auto ttl_entry = selectTTLEntryForTTLInfos(metadata_snapshot->getMoveTTLs(), part->ttl_infos.moves_ttl, time_of_move, true); + auto to_insert = need_to_move.find(part->volume->getDisk()); ReservationPtr reservation; if (ttl_entry) diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp index 69732877ad3..875a0a93f6b 100644 --- a/src/Storages/MergeTree/MergeType.cpp +++ b/src/Storages/MergeTree/MergeType.cpp @@ -26,4 +26,9 @@ String toString(MergeType merge_type) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); } +bool isTTLMergeType(MergeType merge_type) +{ + return merge_type == MergeType::TTL_DELETE || merge_type == MergeType::TTL_RECOMPRESS; +} + } diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index 2bc88352bf2..0f4a0043c54 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -15,4 +15,6 @@ enum class MergeType String toString(MergeType merge_type); +bool isTTLMergeType(MergeType merge_type); + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d1b4217401c..206d8f93038 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1061,7 +1061,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type == MergeType::TTL_DELETE) + UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(isTTLMergeType(entry.merge_type)) : merger_mutator.getMaxSourcePartSizeForMutation(); /** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed), * then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size, @@ -1324,7 +1324,7 @@ ReplicatedMergeTreeQueue::OperationsInQueue ReplicatedMergeTreeQueue::countMerge if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS) { ++count_merges; - if (entry->merge_type == MergeType::TTL_DELETE) + if (isTTLMergeType(entry->merge_type)) ++count_merges_with_ttl; } else if (entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART) diff --git a/src/Storages/MergeTree/TTLMergeSelector.cpp b/src/Storages/MergeTree/TTLMergeSelector.cpp index 5c2d22ab11c..1bc5d563936 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.cpp +++ b/src/Storages/MergeTree/TTLMergeSelector.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -39,7 +40,7 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( { time_t ttl = getTTLForPart(*part_it); - if (ttl && (partition_to_merge_index == -1 || ttl < partition_to_merge_min_ttl)) + if (ttl && !isTTLAlreadySatisfied(*part_it) && (partition_to_merge_index == -1 || ttl < partition_to_merge_min_ttl)) { partition_to_merge_min_ttl = ttl; partition_to_merge_index = i; @@ -59,7 +60,7 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( { time_t ttl = getTTLForPart(*best_begin); - if (!ttl || ttl > current_time + if (!ttl || isTTLAlreadySatisfied(*best_begin) || ttl > current_time || (max_total_size_to_merge && total_size > max_total_size_to_merge)) { ++best_begin; @@ -77,7 +78,7 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( { time_t ttl = getTTLForPart(*best_end); - if (!ttl || ttl > current_time + if (!ttl || isTTLAlreadySatisfied(*best_end) || ttl > current_time || (max_total_size_to_merge && total_size > max_total_size_to_merge)) break; @@ -93,12 +94,32 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( time_t TTLDeleteMergeSelector::getTTLForPart(const IMergeSelector::Part & part) const { - return only_drop_parts ? part.max_delete_ttl : part.min_delete_ttl; + return only_drop_parts ? part.ttl_infos.part_max_ttl : part.ttl_infos.part_min_ttl; } time_t TTLRecompressMergeSelector::getTTLForPart(const IMergeSelector::Part & part) const { - return part.min_recompress_ttl; + return part.ttl_infos.getMinRecompressionTTL(); +} + +bool TTLRecompressMergeSelector::isTTLAlreadySatisfied(const IMergeSelector::Part & part) const +{ + if (recompression_ttls.empty()) + return false; + + auto ttl_description = selectTTLEntryForTTLInfos(recompression_ttls, part.ttl_infos.recompression_ttl, current_time, false); + + if (!ttl_description) + return true; + + auto ast_to_str = [](ASTPtr query) -> String + { + if (!query) + return ""; + return queryToString(query); + }; + + return ast_to_str(ttl_description->recompression_codec) == ast_to_str(part.compression_codec_desc); } } diff --git a/src/Storages/MergeTree/TTLMergeSelector.h b/src/Storages/MergeTree/TTLMergeSelector.h index a7380aa87c9..de4cbc11a57 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.h +++ b/src/Storages/MergeTree/TTLMergeSelector.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -21,9 +22,9 @@ public: using PartitionIdToTTLs = std::map; ITTLMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_) - : merge_due_times(merge_due_times_), - current_time(current_time_), - merge_cooldown_time(merge_cooldown_time_) + : current_time(current_time_) + , merge_due_times(merge_due_times_) + , merge_cooldown_time(merge_cooldown_time_) { } @@ -32,10 +33,13 @@ public: const size_t max_total_size_to_merge) override; virtual time_t getTTLForPart(const IMergeSelector::Part & part) const = 0; + virtual bool isTTLAlreadySatisfied(const IMergeSelector::Part & part) const = 0; + +protected: + time_t current_time; private: PartitionIdToTTLs & merge_due_times; - time_t current_time; Int64 merge_cooldown_time; }; @@ -51,6 +55,11 @@ public: time_t getTTLForPart(const IMergeSelector::Part & part) const override; + bool isTTLAlreadySatisfied(const IMergeSelector::Part &) const override + { + return false; + } + private: bool only_drop_parts; }; @@ -58,9 +67,16 @@ private: class TTLRecompressMergeSelector : public ITTLMergeSelector { public: - using ITTLMergeSelector::ITTLMergeSelector; + TTLRecompressMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_, const TTLDescriptions & recompression_ttls_) + : ITTLMergeSelector(merge_due_times_, current_time_, merge_cooldown_time_) + , recompression_ttls(recompression_ttls_) + {} time_t getTTLForPart(const IMergeSelector::Part & part) const override; + + bool isTTLAlreadySatisfied(const IMergeSelector::Part & part) const override; +private: + TTLDescriptions recompression_ttls; }; } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index bc634fe67bd..07173d61ece 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -328,7 +328,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( for (const auto & ttl_element_ptr : definition_ast->children) { auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); - if (ttl.mode == TTLMode::DELETE) + if (ttl.mode == TTLMode::DELETE || ttl.mode == TTLMode::GROUP_BY) { if (seen_delete_ttl) throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); From 4c039002f9d33873291e8a5aa41e37a2066c6394 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 2 Sep 2020 15:27:15 +0300 Subject: [PATCH 150/535] Update Obfuscator.cpp --- programs/obfuscator/Obfuscator.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 4493842c45a..ba9ed6d3689 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -369,7 +369,6 @@ static void transformUUID(const UInt8 * src, UInt8 * dst, size_t size, UInt64 se SipHash hash; hash.update(seed); hash.update(reinterpret_cast(src), size); - seed = hash.get64(); /// Saving version and variant from an old UUID hash.get128(reinterpret_cast(dst)); From c009ace7a20f0f84f5335022cd14eab2a9977234 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 2 Sep 2020 15:28:29 +0300 Subject: [PATCH 151/535] Update StorageSystemEvents.cpp --- src/Storages/System/StorageSystemEvents.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/System/StorageSystemEvents.cpp b/src/Storages/System/StorageSystemEvents.cpp index a877d7c5265..ddb00659473 100644 --- a/src/Storages/System/StorageSystemEvents.cpp +++ b/src/Storages/System/StorageSystemEvents.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include From 8f50a6769d55b5aea51b8bfe103ff64db7a59600 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 2 Sep 2020 15:28:47 +0300 Subject: [PATCH 152/535] Fix bad merge --- tests/integration/helpers/cluster.py | 39 +++++++++++++++++++++------- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index fc34a7d7373..6209f45b86c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -40,7 +40,6 @@ SANITIZER_SIGN = "==================" def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): full_path = os.path.join(path, fname) with open(full_path, 'w') as f: - f.write('TSAN_OPTIONS="external_symbolizer_path=/usr/bin/llvm-symbolizer"\n') for var, value in variables.items(): f.write("=".join([var, value]) + "\n") return full_path @@ -192,13 +191,36 @@ class ClickHouseCluster: tag = self.docker_base_tag instance = ClickHouseInstance( - self, self.base_dir, name, base_config_dir if base_config_dir else self.base_config_dir, - main_configs or [], user_configs or [], dictionaries or [], macros or {}, with_zookeeper, - self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, - self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, - env_variables=env_variables or {}, image=image, tag=tag, stay_alive=stay_alive, ipv4_address=ipv4_address, + cluster=self, + base_path=self.base_dir, + name=name, + base_config_dir=base_config_dir if base_config_dir else self.base_config_dir, + custom_main_configs=main_configs or [], + custom_user_configs=user_configs or [], + custom_dictionaries=dictionaries or [], + macros=macros or {}, + with_zookeeper=with_zookeeper, + zookeeper_config_path=self.zookeeper_config_path, + with_mysql=with_mysql, + with_kafka=with_kafka, + with_rabbitmq=with_rabbitmq, + with_mongo=with_mongo, + with_redis=with_redis, + with_minio=with_minio, + with_cassandra=with_cassandra, + server_bin_path=self.server_bin_path, + odbc_bridge_bin_path=self.odbc_bridge_bin_path, + clickhouse_path_dir=clickhouse_path_dir, + with_odbc_drivers=with_odbc_drivers, + hostname=hostname, + env_variables=env_variables or {}, + image=image, + tag=tag, + stay_alive=stay_alive, + ipv4_address=ipv4_address, ipv6_address=ipv6_address, - with_installed_binary=with_installed_binary, tmpfs=tmpfs or []) + with_installed_binary=with_installed_binary, + tmpfs=tmpfs or []) docker_compose_yml_dir = get_docker_compose_path() @@ -769,8 +791,7 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, - with_cassandra, server_bin_path, base_config_dir, - clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, + with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag="latest", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None): From 09850dbdbc3e2fb5b0150a74d06f6cbcf473d371 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 2 Sep 2020 15:39:34 +0300 Subject: [PATCH 153/535] Update ASTColumnsTransformers.cpp --- src/Parsers/ASTColumnsTransformers.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 29bc8420066..2625a03830b 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -1,3 +1,4 @@ +#include #include "ASTColumnsTransformers.h" #include #include From df668d62978b9c9033b1c6b5d1968ac5e1cfda4e Mon Sep 17 00:00:00 2001 From: antikvist Date: Thu, 18 Jun 2020 18:21:19 +0300 Subject: [PATCH 154/535] Rank Correlation (cherry picked from commit 3009c3885ba6317e2a1518f94a1de92e1ef2b6ed) (cherry picked from commit 024ff439e71f7164602f3e193a0c896dc1fa3fe7) --- .../AggregateFunctionRankCorr.cpp | 59 ++++ .../AggregateFunctionRankCorr.h | 290 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 1 + .../registerAggregateFunctions.h | 1 + 4 files changed, 351 insertions(+) create mode 100644 src/AggregateFunctions/AggregateFunctionRankCorr.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionRankCorr.h diff --git a/src/AggregateFunctions/AggregateFunctionRankCorr.cpp b/src/AggregateFunctions/AggregateFunctionRankCorr.cpp new file mode 100644 index 00000000000..d3e7ecccee2 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionRankCorr.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include "registerAggregateFunctions.h" + +#include +#include + + +namespace ErrorCodes +{ +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int NOT_IMPLEMENTED; +} + +namespace DB +{ + +namespace +{ + +AggregateFunctionPtr createAggregateFunctionRankCorr(const std::string & name, + const DataTypes & argument_types, + const Array & parameters) +{ + assertBinary(name, argument_types); + assertNoParameters(name, parameters); + + AggregateFunctionPtr res; + + if (isDecimal(argument_types[0]) || isDecimal(argument_types[1])) + { + throw Exception("Aggregate function " + name + " only supports numerical types.", ErrorCodes::NOT_IMPLEMENTED); + } + + else + { + res.reset(createWithTwoNumericTypes(*argument_types[0], *argument_types[1], + argument_types)); + } + + + if (!res) + { + throw Exception("Aggregate function " + name + " only supports numerical types.", ErrorCodes::NOT_IMPLEMENTED); + } + + return res; +} + +} + + +void registerAggregateFunctionRankCorr(AggregateFunctionFactory & factory) +{ + factory.registerFunction("RankCorr", createAggregateFunctionRankCorr, AggregateFunctionFactory::CaseInsensitive); +} + +} \ No newline at end of file diff --git a/src/AggregateFunctions/AggregateFunctionRankCorr.h b/src/AggregateFunctions/AggregateFunctionRankCorr.h new file mode 100644 index 00000000000..4d59f3dea16 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionRankCorr.h @@ -0,0 +1,290 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +#include +#include + +#include + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace DB +{ + +template