From d6434f61dc7b08072862d4d10ea6fa9da781b6c1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Jul 2020 03:15:02 +0300 Subject: [PATCH 001/402] 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 002/402] 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 003/402] 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 004/402] 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 005/402] 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 006/402] 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 007/402] 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 008/402] 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 009/402] 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 010/402] 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 011/402] 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 012/402] 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 013/402] 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 014/402] 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 015/402] 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 016/402] 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 017/402] 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 018/402] 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 019/402] 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 020/402] 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 021/402] 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 022/402] 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 023/402] 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 024/402] 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 025/402] 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 026/402] 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 027/402] 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 028/402] 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 029/402] 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 030/402] 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 031/402] 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 032/402] 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 033/402] 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 034/402] 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 035/402] 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 036/402] 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 037/402] 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 038/402] 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 039/402] 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 040/402] 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 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 041/402] 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 042/402] 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 1e7eb494812ffb48f77e6cb58e1f9f323050aa11 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 15 Aug 2020 14:38:29 +0000 Subject: [PATCH 043/402] 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 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 044/402] 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 045/402] 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 046/402] 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 047/402] 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 048/402] 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 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 049/402] 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 4fecfdbe2f85c6abe1f3f68843f415885618411c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 26 Aug 2020 08:54:29 +0000 Subject: [PATCH 050/402] 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 4e0c61972109f7c9ffd6962b37e3652e7201bfd8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 28 Aug 2020 08:52:02 +0000 Subject: [PATCH 051/402] 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 647cf5718ed9b76c72413b699930cb448f1627c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Aug 2020 09:12:36 +0000 Subject: [PATCH 052/402] 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 053/402] 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 054/402] 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 055/402] 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 056/402] 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 057/402] 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 058/402] 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 c2fb72abf8ec372df0dfe23af3223d74274708aa Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Aug 2020 16:34:16 +0000 Subject: [PATCH 059/402] 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 060/402] 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 061/402] 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 062/402] 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 063/402] 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 064/402] 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 065/402] 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 f0dc5a30853ff4b40d0097f07068fbb0f49eb714 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Sep 2020 13:49:53 +0300 Subject: [PATCH 066/402] 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 067/402] 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 068/402] 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 ac5877e601714450a369062abbf80f84485bc6f5 Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 00:58:39 +0800 Subject: [PATCH 069/402] 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 6682c62a905cca886bb26c8856c0243420635248 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Sep 2020 14:11:34 +0000 Subject: [PATCH 070/402] 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 a80c1adee81631f770f642ad4430a8ff44ff46af Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 12:05:02 +0800 Subject: [PATCH 071/402] 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 b08056fa8c0f84670bab96b5643dd36850db0d8a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 11:18:50 +0300 Subject: [PATCH 072/402] 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 073/402] 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 13e04641381461d7de2997bef7514f826a94a925 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Sep 2020 12:39:49 +0300 Subject: [PATCH 074/402] 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 075/402] 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 076/402] 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 077/402] 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 078/402] 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 079/402] 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 080/402] 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 081/402] 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 082/402] 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 083/402] 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 084/402] 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 085/402] 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 df668d62978b9c9033b1c6b5d1968ac5e1cfda4e Mon Sep 17 00:00:00 2001 From: antikvist Date: Thu, 18 Jun 2020 18:21:19 +0300 Subject: [PATCH 086/402] 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